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

    https://github.com/apache/storm/pull/1081#discussion_r53811031
  
    --- 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));
    --- End diff --
    
    This should be backpressureCallback.


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