[ 
https://issues.apache.org/jira/browse/STORM-1279?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15192368#comment-15192368
 ] 

ASF GitHub Bot commented on STORM-1279:
---------------------------------------

Github user abhishekagarwal87 commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1184#discussion_r55938008
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/supervisor/SupervisorManger.java ---
    @@ -0,0 +1,97 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.storm.daemon.supervisor;
    +
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.utils.Utils;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Collection;
    +import java.util.Map;
    +
    +public class SupervisorManger implements SupervisorDaemon, DaemonCommon, 
Runnable {
    +
    +    private static final Logger LOG = 
LoggerFactory.getLogger(SupervisorManger.class);
    +    private final EventManager eventManager;
    +    private final EventManager processesEventManager;
    +    private SupervisorData supervisorData;
    +
    +    public SupervisorManger(SupervisorData supervisorData, EventManager 
eventManager, EventManager processesEventManager) {
    +        this.eventManager = eventManager;
    +        this.supervisorData = supervisorData;
    +        this.processesEventManager = processesEventManager;
    +    }
    +
    +    public void shutdown() {
    +        LOG.info("Shutting down supervisor{}", 
supervisorData.getSupervisorId());
    +        supervisorData.setActive(false);
    +        try {
    +            supervisorData.getHeartbeatTimer().close();
    +            supervisorData.getEventTimer().close();
    +            supervisorData.getBlobUpdateTimer().close();
    +            eventManager.close();
    +            processesEventManager.close();
    +        } catch (Exception e) {
    +            throw Utils.wrapInRuntime(e);
    +        }
    +        supervisorData.getStormClusterState().disconnect();
    +    }
    +
    +    @Override
    +    public void shutdownAllWorkers() {
    +
    +        Collection<String> workerIds = 
SupervisorUtils.supervisorWorkerIds(supervisorData.getConf());
    +        try {
    +            for (String workerId : workerIds) {
    +                SupervisorUtils.shutWorker(supervisorData, workerId);
    +            }
    +        } catch (Exception e) {
    +            LOG.error("shutWorker failed");
    +            throw Utils.wrapInRuntime(e);
    +        }
    +    }
    +
    +    @Override
    +    public Map getConf() {
    +        return supervisorData.getConf();
    +    }
    +
    +    @Override
    +    public String getId() {
    +        return supervisorData.getSupervisorId();
    +    }
    +
    +    @Override
    +    public boolean isWaiting() {
    +        if (!supervisorData.isActive()) {
    +            return true;
    +        }
    +
    +        if (supervisorData.getHeartbeatTimer().isTimerWaiting() && 
supervisorData.getEventTimer().isTimerWaiting() && eventManager.waiting()
    +                && processesEventManager.waiting()) {
    +            return true;
    +        }
    +        return false;
    +    }
    +
    +    public void run() {
    --- End diff --
    
    it is not very intuitive that SupervisorManager is also a shutdown hook. 
You can remove this from here and in the use an anonymous shutdown hook


> port backtype.storm.daemon.supervisor to java
> ---------------------------------------------
>
>                 Key: STORM-1279
>                 URL: https://issues.apache.org/jira/browse/STORM-1279
>             Project: Apache Storm
>          Issue Type: New Feature
>          Components: storm-core
>            Reporter: Robert Joseph Evans
>            Assignee: John Fang
>              Labels: java-migration, jstorm-merger
>         Attachments: Discussion about supervisor.pdf
>
>
> https://github.com/apache/storm/tree/jstorm-import/jstorm-core/src/main/java/com/alibaba/jstorm/daemon/supervisor
>  as an example
> backtype.storm.event usage should be replaced with built-in java threadpools.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to