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

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

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

    https://github.com/apache/storm/pull/1184#discussion_r57228512
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/daemon/supervisor/SyncSupervisorEvent.java 
---
    @@ -0,0 +1,632 @@
    +/**
    + * 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.commons.io.FileUtils;
    +import org.apache.storm.Config;
    +import org.apache.storm.blobstore.BlobStore;
    +import org.apache.storm.blobstore.ClientBlobStore;
    +import org.apache.storm.cluster.IStateStorage;
    +import org.apache.storm.cluster.IStormClusterState;
    +import org.apache.storm.event.EventManager;
    +import org.apache.storm.generated.*;
    +import org.apache.storm.localizer.LocalResource;
    +import org.apache.storm.localizer.LocalizedResource;
    +import org.apache.storm.localizer.Localizer;
    +import org.apache.storm.utils.*;
    +import org.apache.thrift.transport.TTransportException;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.File;
    +import java.io.FileOutputStream;
    +import java.io.IOException;
    +import java.net.JarURLConnection;
    +import java.net.URL;
    +import java.nio.file.Files;
    +import java.nio.file.StandardCopyOption;
    +import java.util.*;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +public class SyncSupervisorEvent implements Runnable {
    +
    +    private static final Logger LOG = 
LoggerFactory.getLogger(SyncSupervisorEvent.class);
    +
    +    private EventManager syncSupEventManager;
    +    private EventManager syncProcessManager;
    +    private IStormClusterState stormClusterState;
    +    private LocalState localState;
    +    private SyncProcessEvent syncProcesses;
    +    private SupervisorData supervisorData;
    +
    +    public SyncSupervisorEvent(SupervisorData supervisorData, 
SyncProcessEvent syncProcesses, EventManager syncSupEventManager,
    +            EventManager syncProcessManager) {
    +
    +        this.syncProcesses = syncProcesses;
    +        this.syncSupEventManager = syncSupEventManager;
    +        this.syncProcessManager = syncProcessManager;
    +        this.stormClusterState = supervisorData.getStormClusterState();
    +        this.localState = supervisorData.getLocalState();
    +        this.supervisorData = supervisorData;
    +    }
    +
    +    @Override
    +    public void run() {
    +        try {
    +            Map conf = supervisorData.getConf();
    +            Runnable syncCallback = new EventManagerPushCallback(this, 
syncSupEventManager);
    +            List<String> stormIds = 
stormClusterState.assignments(syncCallback);
    +            Map<String, Map<String, Object>> assignmentsSnapshot =
    +                    getAssignmentsSnapshot(stormClusterState, stormIds, 
supervisorData.getAssignmentVersions().get(), syncCallback);
    +            Map<String, List<ProfileRequest>> stormIdToProfilerActions = 
getProfileActions(stormClusterState, stormIds);
    +
    +            Set<String> allDownloadedTopologyIds = 
SupervisorUtils.readDownLoadedStormIds(conf);
    +            Map<String, String> stormcodeMap = 
readStormCodeLocations(assignmentsSnapshot);
    +            Map<Integer, LocalAssignment> existingAssignment = 
localState.getLocalAssignmentsMap();
    +            if (existingAssignment == null) {
    +                existingAssignment = new HashMap<>();
    +            }
    +
    +            Map<Integer, LocalAssignment> allAssignment =
    +                    readAssignments(assignmentsSnapshot, 
existingAssignment, supervisorData.getAssignmentId(), 
supervisorData.getSyncRetry());
    +
    +            Map<Integer, LocalAssignment> newAssignment = new HashMap<>();
    +            Set<String> assignedStormIds = new HashSet<>();
    +
    +            for (Map.Entry<Integer, LocalAssignment> entry : 
allAssignment.entrySet()) {
    +                if 
(supervisorData.getiSupervisor().confirmAssigned(entry.getKey())) {
    +                    newAssignment.put(entry.getKey(), entry.getValue());
    +                    
assignedStormIds.add(entry.getValue().get_topology_id());
    +                }
    +            }
    +
    +            Set<String> srashStormIds = verifyDownloadedFiles(conf, 
supervisorData.getLocalizer(), assignedStormIds, allDownloadedTopologyIds);
    +            Set<String> downloadedStormIds = new HashSet<>();
    +            downloadedStormIds.addAll(allDownloadedTopologyIds);
    +            downloadedStormIds.removeAll(srashStormIds);
    +
    +            LOG.debug("Synchronizing supervisor");
    +            LOG.debug("Storm code map: {}", stormcodeMap);
    +            LOG.debug("All assignment: {}", allAssignment);
    +            LOG.debug("New assignment: {}", newAssignment);
    +            LOG.debug("Assigned Storm Ids {}", assignedStormIds);
    +            LOG.debug("All Downloaded Ids {}", allDownloadedTopologyIds);
    +            LOG.debug("Checked Downloaded Ids {}", srashStormIds);
    +            LOG.debug("Downloaded Ids {}", downloadedStormIds);
    +            LOG.debug("Storm Ids Profiler Actions {}", 
stormIdToProfilerActions);
    +
    +            // download code first
    +            // This might take awhile
    +            // - should this be done separately from usual monitoring?
    +            // should we only download when topology is assigned to this 
supervisor?
    +            for (Map.Entry<String, String> entry : 
stormcodeMap.entrySet()) {
    +                String stormId = entry.getKey();
    +                if (!downloadedStormIds.contains(stormId) && 
assignedStormIds.contains(stormId)) {
    +                    LOG.info("Downloading code for storm id {}.", stormId);
    +                    try {
    +                        downloadStormCode(conf, stormId, entry.getValue(), 
supervisorData.getLocalizer());
    +                    } catch (Exception e) {
    +                        if 
(Utils.exceptionCauseIsInstanceOf(NimbusLeaderNotFoundException.class, e)) {
    +                            LOG.warn("Nimbus leader was not available.", 
e);
    +                        } else if 
(Utils.exceptionCauseIsInstanceOf(TTransportException.class, e)) {
    +                            LOG.warn("There was a connection problem with 
nimbus.", e);
    +                        } else {
    +                            throw e;
    +                        }
    +                    }
    +                    LOG.info("Finished downloading code for storm id {}", 
stormId);
    +                }
    +            }
    +
    +            LOG.debug("Writing new assignment {}", newAssignment);
    +
    +            Set<Integer> killWorkers = new HashSet<>();
    +            killWorkers.addAll(existingAssignment.keySet());
    +            killWorkers.removeAll(newAssignment.keySet());
    +            for (Integer port : killWorkers) {
    +                supervisorData.getiSupervisor().killedWorker(port);
    +            }
    +
    +            killExistingWorkersWithChangeInComponents(supervisorData, 
existingAssignment, newAssignment);
    +
    +            
supervisorData.getiSupervisor().assigned(newAssignment.keySet());
    +            localState.setLocalAssignmentsMap(newAssignment);
    +            supervisorData.setAssignmentVersions(assignmentsSnapshot);
    +            
supervisorData.setStormIdToProfileActions(stormIdToProfilerActions);
    +
    +            Map<Long, LocalAssignment> convertNewAssignment = new 
HashMap<>();
    +            for (Map.Entry<Integer, LocalAssignment> entry : 
newAssignment.entrySet()) {
    +                convertNewAssignment.put(entry.getKey().longValue(), 
entry.getValue());
    +            }
    +            supervisorData.setCurrAssignment(convertNewAssignment);
    +            // remove any downloaded code that's no longer assigned or 
active
    +            // important that this happens after setting the local 
assignment so that
    +            // synchronize-supervisor doesn't try to launch workers for 
which the
    +            // resources don't exist
    +            if (Utils.isOnWindows()) {
    +                shutdownDisallowedWorkers();
    +            }
    +            for (String stormId : allDownloadedTopologyIds) {
    +                if (!stormcodeMap.containsKey(stormId)) {
    +                    LOG.info("Removing code for storm id {}.", stormId);
    +                    rmTopoFiles(conf, stormId, 
supervisorData.getLocalizer(), true);
    +                }
    +            }
    +            syncProcessManager.add(syncProcesses);
    +        } catch (Exception e) {
    +            LOG.error("Failed to Sync Supervisor", e);
    +            throw new RuntimeException(e);
    +        }
    +
    +    }
    +
    +    private void killExistingWorkersWithChangeInComponents(SupervisorData 
supervisorData, Map<Integer, LocalAssignment> existingAssignment,
    +            Map<Integer, LocalAssignment> newAssignment) throws Exception {
    +        LocalState localState = supervisorData.getLocalState();
    +        Map<Integer, LocalAssignment> assignedExecutors = 
localState.getLocalAssignmentsMap();
    +        if (assignedExecutors == null) {
    +            assignedExecutors = new HashMap<>();
    +        }
    +        int now = Time.currentTimeSecs();
    +        Map<String, StateHeartbeat> workerIdHbstate = 
syncProcesses.getLocalWorkerStats(supervisorData, assignedExecutors, now);
    +        Map<Integer, String> vaildPortToWorkerIds = new HashMap<>();
    +        for (Map.Entry<String, StateHeartbeat> entry : 
workerIdHbstate.entrySet()) {
    +            String workerId = entry.getKey();
    +            StateHeartbeat stateHeartbeat = entry.getValue();
    +            if (stateHeartbeat != null && stateHeartbeat.getState() == 
State.VALID) {
    +                
vaildPortToWorkerIds.put(stateHeartbeat.getHeartbeat().get_port(), workerId);
    +            }
    +        }
    +
    +        Map<Integer, LocalAssignment> intersectAssignment = new 
HashMap<>();
    +        for (Map.Entry<Integer, LocalAssignment> entry : 
newAssignment.entrySet()) {
    +            Integer port = entry.getKey();
    +            if (existingAssignment.containsKey(port)) {
    +                intersectAssignment.put(port, entry.getValue());
    +            }
    +        }
    +
    +        for (Integer port : intersectAssignment.keySet()) {
    +            List<ExecutorInfo> existExecutors = 
existingAssignment.get(port).get_executors();
    +            List<ExecutorInfo> newExecutors = 
newAssignment.get(port).get_executors();
    +            if (newExecutors.size() != existExecutors.size()) {
    --- End diff --
    
    I think this logic will work just wasn't clear to me


> 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