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

    https://github.com/apache/storm/pull/1081#discussion_r53827548
  
    --- Diff: 
storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java ---
    @@ -0,0 +1,687 @@
    +/**
    + * 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.cluster;
    +
    +import clojure.lang.*;
    +import org.apache.commons.lang.StringUtils;
    +import org.apache.curator.framework.CuratorFramework;
    +import org.apache.curator.framework.state.*;
    +import org.apache.curator.framework.state.ConnectionState;
    +import org.apache.storm.callback.ZKStateChangedCallback;
    +import org.apache.storm.generated.*;
    +import org.apache.storm.nimbus.NimbusInfo;
    +import org.apache.storm.utils.Time;
    +import org.apache.storm.utils.Utils;
    +import org.apache.storm.zookeeper.Zookeeper;
    +import org.apache.zookeeper.KeeperException;
    +import org.apache.zookeeper.Watcher;
    +import org.apache.zookeeper.data.ACL;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.security.NoSuchAlgorithmException;
    +import java.util.*;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +public class StormClusterStateImpl implements IStormClusterState {
    +
    +    private static Logger LOG = 
LoggerFactory.getLogger(StormClusterStateImpl.class);
    +
    +    private IStateStorage stateStorage;
    +
    +    private ConcurrentHashMap<String, Runnable> assignmentInfoCallback;
    +    private ConcurrentHashMap<String, Runnable> 
assignmentInfoWithVersionCallback;
    +    private ConcurrentHashMap<String, Runnable> assignmentVersionCallback;
    +    private AtomicReference<Runnable> supervisorsCallback;
    +    // we want to reigister a topo directory getChildren callback for all 
workers of this dir
    +    private ConcurrentHashMap<String, Runnable> backPressureCallback;
    +    private AtomicReference<Runnable> assignmentsCallback;
    +    private ConcurrentHashMap<String, Runnable> stormBaseCallback;
    +    private AtomicReference<Runnable> blobstoreCallback;
    +    private ConcurrentHashMap<String, Runnable> credentialsCallback;
    +    private ConcurrentHashMap<String, Runnable> logConfigCallback;
    +
    +    private List<ACL> acls;
    +    private String stateId;
    +    private boolean solo;
    +
    +    public StormClusterStateImpl(IStateStorage StateStorage, List<ACL> 
acls, ClusterStateContext context, boolean solo) throws Exception {
    +
    +        this.stateStorage = StateStorage;
    +        this.solo = solo;
    +        this.acls = acls;
    +
    +        assignmentInfoCallback = new ConcurrentHashMap<>();
    +        assignmentInfoWithVersionCallback = new ConcurrentHashMap<>();
    +        assignmentVersionCallback = new ConcurrentHashMap<>();
    +        supervisorsCallback = new AtomicReference<>();
    +        backPressureCallback = new ConcurrentHashMap<>();
    +        assignmentsCallback = new AtomicReference<>();
    +        stormBaseCallback = new ConcurrentHashMap<>();
    +        credentialsCallback = new ConcurrentHashMap<>();
    +        logConfigCallback = new ConcurrentHashMap<>();
    +        blobstoreCallback = new AtomicReference<>();
    +
    +        stateId = this.stateStorage.register(new ZKStateChangedCallback() {
    +
    +            public void changed(Watcher.Event.EventType type, String path) 
{
    +                List<String> toks = Zookeeper.tokenizePath(path);
    +                int size = toks.size();
    +                if (size >= 1) {
    +                    String root = toks.get(0);
    +                    if (root.equals(ClusterUtils.ASSIGNMENTS_ROOT)) {
    +                        if (size == 1) {
    +                            // set null and get the old value
    +                            issueCallback(assignmentsCallback);
    +                        } else {
    +                            issueMapCallback(assignmentInfoCallback, 
toks.get(1));
    +                            issueMapCallback(assignmentVersionCallback, 
toks.get(1));
    +                            
issueMapCallback(assignmentInfoWithVersionCallback, toks.get(1));
    +                        }
    +
    +                    } else if (root.equals(ClusterUtils.SUPERVISORS_ROOT)) 
{
    +                        issueCallback(supervisorsCallback);
    +                    } else if (root.equals(ClusterUtils.BLOBSTORE_ROOT)) {
    +                        issueCallback(blobstoreCallback);
    +                    } else if (root.equals(ClusterUtils.STORMS_ROOT) && 
size > 1) {
    +                        issueMapCallback(stormBaseCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.CREDENTIALS_ROOT) 
&& size > 1) {
    +                        issueMapCallback(credentialsCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.LOGCONFIG_ROOT) && 
size > 1) {
    +                        issueMapCallback(logConfigCallback, toks.get(1));
    +                    } else if (root.equals(ClusterUtils.BACKPRESSURE_ROOT) 
&& size > 1) {
    +                        issueMapCallback(logConfigCallback, toks.get(1));
    +                    } else {
    +                        LOG.error("{} Unknown callback for subtree {}", 
new RuntimeException("Unknown callback for this path"), path);
    +                        Runtime.getRuntime().exit(30);
    +                    }
    +
    +                }
    +
    +                return;
    +            }
    +
    +        });
    +
    +        String[] pathlist = { ClusterUtils.ASSIGNMENTS_SUBTREE, 
ClusterUtils.STORMS_SUBTREE, ClusterUtils.SUPERVISORS_SUBTREE, 
ClusterUtils.WORKERBEATS_SUBTREE,
    +                ClusterUtils.ERRORS_SUBTREE, 
ClusterUtils.BLOBSTORE_SUBTREE, ClusterUtils.NIMBUSES_SUBTREE, 
ClusterUtils.LOGCONFIG_SUBTREE };
    +        for (String path : pathlist) {
    +            this.stateStorage.mkdirs(path, acls);
    +        }
    +
    +    }
    +
    +    protected void issueCallback(AtomicReference<Runnable> cb) {
    +        Runnable callback = cb.getAndSet(null);
    +        if (callback != null)
    +            callback.run();
    +    }
    +
    +    protected void issueMapCallback(ConcurrentHashMap<String, Runnable> 
callbackConcurrentHashMap, String key) {
    +        Runnable callback = callbackConcurrentHashMap.remove(key);
    +        if (callback != null)
    +            callback.run();
    +    }
    +
    +    @Override
    +    public List<String> assignments(Runnable callback) {
    +        if (callback != null) {
    +            assignmentsCallback.set(callback);
    +        }
    +        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, 
callback != null);
    +    }
    +
    +    @Override
    +    public Assignment assignmentInfo(String stormId, Runnable callback) {
    +        if (callback != null) {
    +            assignmentInfoCallback.put(stormId, callback);
    +        }
    +        byte[] serialized = 
stateStorage.get_data(ClusterUtils.assignmentPath(stormId), callback != null);
    +        return ClusterUtils.maybeDeserialize(serialized, Assignment.class);
    +    }
    +
    +    @Override
    +    public Map assignmentInfoWithVersion(String stormId, Runnable 
callback) {
    +        Map map = new HashMap();
    +        if (callback != null) {
    +            assignmentInfoWithVersionCallback.put(stormId, callback);
    +        }
    +        Assignment assignment = null;
    +        Integer version = 0;
    +        Map dataWithVersionMap = 
stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), 
callback != null);
    +        if (dataWithVersionMap != null) {
    +            assignment = ClusterUtils.maybeDeserialize((byte[]) 
dataWithVersionMap.get(IStateStorage.DATA), Assignment.class);
    +            version = (Integer) 
dataWithVersionMap.get(IStateStorage.VERSION);
    +        }
    +        map.put(IStateStorage.DATA, assignment);
    +        map.put(IStateStorage.VERSION, version);
    +        return map;
    +    }
    +
    +    @Override
    +    public Integer assignmentVersion(String stormId, Runnable callback) 
throws Exception {
    +        if (callback != null) {
    +            assignmentVersionCallback.put(stormId, callback);
    +        }
    +        return 
stateStorage.get_version(ClusterUtils.assignmentPath(stormId), callback != 
null);
    +    }
    +
    +    // blobstore state
    +    @Override
    +    public List<String> blobstoreInfo(String blobKey) {
    +        String path = ClusterUtils.blobstorePath(blobKey);
    +        stateStorage.sync_path(path);
    +        return stateStorage.get_children(path, false);
    +    }
    +
    +    @Override
    +    public List nimbuses() {
    +        List<NimbusSummary> nimbusSummaries = new ArrayList<>();
    +        List<String> nimbusIds = 
stateStorage.get_children(ClusterUtils.NIMBUSES_SUBTREE, false);
    +        for (String nimbusId : nimbusIds) {
    +            byte[] serialized = 
stateStorage.get_data(ClusterUtils.nimbusPath(nimbusId), false);
    +            NimbusSummary nimbusSummary = 
ClusterUtils.maybeDeserialize(serialized, NimbusSummary.class);
    +            nimbusSummaries.add(nimbusSummary);
    +        }
    +        return nimbusSummaries;
    +    }
    +
    +    @Override
    +    public void addNimbusHost(final String nimbusId, final NimbusSummary 
nimbusSummary) {
    +        // explicit delete for ephmeral node to ensure this session 
creates the entry.
    +        stateStorage.delete_node(ClusterUtils.nimbusPath(nimbusId));
    +        stateStorage.add_listener(new ConnectionStateListener() {
    +            @Override
    +            public void stateChanged(CuratorFramework curatorFramework, 
ConnectionState connectionState) {
    +                LOG.info("Connection state listener invoked, zookeeper 
connection state has changed to {}", connectionState);
    +                if (connectionState.equals(ConnectionState.RECONNECTED)) {
    +                    LOG.info("Connection state has changed to reconnected 
so setting nimbuses entry one more time");
    +                    
stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), 
Utils.serialize(nimbusSummary), acls);
    +                }
    +
    +            }
    +        });
    +
    +        stateStorage.set_ephemeral_node(ClusterUtils.nimbusPath(nimbusId), 
Utils.serialize(nimbusSummary), acls);
    +    }
    +
    +    @Override
    +    public List<String> activeStorms() {
    +        return stateStorage.get_children(ClusterUtils.STORMS_SUBTREE, 
false);
    +    }
    +
    +    @Override
    +    public StormBase stormBase(String stormId, Runnable callback) {
    +        if (callback != null) {
    +            stormBaseCallback.put(stormId, callback);
    +        }
    +        return 
ClusterUtils.maybeDeserialize(stateStorage.get_data(ClusterUtils.stormPath(stormId),
 callback != null), StormBase.class);
    +    }
    +
    +    @Override
    +    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, 
String node, Long port) {
    +        byte[] bytes = 
stateStorage.get_worker_hb(ClusterUtils.workerbeatPath(stormId, node, port), 
false);
    +        return ClusterUtils.maybeDeserialize(bytes, 
ClusterWorkerHeartbeat.class);
    +
    +    }
    +
    +    @Override
    +    public List<ProfileRequest> getWorkerProfileRequests(String stormId, 
NodeInfo nodeInfo, boolean isThrift) {
    +        List<ProfileRequest> requests = new ArrayList<>();
    +        List<ProfileRequest> profileRequests = 
getTopologyProfileRequests(stormId, isThrift);
    +        for (ProfileRequest profileRequest : profileRequests) {
    +            NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
    +            if (nodeInfo1.equals(nodeInfo))
    +                requests.add(profileRequest);
    +        }
    +        return requests;
    +    }
    +
    +    @Override
    +    public List<ProfileRequest> getTopologyProfileRequests(String stormId, 
boolean isThrift) {
    +        List<ProfileRequest> profileRequests = new ArrayList<>();
    +        String path = ClusterUtils.profilerConfigPath(stormId);
    +        if (stateStorage.node_exists(path, false)) {
    +            List<String> strs = stateStorage.get_children(path, false);
    +            for (String str : strs) {
    +                String childPath = path + ClusterUtils.ZK_SEPERATOR + str;
    +                byte[] raw = stateStorage.get_data(childPath, false);
    +                ProfileRequest request = 
ClusterUtils.maybeDeserialize(raw, ProfileRequest.class);
    +                if (request != null)
    +                    profileRequests.add(request);
    +            }
    +        }
    +        return profileRequests;
    +    }
    +
    +    @Override
    +    public void setWorkerProfileRequest(String stormId, ProfileRequest 
profileRequest) {
    +        ProfileAction profileAction = profileRequest.get_action();
    +        String host = profileRequest.get_nodeInfo().get_node();
    +        Long port = 
profileRequest.get_nodeInfo().get_port_iterator().next();
    +        String path = ClusterUtils.profilerConfigPath(stormId, host, port, 
profileAction);
    +        stateStorage.set_data(path, Utils.serialize(profileRequest), acls);
    +    }
    +
    +    @Override
    +    public void deleteTopologyProfileRequests(String stormId, 
ProfileRequest profileRequest) {
    +        ProfileAction profileAction = profileRequest.get_action();
    +        String host = profileRequest.get_nodeInfo().get_node();
    +        Long port = 
profileRequest.get_nodeInfo().get_port_iterator().next();
    +        String path = ClusterUtils.profilerConfigPath(stormId, host, port, 
profileAction);
    +        stateStorage.delete_node(path);
    +    }
    +
    +    /**
    +     * need to take executor->node+port in explicitly so that we don't run 
into a situation where a long dead worker with a skewed clock overrides all the
    +     * timestamps. By only checking heartbeats with an assigned node+port, 
and only reading executors from that heartbeat that are actually assigned, we 
avoid
    +     * situations like that
    +     * 
    +     * @param stormId
    +     * @param executorNodePort
    +     * @return
    +     */
    +    @Override
    +    public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, 
