Github user longdafeng commented on a diff in the pull request:
https://github.com/apache/storm/pull/1257#discussion_r57900100
--- Diff:
storm-core/src/jvm/org/apache/storm/daemon/supervisor/SyncSupervisorEvent.java
---
@@ -0,0 +1,626 @@
+/**
+ * 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);
--- End diff --
This place just info ISupervisor killedWorker(port);
Don't do supervisorData.getWorkerManager.shutdown() worker?
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---