Map<List<Long>, NodeInfo> executorNodePort) {
    +        Map<ExecutorInfo, ExecutorBeat> executorWhbs = new HashMap<>();
    +
    +        Map<NodeInfo, List<List<Long>>> nodePortExecutors = 
Utils.reverseMap(executorNodePort);
    +
    +        for (Map.Entry<NodeInfo, List<List<Long>>> entry : 
nodePortExecutors.entrySet()) {
    +
    +            String node = entry.getKey().get_node();
    +            Long port = entry.getKey().get_port_iterator().next();
    +            ClusterWorkerHeartbeat whb = getWorkerHeartbeat(stormId, node, 
port);
    +            List<ExecutorInfo> executorInfoList = new ArrayList<>();
    +            for (List<Long> list : entry.getValue()) {
    +                executorInfoList.add(new 
ExecutorInfo(list.get(0).intValue(), list.get(list.size() - 1).intValue()));
    +            }
    +            if (whb != null)
    +                
executorWhbs.putAll(ClusterUtils.convertExecutorBeats(executorInfoList, whb));
    +        }
    +        return executorWhbs;
    +    }
    +
    +    @Override
    +    public List<String> supervisors(Runnable callback) {
    +        if (callback != null) {
    +            supervisorsCallback.set(callback);
    +        }
    +        return stateStorage.get_children(ClusterUtils.SUPERVISORS_SUBTREE, 
callback != null);
    +    }
    +
    +    @Override
    +    public SupervisorInfo supervisorInfo(String supervisorId) {
    +        String path = ClusterUtils.supervisorPath(supervisorId);
    +        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, 
false), SupervisorInfo.class);
    +    }
    +
    +    @Override
    +    public void setupHeatbeats(String stormId) {
    +        stateStorage.mkdirs(ClusterUtils.workerbeatStormRoot(stormId), 
acls);
    +    }
    +
    +    @Override
    +    public void teardownHeartbeats(String stormId) {
    +        try {
    +            
stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
    +        } catch (Exception e) {
    +            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, 
e)) {
    +                // do nothing
    +                LOG.warn("Could not teardown heartbeats for {}.", stormId);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void teardownTopologyErrors(String stormId) {
    +        try {
    +            stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
    +        } catch (Exception e) {
    +            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, 
e)) {
    +                // do nothing
    +                LOG.warn("Could not teardown errors for {}.", stormId);
    +            } else {
    +                throw e;
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public List<String> heartbeatStorms() {
    +        return 
stateStorage.get_worker_hb_children(ClusterUtils.WORKERBEATS_SUBTREE, false);
    +    }
    +
    +    @Override
    +    public List<String> errorTopologies() {
    +        return stateStorage.get_children(ClusterUtils.ERRORS_SUBTREE, 
false);
    +    }
    +
    +    @Override
    +    public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
    +        stateStorage.set_data(ClusterUtils.logConfigPath(stormId), 
Utils.serialize(logConfig), acls);
    +    }
    +
    +    @Override
    +    public LogConfig topologyLogConfig(String stormId, Runnable cb) {
    +        String path = ClusterUtils.logConfigPath(stormId);
    --- End diff --
    
    cb is never stored to be called back.


---
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.
---

Reply via email to