You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@storm.apache.org by bo...@apache.org on 2016/02/24 17:17:11 UTC

[01/27] storm git commit: port org.apache.storm.cluster.cluster.clj

Repository: storm
Updated Branches:
  refs/heads/master 11232b539 -> 71d615b7c


port org.apache.storm.cluster.cluster.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/682d31c8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/682d31c8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/682d31c8

Branch: refs/heads/master
Commit: 682d31c8a5f53a59f47b4df4bd35ff828e7e0aa5
Parents: 66d7a39
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 1 15:17:49 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 1 15:17:49 2016 +0800

----------------------------------------------------------------------
 .../jvm/org/apache/storm/cluster/Cluster.java   | 204 ++++++
 .../org/apache/storm/cluster/ClusterState.java  |  11 +-
 .../storm/cluster/DistributedClusterState.java  | 269 ++++++++
 .../apache/storm/cluster/StormClusterState.java | 129 ++++
 .../storm/cluster/StormZkClusterState.java      | 662 +++++++++++++++++++
 5 files changed, 1271 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
new file mode 100644
index 0000000..2d6f306
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
@@ -0,0 +1,204 @@
+/**
+ * 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 org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class Cluster {
+
+    public static final String ZK_SEPERATOR = "/";
+
+    public static final String ASSIGNMENTS_ROOT = "assignments";
+    public static final String CODE_ROOT = "code";
+    public static final String STORMS_ROOT = "storms";
+    public static final String SUPERVISORS_ROOT = "supervisors";
+    public static final String WORKERBEATS_ROOT = "workerbeats";
+    public static final String BACKPRESSURE_ROOT = "backpressure";
+    public static final String ERRORS_ROOT = "errors";
+    public static final String BLOBSTORE_ROOT = "blobstore";
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+    public static final String NIMBUSES_ROOT = "nimbuses";
+    public static final String CREDENTIALS_ROOT = "credentials";
+    public static final String LOGCONFIG_ROOT = "logconfigs";
+    public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+    public static final String ASSIGNMENTS_SUBTREE;
+    public static final String STORMS_SUBTREE;
+    public static final String SUPERVISORS_SUBTREE;
+    public static final String WORKERBEATS_SUBTREE;
+    public static final String BACKPRESSURE_SUBTREE;
+    public static final String ERRORS_SUBTREE;
+    public static final String BLOBSTORE_SUBTREE;
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE;
+    public static final String NIMBUSES_SUBTREE;
+    public static final String CREDENTIALS_SUBTREE;
+    public static final String LOGCONFIG_SUBTREE;
+    public static final String PROFILERCONFIG_SUBTREE;
+
+    static {
+        ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+        STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+        SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+        WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+        BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+        ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+        BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+        BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+        NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+        CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+        LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+        PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+    }
+
+    public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = new ArrayList<>();
+        String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
+        if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
+            ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+            aclList.add(acl1);
+            ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+            aclList.add(acl2);
+        }
+        return aclList;
+    }
+
+    public static String supervisorPath(String id) {
+        return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String assignmentPath(String id) {
+        return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String blobstorePath(String key) {
+        return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String blobstoreMaxKeySequenceNumberPath(String key) {
+        return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String nimbusPath(String id) {
+        return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String stormPath(String id) {
+        return STORMS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String workerbeatStormRoot(String stormId) {
+        return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String workerbeatPath(String stormId, String node, Long port) {
+        return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String backpressureStormRoot(String stormId) {
+        return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String backpressurePath(String stormId, String node, Long port) {
+        return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String errorStormRoot(String stormId) {
+        return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String errorPath(String stormId, String componentId) throws UnsupportedEncodingException {
+        return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+    }
+
+    public static String lastErrorPath(String stormId, String componentId) throws UnsupportedEncodingException {
+        return errorPath(stormId, componentId) + "-last-error";
+    }
+
+    public static String credentialsPath(String stormId) {
+        return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String logConfigPath(String stormId) {
+        return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId) {
+        return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+        return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+    }
+
+    public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz){
+        if (serialized != null){
+            return Utils.deserialize(serialized, clazz);
+        }
+        return null;
+    }
+
+    //Ensures that we only return heartbeats for executors assigned to this worker
+    public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat){
+        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
+        Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+        for (ExecutorInfo executor : executors){
+            if(executorStatsMap.containsKey(executor)){
+                executorWhb.put(executor, workerHeartbeat);
+            }
+        }
+        return executorWhb;
+    }
+    
+    // TO be remove
+    public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
+        HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
+        if (map == null) {
+            return rtn;
+        }
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+            K key = entry.getKey();
+            V val = entry.getValue();
+            List<K> list = rtn.get(val);
+            if (list == null) {
+                list = new ArrayList<K>();
+                rtn.put(entry.getValue(), list);
+            }
+            list.add(key);
+        }
+        return rtn;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
index fdac92c..51e42ff 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
@@ -20,6 +20,9 @@ package org.apache.storm.cluster;
 import clojure.lang.APersistentMap;
 import clojure.lang.IFn;
 import java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.Callback;
 import org.apache.zookeeper.data.ACL;
 
 /**
@@ -47,7 +50,7 @@ public interface ClusterState {
      * @return is an id that can be passed to unregister(...) to unregister the
      * callback.
      */
-    String register(IFn callback);
+    String register(Callback callback);
 
     /**
      * Unregisters a callback function that was registered with register(...).
@@ -73,7 +76,7 @@ public interface ClusterState {
      * @param acls The acls to apply to the path. May be null.
      * @return path
      */
-    String mkdirs(String path, List<ACL> acls);
+    void mkdirs(String path, List<ACL> acls);
 
     /**
      * Deletes the node at a given path, and any child nodes that may exist.
@@ -99,7 +102,7 @@ public interface ClusterState {
      * register method. Very useful for catching updates to nodes.
      * @return The integer version of this node.
      */
-    Integer get_version(String path, boolean watch);
+    Integer get_version(String path, boolean watch) throws Exception;
 
     /**
      * Check if a node exists and optionally set a watch on the path.
@@ -197,7 +200,7 @@ public interface ClusterState {
      * @param listener A ClusterStateListener to handle changing cluster state
      * events.
      */
-    void add_listener(ClusterStateListener listener);
+    void add_listener(final ConnectionStateListener listener);
 
     /**
      * Force consistency on a path. Any writes committed on the path before

http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
new file mode 100644
index 0000000..3e0beb1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
@@ -0,0 +1,269 @@
+/**
+ * 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.APersistentMap;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.Config;
+import org.apache.storm.callback.Callback;
+import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.Zookeeper;
+import org.apache.zookeeper.CreateMode;
+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.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class DistributedClusterState implements ClusterState {
+
+    private static Logger LOG = LoggerFactory.getLogger(DistributedClusterState.class);
+
+    private ConcurrentHashMap<String, Callback> callbacks = new ConcurrentHashMap<String, Callback>();
+    private CuratorFramework zkWriter;
+    private CuratorFramework zkReader;
+    private AtomicBoolean active;
+
+    private boolean isNimbus;
+    private Map authConf;
+    private Map<Object, Object> conf;
+
+    public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        this.conf = conf;
+        this.authConf = authConf;
+        if (context.getDaemonType().equals(DaemonType.NIMBUS))
+            this.isNimbus = true;
+
+        // just mkdir STORM_ZOOKEEPER_ROOT dir
+        CuratorFramework zkTemp = mkZk();
+        String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
+        Zookeeper.mkdirs(zkTemp, rootPath, acls);
+        zkTemp.close();
+
+        active = new AtomicBoolean(true);
+        zkWriter = mkZk(new WatcherCallBack() {
+            @Override
+            public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+                if (active.get()) {
+                    if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                        LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                    } else {
+                        LOG.info("Received event {} : {} : {}", state, type, path);
+                    }
+
+                    if (!type.equals(Watcher.Event.EventType.None)) {
+                        for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
+                            Callback fn = e.getValue();
+                            fn.execute(type, path);
+                        }
+                    }
+                }
+            }
+        });
+        if (isNimbus) {
+            zkReader = mkZk(new WatcherCallBack() {
+                @Override
+                public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+                    if (active.get()) {
+                        if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                            LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                        } else {
+                            LOG.info("Received event {} : {} : {}", state, type, path);
+                        }
+
+                        if (!type.equals(Watcher.Event.EventType.None)) {
+                            for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
+                                Callback fn = e.getValue();
+                                fn.execute(type, path);
+                            }
+                        }
+                    }
+                }
+            });
+        } else {
+            zkReader = zkWriter;
+        }
+
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk() throws IOException {
+        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", authConf);
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
+        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
+                String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
+    }
+
+    @Override
+    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+
+    }
+
+    @Override
+    public String register(Callback callback) {
+        String id = UUID.randomUUID().toString();
+        this.callbacks.put(id, callback);
+        return id;
+    }
+
+    @Override
+    public void unregister(String id) {
+        this.callbacks.remove(id);
+    }
+
+    @Override
+    public String create_sequential(String path, byte[] data, List<ACL> acls) {
+        return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
+    }
+
+    @Override
+    public void mkdirs(String path, List<ACL> acls) {
+        Zookeeper.mkdirs(zkWriter, path, acls);
+    }
+
+    @Override
+    public void delete_node(String path) {
+        Zookeeper.deleteNode(zkWriter, path);
+    }
+
+    @Override
+    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+        Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+        if (Zookeeper.exists(zkWriter, path, false)) {
+            try {
+                Zookeeper.setData(zkWriter, path, data);
+            } catch (RuntimeException e) {
+                if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
+                    Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+                } else {
+                    throw e;
+                }
+            }
+
+        } else {
+            Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+        }
+    }
+
+    @Override
+    public Integer get_version(String path, boolean watch) throws Exception {
+        Integer ret = Zookeeper.getVersion(zkReader, path, watch);
+        return ret;
+    }
+
+    @Override
+    public boolean node_exists(String path, boolean watch) {
+        return Zookeeper.existsNode(zkWriter, path, watch);
+    }
+
+    @Override
+    public List<String> get_children(String path, boolean watch) {
+        return Zookeeper.getChildren(zkReader, path, watch);
+    }
+
+    @Override
+    public void close() {
+        this.active.set(false);
+        zkWriter.close();
+        if (isNimbus) {
+            zkReader.close();
+        }
+    }
+
+    @Override
+    public void set_data(String path, byte[] data, List<ACL> acls) {
+        if (Zookeeper.exists(zkWriter, path, false)) {
+            Zookeeper.setData(zkWriter, path, data);
+        } else {
+            Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+            Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
+        }
+    }
+
+    @Override
+    public byte[] get_data(String path, boolean watch) {
+        byte[] ret = null;
+
+        ret = Zookeeper.getData(zkReader, path, watch);
+
+        return ret;
+    }
+
+    @Override
+    public APersistentMap get_data_with_version(String path, boolean watch) {
+        return Zookeeper.getDataWithVersion(zkReader, path, watch);
+    }
+
+    @Override
+    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+        set_data(path, data, acls);
+    }
+
+    @Override
+    public byte[] get_worker_hb(String path, boolean watch) {
+        return Zookeeper.getData(zkReader, path, watch);
+    }
+
+    @Override
+    public List<String> get_worker_hb_children(String path, boolean watch) {
+        return get_children(path, watch);
+    }
+
+    @Override
+    public void delete_worker_hb(String path) {
+        delete_node(path);
+    }
+
+    @Override
+    public void add_listener(final ConnectionStateListener listener) {
+        Zookeeper.addListener(zkReader, new ConnectionStateListener() {
+            @Override
+            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+                listener.stateChanged(curatorFramework, connectionState);
+            }
+        });
+    }
+
+    @Override
+    public void sync_path(String path) {
+        Zookeeper.syncPath(zkWriter, path);
+    }
+
+    // To be remove when finished port Util.clj
+    public static String parentPath(String path) {
+        List<String> toks = Zookeeper.tokenizePath(path);
+        int size = toks.size();
+        if (size > 0) {
+            toks.remove(size - 1);
+        }
+        return Zookeeper.toksToPath(toks);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
new file mode 100644
index 0000000..b3c0f90
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -0,0 +1,129 @@
+/**
+ * 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.APersistentMap;
+import org.apache.storm.callback.Callback;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+public interface StormClusterState {
+    public List<String> assignments(Callback callback);
+
+    public Assignment assignmentInfo(String stormId, Callback callback);
+
+    public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback);
+
+    public Integer assignmentVersion(String stormId, Callback callback) throws Exception;
+
+    // returns key information under /storm/blobstore/key
+    public List<String> blobstoreInfo(String blobKey);
+
+    // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+    public List nimbuses();
+
+    // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+    public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+    public List<String> activeStorms();
+
+    public StormBase stormBase(String stormId, Callback callback);
+
+    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+    public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift);
+
+    public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift);
+
+    public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest);
+
+    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
+    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+    public List<String> supervisors(Callback callback);
+
+    public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+    public void setupHeatbeats(String stormId);
+
+    public void teardownHeatbeats(String stormId);
+
+    public void teardownTopologyErrors(String stormId);
+
+    public List<String> heartbeatStorms();
+
+    public List<String> errorTopologies();
+
+    public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+    public LogConfig topologyLogConfig(String stormId, Callback cb);
+
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+    public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+    public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+    public boolean topologyBackpressure(String stormId, Callback callback);
+
+    public void setupBackpressure(String stormId);
+
+    public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+    public void activateStorm(String stormId, StormBase stormBase);
+
+    public void updateStorm(String stormId, StormBase newElems);
+
+    public void removeStormBase(String stormId);
+
+    public void setAssignment(String stormId, Assignment info);
+
+    // sets up information related to key consisting of nimbus
+    // host:port and version info of the blob
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo);
+
+    public List<String> activeKeys();
+
+    public List<String> blobstore(Callback callback);
+
+    public void removeStorm(String stormId);
+
+    public void removeBlobstoreKey(String blobKey);
+
+    public void removeKeyVersion(String blobKey);
+
+    public void reportError(String stormId, String componentId, String node, Long port, String error);
+
+    public List<ErrorInfo> errors(String stormId, String componentId);
+
+    public ErrorInfo lastError(String stormId, String componentId);
+
+    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
+
+    public Credentials credentials(String stormId, Callback callback);
+
+    public void disconnect();
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/682d31c8/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
new file mode 100644
index 0000000..93d29b2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -0,0 +1,662 @@
+/**
+ * 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.APersistentMap;
+import clojure.lang.PersistentArrayMap;
+import clojure.lang.RT;
+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.Callback;
+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.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.UnsupportedEncodingException;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
+
+public class StormZkClusterState implements StormClusterState {
+
+    private static Logger LOG = LoggerFactory.getLogger(StormZkClusterState.class);
+
+    private ClusterState clusterState;
+
+    private ConcurrentHashMap<String, Callback> assignmentInfoCallback;
+    private ConcurrentHashMap<String, Callback> assignmentInfoWithVersionCallback;
+    private ConcurrentHashMap<String, Callback> assignmentVersionCallback;
+    private AtomicReference<Callback> supervisorsCallback;
+    // we want to reigister a topo directory getChildren callback for all workers of this dir
+    private ConcurrentHashMap<String, Callback> backPressureCallback;
+    private AtomicReference<Callback> assignmentsCallback;
+    private ConcurrentHashMap<String, Callback> stormBaseCallback;
+    private AtomicReference<Callback> blobstoreCallback;
+    private ConcurrentHashMap<String, Callback> credentialsCallback;
+    private ConcurrentHashMap<String, Callback> logConfigCallback;
+
+    private List<ACL> acls;
+    private String stateId;
+    private boolean solo;
+
+    public StormZkClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception {
+
+        if (clusterState instanceof ClusterState) {
+            solo = false;
+            this.clusterState = (ClusterState) clusterState;
+        } else {
+
+            solo = true;
+            this.clusterState = new DistributedClusterState((Map) clusterState, (Map) clusterState, acls, context);
+        }
+
+        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.clusterState.register(new Callback() {
+
+            public <T> Object execute(T... args) {
+                if (args == null) {
+                    LOG.warn("Input args is null");
+                    return null;
+                } else if (args.length < 2) {
+                    LOG.warn("Input args is invalid, args length:" + args.length);
+                    return null;
+                }
+                String path = (String) args[1];
+
+                List<String> toks = Zookeeper.tokenizePath(path);
+                int size = toks.size();
+                if (size >= 1) {
+                    String params = null;
+                    String root = toks.get(0);
+                    Callback fn = null;
+                    if (root.equals(Cluster.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(Cluster.SUPERVISORS_ROOT)) {
+                        issueCallback(supervisorsCallback);
+                    } else if (root.equals(Cluster.BLOBSTORE_ROOT)) {
+                        issueCallback(blobstoreCallback);
+                    } else if (root.equals(Cluster.STORMS_ROOT) && size > 1) {
+                        issueMapCallback(stormBaseCallback, toks.get(1));
+                    } else if (root.equals(Cluster.CREDENTIALS_ROOT) && size > 1) {
+                        issueMapCallback(credentialsCallback, toks.get(1));
+                    } else if (root.equals(Cluster.LOGCONFIG_ROOT) && size > 1) {
+                        issueMapCallback(logConfigCallback, toks.get(1));
+                    } else if (root.equals(Cluster.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 null;
+            }
+
+        });
+
+        String[] pathlist = { Cluster.ASSIGNMENTS_SUBTREE, Cluster.STORMS_SUBTREE, Cluster.SUPERVISORS_SUBTREE, Cluster.WORKERBEATS_SUBTREE,
+                Cluster.ERRORS_SUBTREE, Cluster.BLOBSTORE_SUBTREE, Cluster.NIMBUSES_SUBTREE, Cluster.LOGCONFIG_SUBTREE };
+        for (String path : pathlist) {
+            this.clusterState.mkdirs(path, acls);
+        }
+
+    }
+
+    protected void issueCallback(AtomicReference<Callback> cb) {
+        Callback callback = cb.getAndSet(null);
+        callback.execute();
+    }
+
+    protected void issueMapCallback(ConcurrentHashMap<String, Callback> callbackConcurrentHashMap, String key) {
+        Callback callback = callbackConcurrentHashMap.remove(key);
+        callback.execute();
+    }
+
+    @Override
+    public List<String> assignments(Callback callback) {
+        if (callback != null) {
+            assignmentsCallback.set(callback);
+        }
+        return clusterState.get_children(Cluster.ASSIGNMENTS_SUBTREE, callback != null);
+    }
+
+    @Override
+    public Assignment assignmentInfo(String stormId, Callback callback) {
+        if (callback != null) {
+            assignmentInfoCallback.put(stormId, callback);
+        }
+        byte[] serialized = clusterState.get_data(Cluster.assignmentPath(stormId), callback != null);
+        return Cluster.maybeDeserialize(serialized, Assignment.class);
+    }
+
+    @Override
+    public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback) {
+        if (callback != null) {
+            assignmentInfoWithVersionCallback.put(stormId, callback);
+        }
+        APersistentMap aPersistentMap = clusterState.get_data_with_version(Cluster.assignmentPath(stormId), callback != null);
+        Assignment assignment = Cluster.maybeDeserialize((byte[]) aPersistentMap.get("data"), Assignment.class);
+        Integer version = (Integer) aPersistentMap.get("version");
+        APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
+        return map;
+    }
+
+    @Override
+    public Integer assignmentVersion(String stormId, Callback callback) throws Exception {
+        if (callback != null) {
+            assignmentVersionCallback.put(stormId, callback);
+        }
+        return clusterState.get_version(Cluster.assignmentPath(stormId), callback != null);
+    }
+
+    // blobstore state
+    @Override
+    public List<String> blobstoreInfo(String blobKey) {
+        String path = Cluster.blobstorePath(blobKey);
+        clusterState.sync_path(path);
+        return clusterState.get_children(path, false);
+    }
+
+    @Override
+    public List nimbuses() {
+        List<NimbusSummary> nimbusSummaries = new ArrayList<>();
+        List<String> nimbusIds = clusterState.get_children(Cluster.NIMBUSES_SUBTREE, false);
+        for (String nimbusId : nimbusIds) {
+            byte[] serialized = clusterState.get_data(Cluster.nimbusPath(nimbusId), false);
+            NimbusSummary nimbusSummary = Cluster.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.
+        clusterState.delete_node(Cluster.nimbusPath(nimbusId));
+        clusterState.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");
+                    clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+                }
+
+            }
+        });
+
+        clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
+    }
+
+    @Override
+    public List<String> activeStorms() {
+        return clusterState.get_children(Cluster.STORMS_SUBTREE, false);
+    }
+
+    @Override
+    public StormBase stormBase(String stormId, Callback callback) {
+        if (callback != null) {
+            stormBaseCallback.put(stormId, callback);
+        }
+        return Cluster.maybeDeserialize(clusterState.get_data(Cluster.stormPath(stormId), callback != null), StormBase.class);
+    }
+
+    @Override
+    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) {
+        byte[] bytes = clusterState.get_worker_hb(Cluster.workerbeatPath(stormId, node, port), false);
+        if (bytes != null) {
+            return Cluster.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
+        }
+        return null;
+    }
+
+    @Override
+    public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+        List<ProfileRequest> requests = new ArrayList<>();
+        List<ProfileRequest> profileRequests = getTopologyProfileRequets(stormId, isThrift);
+        for (ProfileRequest profileRequest : profileRequests) {
+            NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
+            if (nodeInfo1.equals(nodeInfo))
+                requests.add(profileRequest);
+        }
+        return requests;
+    }
+
+    @Override
+    public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift) {
+        List<ProfileRequest> profileRequests = new ArrayList<>();
+        String path = Cluster.profilerConfigPath(stormId);
+        if (clusterState.node_exists(path, false)) {
+            List<String> strs = clusterState.get_children(path, false);
+            for (String str : strs) {
+                String childPath = path + Cluster.ZK_SEPERATOR + str;
+                byte[] raw = clusterState.get_data(childPath, false);
+                ProfileRequest request = Cluster.maybeDeserialize(raw, ProfileRequest.class);
+                if (request != null)
+                    profileRequests.add(request);
+            }
+        }
+        return profileRequests;
+    }
+
+    @Override
+    public void setWorkerProfileRequests(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 = Cluster.profilerConfigPath(stormId, host, port, profileAction);
+        clusterState.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 = Cluster.profilerConfigPath(stormId, host, port, profileAction);
+        clusterState.delete_node(path);
+    }
+
+    @Override
+    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
+
+        Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.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()));
+            }
+            executorWhbs.putAll(Cluster.convertExecutorBeats(executorInfoList, whb));
+        }
+        return executorWhbs;
+    }
+
+    @Override
+    public List<String> supervisors(Callback callback) {
+        if (callback != null) {
+            supervisorsCallback.set(callback);
+        }
+        return clusterState.get_children(Cluster.SUPERVISORS_SUBTREE, callback != null);
+    }
+
+    @Override
+    public SupervisorInfo supervisorInfo(String supervisorId) {
+        String path = Cluster.supervisorPath(supervisorId);
+        return Cluster.maybeDeserialize(clusterState.get_data(path, false), SupervisorInfo.class);
+    }
+
+    @Override
+    public void setupHeatbeats(String stormId) {
+        clusterState.mkdirs(Cluster.workerbeatStormRoot(stormId), acls);
+    }
+
+    @Override
+    public void teardownHeatbeats(String stormId) {
+        try {
+            clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
+        } catch (Exception e) {
+            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+                // do nothing
+                LOG.warn("Could not teardown heartbeats for {}.", stormId);
+            } else {
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public void teardownTopologyErrors(String stormId) {
+        try {
+            clusterState.delete_node(Cluster.errorStormRoot(stormId));
+        } catch (Exception e) {
+            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+                // do nothing
+                LOG.warn("Could not teardown errors for {}.", stormId);
+            } else {
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public List<String> heartbeatStorms() {
+        return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false);
+    }
+
+    @Override
+    public List<String> errorTopologies() {
+        return clusterState.get_children(Cluster.ERRORS_SUBTREE, false);
+    }
+
+    @Override
+    public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
+        clusterState.set_data(Cluster.logConfigPath(stormId), Utils.serialize(logConfig), acls);
+    }
+
+    @Override
+    public LogConfig topologyLogConfig(String stormId, Callback cb) {
+        String path = Cluster.logConfigPath(stormId);
+        return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
+    }
+
+    @Override
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+        if (info != null) {
+            String path = Cluster.workerbeatPath(stormId, node, port);
+            clusterState.set_worker_hb(path, Utils.serialize(info), acls);
+        }
+    }
+
+    @Override
+    public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+        String path = Cluster.workerbeatPath(stormId, node, port);
+        clusterState.delete_worker_hb(path);
+    }
+
+    @Override
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+        String path = Cluster.supervisorPath(supervisorId);
+        clusterState.set_ephemeral_node(path, Utils.serialize(info), acls);
+    }
+
+    // if znode exists and to be not on?, delete; if exists and on?, do nothing;
+    // if not exists and to be on?, create; if not exists and not on?, do nothing;
+    @Override
+    public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+        String path = Cluster.backpressurePath(stormId, node, port);
+        boolean existed = clusterState.node_exists(path, false);
+        if (existed) {
+            if (on == false)
+                clusterState.delete_node(path);
+
+        } else {
+            if (on == true) {
+                clusterState.set_ephemeral_node(path, null, acls);
+            }
+        }
+    }
+
+    // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
+    @Override
+    public boolean topologyBackpressure(String stormId, Callback callback) {
+        if (callback != null) {
+            backPressureCallback.put(stormId, callback);
+        }
+        String path = Cluster.backpressureStormRoot(stormId);
+        List<String> childrens = clusterState.get_children(path, callback != null);
+        return childrens.size() > 0;
+
+    }
+
+    @Override
+    public void setupBackpressure(String stormId) {
+        clusterState.mkdirs(Cluster.backpressureStormRoot(stormId), acls);
+    }
+
+    @Override
+    public void removeWorkerBackpressure(String stormId, String node, Long port) {
+        clusterState.delete_node(Cluster.backpressurePath(stormId, node, port));
+    }
+
+    @Override
+    public void activateStorm(String stormId, StormBase stormBase) {
+        String path = Cluster.stormPath(stormId);
+        clusterState.set_data(path, Utils.serialize(stormBase), acls);
+    }
+
+    // maybe exit some questions for updateStorm
+    @Override
+    public void updateStorm(String stormId, StormBase newElems) {
+
+        StormBase stormBase = stormBase(stormId, null);
+        if (stormBase.get_component_executors() != null) {
+            Map<String, Integer> componentExecutors = newElems.get_component_executors();
+            if (componentExecutors == null) {
+                componentExecutors = new HashMap<>();
+            }
+            for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+                if (!componentExecutors.containsKey(entry.getKey())) {
+                    componentExecutors.put(entry.getKey(), entry.getValue());
+                }
+            }
+            if (componentExecutors.size() > 0)
+                newElems.set_component_executors(componentExecutors);
+        }
+
+        Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+        Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+        if (oldComponentDebug == null)
+            oldComponentDebug = new HashMap<>();
+        Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+        if (newComponentDebug == null)
+            newComponentDebug = new HashMap<>();
+        Set<String> debugOptionsKeys = oldComponentDebug.keySet();
+        debugOptionsKeys.addAll(newComponentDebug.keySet());
+        for (String key : debugOptionsKeys) {
+            boolean enable = false;
+            double samplingpct = 0;
+            if (oldComponentDebug.containsKey(key)) {
+                enable = oldComponentDebug.get(key).is_enable();
+                samplingpct = oldComponentDebug.get(key).get_samplingpct();
+            }
+            if (newComponentDebug.containsKey(key)) {
+                enable = newComponentDebug.get(key).is_enable();
+                samplingpct += newComponentDebug.get(key).get_samplingpct();
+            }
+            DebugOptions debugOptions = new DebugOptions();
+            debugOptions.set_enable(enable);
+            debugOptions.set_samplingpct(samplingpct);
+            ComponentDebug.put(key, debugOptions);
+        }
+        if (ComponentDebug.size() > 0) {
+            newElems.set_component_debug(ComponentDebug);
+        }
+        // only merge some parameters which are optional
+        if (newElems.get_launch_time_secs() == 0) {
+            newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+        }
+        if (StringUtils.isBlank(newElems.get_owner())) {
+            newElems.set_owner(stormBase.get_owner());
+        }
+        if (newElems.get_topology_action_options() == null) {
+            newElems.set_topology_action_options(stormBase.get_topology_action_options());
+        }
+        if (newElems.get_status() == null) {
+            newElems.set_status(stormBase.get_status());
+        }
+        clusterState.set_data(Cluster.stormPath(stormId), Utils.serialize(newElems), acls);
+    }
+
+    @Override
+    public void removeStormBase(String stormId) {
+        clusterState.delete_node(Cluster.stormPath(stormId));
+    }
+
+    @Override
+    public void setAssignment(String stormId, Assignment info) {
+        clusterState.set_data(Cluster.assignmentPath(stormId), Utils.serialize(info), acls);
+    }
+
+    @Override
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo) {
+        String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "_" + versionInfo;
+        LOG.info("set-path: {}", path);
+        clusterState.mkdirs(Cluster.blobstorePath(key), acls);
+        clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
+        clusterState.set_ephemeral_node(path, null, acls);
+    }
+
+    @Override
+    public List<String> activeKeys() {
+        return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false);
+    }
+
+    // blobstore state
+    @Override
+    public List<String> blobstore(Callback callback) {
+        if (callback != null) {
+            blobstoreCallback.set(callback);
+        }
+        clusterState.sync_path(Cluster.BLOBSTORE_SUBTREE);
+        return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, callback != null);
+
+    }
+
+    @Override
+    public void removeStorm(String stormId) {
+        clusterState.delete_node(Cluster.assignmentPath(stormId));
+        clusterState.delete_node(Cluster.credentialsPath(stormId));
+        clusterState.delete_node(Cluster.logConfigPath(stormId));
+        clusterState.delete_node(Cluster.profilerConfigPath(stormId));
+        removeStormBase(stormId);
+    }
+
+    @Override
+    public void removeBlobstoreKey(String blobKey) {
+        LOG.debug("remove key {}", blobKey);
+        clusterState.delete_node(Cluster.blobstorePath(blobKey));
+    }
+
+    @Override
+    public void removeKeyVersion(String blobKey) {
+        clusterState.delete_node(Cluster.blobstoreMaxKeySequenceNumberPath(blobKey));
+    }
+
+    @Override
+    public void reportError(String stormId, String componentId, String node, Long port, String error) {
+
+        try {
+            String path = Cluster.errorPath(stormId, componentId);
+            String lastErrorPath = Cluster.lastErrorPath(stormId, componentId);
+            ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
+            errorInfo.set_host(node);
+            errorInfo.set_port(port.intValue());
+            byte[] serData = Utils.serialize(errorInfo);
+            clusterState.mkdirs(path, acls);
+            clusterState.create_sequential(path + Cluster.ZK_SEPERATOR + "e", serData, acls);
+            clusterState.set_data(lastErrorPath, serData, acls);
+            List<String> childrens = clusterState.get_children(path, false);
+
+            Collections.sort(childrens);
+
+            while (childrens.size() >= 10) {
+                clusterState.delete_node(path + Cluster.ZK_SEPERATOR + childrens.remove(0));
+            }
+        } catch (UnsupportedEncodingException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    @Override
+    public List<ErrorInfo> errors(String stormId, String componentId) {
+        List<ErrorInfo> errorInfos = new ArrayList<>();
+        try {
+            String path = Cluster.errorPath(stormId, componentId);
+            if (clusterState.node_exists(path, false)) {
+                List<String> childrens = clusterState.get_children(path, false);
+                for (String child : childrens) {
+                    String childPath = path + Cluster.ZK_SEPERATOR + child;
+                    ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(childPath, false), ErrorInfo.class);
+                    if (errorInfo != null)
+                        errorInfos.add(errorInfo);
+                }
+            }
+            Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+                public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+                    return Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+                }
+            });
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+
+        return errorInfos;
+    }
+
+    @Override
+    public ErrorInfo lastError(String stormId, String componentId) {
+        try {
+            String path = Cluster.lastErrorPath(stormId, componentId);
+            if (clusterState.node_exists(path, false)) {
+                ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(path, false), ErrorInfo.class);
+                return errorInfo;
+            }
+        } catch (UnsupportedEncodingException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+        return null;
+    }
+
+    @Override
+    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = Cluster.mkTopoOnlyAcls(topoConf);
+        String path = Cluster.credentialsPath(stormId);
+        clusterState.set_data(path, Utils.serialize(creds), aclList);
+
+    }
+
+    @Override
+    public Credentials credentials(String stormId, Callback callback) {
+        if (callback != null) {
+            credentialsCallback.put(stormId, callback);
+        }
+        String path = Cluster.credentialsPath(stormId);
+        return Cluster.maybeDeserialize(clusterState.get_data(path, callback != null), Credentials.class);
+
+    }
+
+    @Override
+    public void disconnect() {
+        clusterState.unregister(stateId);
+        if (solo)
+            clusterState.close();
+    }
+}


[12/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils

Conflicts:
	storm-core/src/clj/org/apache/storm/cluster.clj
	storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
	storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
	storm-core/src/clj/org/apache/storm/daemon/common.clj
	storm-core/src/clj/org/apache/storm/daemon/executor.clj
	storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
	storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
	storm-core/src/clj/org/apache/storm/stats.clj
	storm-core/src/clj/org/apache/storm/testing.clj
	storm-core/src/clj/org/apache/storm/thrift.clj
	storm-core/src/clj/org/apache/storm/util.clj
	storm-core/src/clj/org/apache/storm/zookeeper.clj
	storm-core/test/clj/integration/org/apache/storm/integration_test.clj
	storm-core/test/clj/org/apache/storm/cluster_test.clj
	storm-core/test/clj/org/apache/storm/nimbus_test.clj
	storm-core/test/clj/org/apache/storm/supervisor_test.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9a8962de
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9a8962de
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9a8962de

Branch: refs/heads/master
Commit: 9a8962de9c80fc6a5388fd6d63ab225268530adf
Parents: 2ee8bec 12ceb09
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 15:13:33 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 15 22:15:21 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |  16 +
 README.markdown                                 |   1 +
 bin/storm-config.cmd                            |   6 +-
 bin/storm.cmd                                   |  47 +-
 bin/storm.py                                    |   8 +-
 conf/defaults.yaml                              |   4 +
 dev-tools/travis/travis-script.sh               |   4 +-
 .../starter/trident/TridentMapExample.java      | 123 +++
 external/sql/storm-sql-core/pom.xml             |   9 +
 external/storm-elasticsearch/pom.xml            |   2 +
 .../storm/hbase/security/HBaseSecurityUtil.java |  36 +-
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  |   8 +-
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  |  44 +-
 .../apache/storm/kafka/PartitionManager.java    |  42 +-
 .../kafka/trident/TridentKafkaEmitter.java      |  23 +-
 external/storm-mqtt/core/pom.xml                |   4 +-
 log4j2/cluster.xml                              |   2 +-
 log4j2/worker.xml                               |   2 +-
 pom.xml                                         |   9 +-
 storm-core/pom.xml                              |  11 +-
 .../src/clj/org/apache/storm/LocalCluster.clj   |   4 +-
 storm-core/src/clj/org/apache/storm/clojure.clj |   8 +-
 .../clj/org/apache/storm/command/blobstore.clj  |  11 +-
 .../org/apache/storm/command/config_value.clj   |  25 -
 .../org/apache/storm/command/dev_zookeeper.clj  |   6 +-
 .../clj/org/apache/storm/command/get_errors.clj |  12 +-
 .../apache/storm/command/shell_submission.clj   |   4 +-
 storm-core/src/clj/org/apache/storm/config.clj  |  18 +-
 .../src/clj/org/apache/storm/converter.clj      |  14 +-
 .../src/clj/org/apache/storm/daemon/acker.clj   |  21 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |  46 +-
 .../src/clj/org/apache/storm/daemon/drpc.clj    |  25 +-
 .../clj/org/apache/storm/daemon/executor.clj    | 530 +++++-----
 .../clj/org/apache/storm/daemon/logviewer.clj   |  70 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 168 ++--
 .../clj/org/apache/storm/daemon/supervisor.clj  | 205 ++--
 .../src/clj/org/apache/storm/daemon/task.clj    |   2 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  78 +-
 .../src/clj/org/apache/storm/disruptor.clj      |  10 +-
 storm-core/src/clj/org/apache/storm/event.clj   |   2 +-
 .../src/clj/org/apache/storm/local_state.clj    |   9 +-
 .../clj/org/apache/storm/messaging/loader.clj   |  34 -
 .../clj/org/apache/storm/messaging/local.clj    |  23 -
 .../org/apache/storm/pacemaker/pacemaker.clj    |   7 +-
 .../storm/pacemaker/pacemaker_state_factory.clj | 122 ---
 .../clj/org/apache/storm/process_simulator.clj  |   4 +-
 .../apache/storm/scheduler/DefaultScheduler.clj |   7 +-
 .../apache/storm/scheduler/EvenScheduler.clj    |  23 +-
 .../storm/scheduler/IsolationScheduler.clj      |  29 +-
 storm-core/src/clj/org/apache/storm/stats.clj   |  82 +-
 storm-core/src/clj/org/apache/storm/testing.clj |  89 +-
 storm-core/src/clj/org/apache/storm/thrift.clj  |   6 +-
 storm-core/src/clj/org/apache/storm/timer.clj   |  12 +-
 .../clj/org/apache/storm/trident/testing.clj    |   9 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |  99 +-
 .../src/clj/org/apache/storm/ui/helpers.clj     |  14 +-
 storm-core/src/clj/org/apache/storm/util.clj    | 925 +----------------
 storm-core/src/jvm/org/apache/storm/Config.java |  39 +
 .../storm/cluster/PaceMakerStateStorage.java    |   4 +-
 .../org/apache/storm/command/ConfigValue.java   |  30 +
 .../storm/daemon/metrics/MetricsUtils.java      | 108 ++
 .../reporters/ConsolePreparableReporter.java    |  76 ++
 .../reporters/CsvPreparableReporter.java        |  80 ++
 .../reporters/JmxPreparableReporter.java        |  70 ++
 .../metrics/reporters/PreparableReporter.java   |  32 +
 .../storm/logging/ThriftAccessLogger.java       |  13 +-
 .../apache/storm/pacemaker/PacemakerClient.java |   5 +
 .../serialization/SerializationFactory.java     |  17 +-
 .../staticmocking/MockedConfigUtils.java        |  31 -
 .../jvm/org/apache/storm/trident/Stream.java    |  87 +-
 .../storm/trident/operation/Consumer.java       |  35 +
 .../trident/operation/FlatMapFunction.java      |  37 +
 .../storm/trident/operation/MapFunction.java    |  36 +
 .../operation/impl/ConsumerExecutor.java        |  38 +
 .../operation/impl/FlatMapFunctionExecutor.java |  43 +
 .../operation/impl/MapFunctionExecutor.java     |  41 +
 .../trident/planner/processor/MapProcessor.java |  87 ++
 .../jvm/org/apache/storm/utils/ConfigUtils.java |  20 +-
 .../jvm/org/apache/storm/utils/Container.java   |  11 +-
 .../jvm/org/apache/storm/utils/IPredicate.java  |  27 +
 .../org/apache/storm/utils/NimbusClient.java    |   2 +-
 .../utils/StormConnectionStateConverter.java    |  44 +
 .../jvm/org/apache/storm/utils/TestUtils.java   |  34 -
 .../src/jvm/org/apache/storm/utils/Time.java    |  26 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   | 989 +++++++++++++++++--
 .../storm/validation/ConfigValidation.java      |   2 +-
 .../org/apache/storm/zookeeper/Zookeeper.java   |   7 +
 .../org/apache/storm/integration_test.clj       |  98 +-
 .../org/apache/storm/testing4j_test.clj         |  37 +-
 .../apache/storm/trident/integration_test.clj   |  15 +-
 .../test/clj/org/apache/storm/cluster_test.clj  |  27 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |  23 +-
 .../clj/org/apache/storm/logviewer_test.clj     | 267 ++---
 .../storm/messaging/netty_integration_test.clj  |   2 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   | 112 ++-
 .../storm/pacemaker_state_factory_test.clj      |  74 +-
 .../scheduler/resource_aware_scheduler_test.clj |  21 +-
 .../apache/storm/security/auth/auth_test.clj    |  11 +-
 .../authorizer/DRPCSimpleACLAuthorizer_test.clj |   2 +-
 .../BlowfishTupleSerializer_test.clj            |   1 -
 .../clj/org/apache/storm/serialization_test.clj |  23 +-
 .../clj/org/apache/storm/supervisor_test.clj    | 649 ++++++------
 .../clj/org/apache/storm/transactional_test.clj |  18 +
 .../clj/org/apache/storm/trident/state_test.clj |   5 +-
 .../clj/org/apache/storm/trident/tuple_test.clj |  15 +-
 .../test/clj/org/apache/storm/utils_test.clj    |  16 +-
 .../test/clj/org/apache/storm/worker_test.clj   |   1 -
 .../staticmocking/ConfigUtilsInstaller.java     |  38 +
 .../utils/staticmocking/UtilsInstaller.java     |  38 +
 .../storm/utils/staticmocking/package-info.java |  95 ++
 110 files changed, 4199 insertions(+), 2614 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index 7be526d,657e242..ab5cbed
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@@ -14,7 -14,8 +14,8 @@@
  ;; See the License for the specific language governing permissions and
  ;; limitations under the License.
  (ns org.apache.storm.command.dev-zookeeper
+   (:import [org.apache.storm.utils Utils])
 -  (:use [org.apache.storm zookeeper util config])
 +  (:use [org.apache.storm util config])
    (:import [org.apache.storm.utils ConfigUtils])
    (:import [org.apache.storm.zookeeper Zookeeper])
    (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 3978d2f,0253338..870e7f6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -15,8 -15,9 +15,9 @@@
  ;; limitations under the License.
  (ns org.apache.storm.command.shell-submission
    (:import [org.apache.storm StormSubmitter]
+            [org.apache.storm.utils Utils]
             [org.apache.storm.zookeeper Zookeeper])
 -  (:use [org.apache.storm thrift util config log zookeeper])
 +  (:use [org.apache.storm thrift util config log])
    (:require [clojure.string :as str])
    (:import [org.apache.storm.utils ConfigUtils])
    (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index b144f40,eb1ec1e..db342d2
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,17 -15,19 +15,20 @@@
  ;; limitations under the License.
  (ns org.apache.storm.daemon.common
    (:use [org.apache.storm log config util])
 -  (:import [org.apache.storm.generated StormTopology
 +  (:import [org.apache.storm.generated StormTopology NodeInfo
              InvalidTopologyException GlobalStreamId]
-            [org.apache.storm.utils ThriftTopologyUtils])
-   (:import [org.apache.storm.utils Utils ConfigUtils])
+            [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
+            [org.apache.storm.daemon.metrics.reporters PreparableReporter]
+            [com.codahale.metrics MetricRegistry])
+   (:import [org.apache.storm.daemon.metrics MetricsUtils])
    (:import [org.apache.storm.task WorkerTopologyContext])
    (:import [org.apache.storm Constants])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl])
    (:import [org.apache.storm.metric SystemBolt])
    (:import [org.apache.storm.metric EventLoggerBolt])
-   (:import [org.apache.storm.security.auth IAuthorizer]) 
-   (:import [java.io InterruptedIOException])
+   (:import [org.apache.storm.security.auth IAuthorizer])
+   (:import [java.io InterruptedIOException]
+            [org.json.simple JSONValue])
    (:require [clojure.set :as set])  
    (:require [org.apache.storm.daemon.acker :as acker])
    (:require [org.apache.storm.thrift :as thrift])
@@@ -73,12 -83,10 +84,11 @@@
  (defn new-executor-stats []
    (ExecutorStats. 0 0 0 0 0))
  
 +
  (defn get-storm-id [storm-cluster-state storm-name]
-   (let [active-storms (.activeStorms storm-cluster-state)]
-     (find-first
-       #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
-       active-storms)
 -  (let [active-storms (.active-storms storm-cluster-state)
 -        pred  (reify IPredicate (test [this x] (= storm-name (:storm-name (.storm-base storm-cluster-state x nil)))))]
++  (let [active-storms (.activeStorms storm-cluster-state)
++        pred  (reify IPredicate (test [this x] (= storm-name (.get_name (.stormBase storm-cluster-state x nil)))))]
+     (Utils/findOne pred active-storms)
      ))
  
  (defn topology-bases [storm-cluster-state]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 49ae6cf,e2380b7..33b89ed
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -34,10 -34,13 +34,12 @@@
    (:import [org.apache.storm.daemon Shutdownable])
    (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
    (:import [org.apache.storm Config Constants])
 -  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
 +  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
    (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
-   (:import [java.util.concurrent ConcurrentLinkedQueue])
+   (:import [java.lang Thread Thread$UncaughtExceptionHandler]
+            [java.util.concurrent ConcurrentLinkedQueue]
+            [org.json.simple JSONValue])
 -  (:require [org.apache.storm [thrift :as thrift]
 -             [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
 +  (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
    (:require [org.apache.storm.daemon [task :as task]])
    (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
    (:require [clojure.set :as set]))
@@@ -206,9 -211,9 +210,9 @@@
        (swap! interval-errors inc)
  
        (when (<= @interval-errors max-per-interval)
 -        (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
 +        (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
-                               (hostname storm-conf)
+                               (Utils/hostname storm-conf)
 -                              (.getThisWorkerPort (:worker-context executor)) error)
 +          (long (.getThisWorkerPort (:worker-context executor))) error)
          ))))
  
  ;; in its own function so that it can be mocked out by tracked topologies

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index daf5e45,710cd83..6af1b81
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -38,9 -39,9 +39,9 @@@
    (:import [org.apache.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
              Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
    (:import [org.apache.storm.nimbus NimbusInfo])
-   (:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
+   (:import [org.apache.storm.utils TimeCacheMap Time TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
              BufferFileInputStream BufferInputStream])
 -  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
 +  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
              ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
              KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
              ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@@ -407,10 -414,10 +409,10 @@@
    [storm-cluster-state]
  
    (let [assignments (.assignments storm-cluster-state nil)]
-     (defaulted
+     (or
        (apply merge-with set/union
               (for [a assignments
 -                   [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
 +                   [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
                 {node #{port}}
                 ))
        {})
@@@ -988,10 -1002,10 +1002,10 @@@
          topology (system-topology! storm-conf (read-storm-topology storm-id blob-store))
          num-executors (->> (all-components topology) (map-val num-start-executors))]
      (log-message "Activating " storm-name ": " storm-id)
 -    (.activate-storm! storm-cluster-state
 +    (.activateStorm storm-cluster-state
                        storm-id
 -                      (StormBase. storm-name
 +      (thriftify-storm-base (StormBase. storm-name
-                                   (current-time-secs)
+                                   (Time/currentTimeSecs)
                                    {:type topology-initial-status}
                                    (storm-conf TOPOLOGY-WORKERS)
                                    num-executors
@@@ -1137,9 -1152,9 +1152,9 @@@
          (when-not (empty? to-cleanup-ids)
            (doseq [id to-cleanup-ids]
              (log-message "Cleaning up " id)
 -            (.teardown-heartbeats! storm-cluster-state id)
 -            (.teardown-topology-errors! storm-cluster-state id)
 +            (.teardownHeartbeats storm-cluster-state id)
 +            (.teardownTopologyErrors storm-cluster-state id)
-             (rmr (ConfigUtils/masterStormDistRoot conf id))
+             (Utils/forceDelete (ConfigUtils/masterStormDistRoot conf id))
              (blob-rm-topology-keys id blob-store storm-cluster-state)
              (swap! (:heartbeats-cache nimbus) dissoc id)))))
      (log-message "not a leader, skipping cleanup")))
@@@ -1811,8 -1830,8 +1838,8 @@@
                                           (.set_used_cpu sup-sum used-cpu))
                                         (when-let [version (:version info)] (.set_version sup-sum version))
                                         sup-sum))
-               nimbus-uptime ((:uptime nimbus))
+               nimbus-uptime (. (:uptime nimbus) upTime)
 -              bases (topology-bases storm-cluster-state)
 +              bases (nimbus-topology-bases storm-cluster-state)
                nimbuses (.nimbuses storm-cluster-state)
  
                ;;update the isLeader field for each nimbus summary

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 3a83d03,ae9e92f..c1f058f
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -16,14 -16,15 +16,15 @@@
  (ns org.apache.storm.daemon.supervisor
    (:import [java.io File IOException FileOutputStream])
    (:import [org.apache.storm.scheduler ISupervisor]
-            [org.apache.storm.utils LocalState Time Utils ConfigUtils]
+            [org.apache.storm.utils LocalState Time Utils Utils$ExitCodeCallable
+                                    ConfigUtils]
             [org.apache.storm.daemon Shutdownable]
             [org.apache.storm Constants]
 -           [org.apache.storm.cluster ClusterStateContext DaemonType]
 +           [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
             [java.net JarURLConnection]
-            [java.net URI]
+            [java.net URI URLDecoder]
             [org.apache.commons.io FileUtils])
 -  (:use [org.apache.storm config util log timer local-state])
 +  (:use [org.apache.storm config util log timer local-state converter])
    (:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
    (:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
    (:import [java.nio.file Files StandardCopyOption])
@@@ -315,12 -319,14 +321,12 @@@
     :shared-context shared-context
     :isupervisor isupervisor
     :active (atom true)
-    :uptime (uptime-computer)
+    :uptime (Utils/makeUptimeComputer)
     :version STORM-VERSION
     :worker-thread-pids-atom (atom {})
 -   :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
 -                                                                     (Utils/isZkAuthenticationConfiguredStormServer
 -                                                                       conf)
 -                                                                     SUPERVISOR-ZK-ACLS)
 -                                                        :context (ClusterStateContext. DaemonType/SUPERVISOR))
 +   :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
 +                                                     SUPERVISOR-ZK-ACLS)
 +                                                        (ClusterStateContext. DaemonType/SUPERVISOR))
     :local-state (ConfigUtils/supervisorState conf)
     :supervisor-id (.getSupervisorId isupervisor)
     :assignment-id (.getAssignmentId isupervisor)
@@@ -777,19 -791,19 +792,19 @@@
          synchronize-blobs-fn (update-blobs-for-all-topologies-fn supervisor)
          downloaded-storm-ids (set (read-downloaded-storm-ids conf))
          run-profiler-actions-fn (mk-run-profiler-actions-for-all-topologies supervisor)
 -        heartbeat-fn (fn [] (.supervisor-heartbeat!
 +        heartbeat-fn (fn [] (.supervisorHeartbeat
                                 (:storm-cluster-state supervisor)
                                 (:supervisor-id supervisor)
-                               (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
 -                               (->SupervisorInfo (Time/currentTimeSecs)
++                               (thriftify-supervisor-info (->SupervisorInfo (Time/currentTimeSecs)
                                                   (:my-hostname supervisor)
                                                   (:assignment-id supervisor)
                                                   (keys @(:curr-assignment supervisor))
                                                    ;; used ports
                                                   (.getMetadata isupervisor)
                                                   (conf SUPERVISOR-SCHEDULER-META)
-                                                  ((:uptime supervisor))
+                                                  (. (:uptime supervisor) upTime)
                                                   (:version supervisor)
 -                                                 (mk-supervisor-capacities conf))))]
 +                                                 (mk-supervisor-capacities conf)))))]
      (heartbeat-fn)
  
      ;; should synchronize supervisor so it doesn't launch anything after being down (optimization)
@@@ -1162,7 -1183,7 +1184,8 @@@
        (.readBlobTo blob-store (ConfigUtils/masterStormConfKey storm-id) (FileOutputStream. (ConfigUtils/supervisorStormConfPath tmproot)) nil)
        (finally
          (.shutdown blob-store)))
--    (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
++    (try (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) (catch Exception e))
++
      (setup-storm-code-dir conf (clojurify-structure (ConfigUtils/readSupervisorStormConf conf storm-id)) stormroot)
      (let [classloader (.getContextClassLoader (Thread/currentThread))
            resources-jar (resources-jar)

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index ae5be57,fe8cfae..9863427
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,16 -19,18 +19,18 @@@
    (:require [clj-time.core :as time])
    (:require [clj-time.coerce :as coerce])
    (:require [org.apache.storm.daemon [executor :as executor]])
 -  (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
 +  (:require [org.apache.storm [disruptor :as disruptor]])
    (:require [clojure.set :as set])
-   (:require [org.apache.storm.messaging.loader :as msg-loader])
    (:import [java.util.concurrent Executors]
-            [org.apache.storm.hooks IWorkerHook BaseWorkerHook])
-   (:import [java.util ArrayList HashMap])
-   (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+            [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
+            [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
+   (:import [java.util ArrayList HashMap]
+            [java.util.concurrent.locks ReentrantReadWriteLock])
+   (:import [org.apache.commons.io FileUtils])
+   (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue Time])
    (:import [org.apache.storm.grouping LoadMapping])
    (:import [org.apache.storm.messaging TransportFactory])
-   (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
+   (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status DeserializingConnectionCallback])
    (:import [org.apache.storm.daemon Shutdownable])
    (:import [org.apache.storm.serialization KryoTupleSerializer])
    (:import [org.apache.storm.generated StormTopology])

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index a36da3a,be4361a..0000000
deleted file mode 100644,100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ /dev/null
@@@ -1,122 -1,141 +1,0 @@@
--;; 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.
--
--(ns org.apache.storm.pacemaker.pacemaker-state-factory
--  (:require [org.apache.storm.pacemaker pacemaker]
 -            [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
--            [org.apache.storm
--             [config :refer :all]
 -             [cluster :refer :all]
--             [log :refer :all]
--             [util :as util]])
--  (:import [org.apache.storm.generated
--            HBExecutionException HBServerMessageType HBMessage
--            HBMessageData HBPulse]
-            [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
 -           [org.apache.storm.cluster_state zookeeper_state_factory]
 -           [org.apache.storm.cluster ClusterState]
--           [org.apache.storm.pacemaker PacemakerClient])
--  (:gen-class
-     :implements [org.apache.storm.cluster.StateStorageFactory]))
 -   :implements [org.apache.storm.cluster.ClusterStateFactory]))
--
--;; So we can mock the client for testing
--(defn makeClient [conf]
--  (PacemakerClient. conf))
--
--(defn makeZKState [conf auth-conf acls context]
-   (ClusterUtils/mkStateStorage conf auth-conf acls context))
 -  (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
--
--(def max-retries 10)
 -
 -(defn retry-on-exception
 -  "Retries specific function on exception based on retries count"
 -  [retries task-description f & args]
 -  (let [res (try {:value (apply f args)}
 -                 (catch Exception e
 -                   (if (<= 0 retries)
 -                     (throw e)
 -                     {:exception e})))]
 -    (if (:exception res)
 -      (do 
 -        (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
 -        (recur (dec retries) task-description f args))
 -      (do 
 -        (log-debug (str "Successful " task-description "."))
 -        (:value res)))))
--
--(defn -mkState [this conf auth-conf acls context]
--  (let [zk-state (makeZKState conf auth-conf acls context)
--        pacemaker-client (makeClient conf)]
--
--    (reify
-       IStateStorage
 -      ClusterState
--      ;; Let these pass through to the zk-state. We only want to handle heartbeats.
--      (register [this callback] (.register zk-state callback))
--      (unregister [this callback] (.unregister zk-state callback))
--      (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
--      (create_sequential [this path data acls] (.create_sequential zk-state path data acls))
--      (set_data [this path data acls] (.set_data zk-state path data acls))
--      (delete_node [this path] (.delete_node zk-state path))
--      (delete_node_blobstore [this path nimbus-host-port-info] (.delete_node_blobstore zk-state path nimbus-host-port-info))
--      (get_data [this path watch?] (.get_data zk-state path watch?))
--      (get_data_with_version [this path watch?] (.get_data_with_version zk-state path watch?))
--      (get_version [this path watch?] (.get_version zk-state path watch?))
--      (get_children [this path watch?] (.get_children zk-state path watch?))
--      (mkdirs [this path acls] (.mkdirs zk-state path acls))
--      (node_exists [this path watch?] (.node_exists zk-state path watch?))
--      (add_listener [this listener] (.add_listener zk-state listener))
--      (sync_path [this path] (.sync_path zk-state path))
--      
--      (set_worker_hb [this path data acls]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "set_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/SEND_PULSE
--                                   (HBMessageData/pulse
--                                    (doto (HBPulse.)
--                                      (.set_id path)
--                                      (.set_details data)))))]
--            (if (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE)
--              :ok
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--
--      (delete_worker_hb [this path]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "delete_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/DELETE_PATH
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE)
--              :ok
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (get_worker_hb [this path watch?]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "get_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/GET_PULSE
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE)
--              (try 
--                (.get_details (.get_pulse (.get_data response)))
--                (catch Exception e
--                  (throw (HBExecutionException. (.toString e)))))
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (get_worker_hb_children [this path watch?]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "get_worker_hb_children"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE)
--              (try
--                (into [] (.get_pulseIds (.get_nodes (.get_data response))))
--                (catch Exception e
--                  (throw (HBExecutionException. (.toString e)))))
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (close [this]
--        (.close zk-state)
--        (.close pacemaker-client)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/stats.clj
index 0bf1757,8b37fc3..8632ed3
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@@ -24,8 -24,8 +24,9 @@@
              ExecutorAggregateStats SpecificAggregateStats
              SpoutAggregateStats TopologyPageInfo TopologyStats])
    (:import [org.apache.storm.utils Utils])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl])
-   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+            [java.util Collection])
    (:use [org.apache.storm log util])
    (:use [clojure.math.numeric-tower :only [ceil]]))
  

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 5a0bdf2,c872742..eef7754
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -44,11 -45,12 +45,12 @@@
    (:import [org.apache.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor])
    (:import [org.apache.storm.tuple Tuple])
    (:import [org.apache.storm.generated StormTopology])
-   (:import [org.apache.storm.task TopologyContext])
+   (:import [org.apache.storm.task TopologyContext]
+            (org.apache.storm.messaging IContext)
+            [org.json.simple JSONValue])
 -  (:require [org.apache.storm [zookeeper :as zk]])
 +  (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
-   (:require [org.apache.storm.messaging.loader :as msg-loader])
    (:require [org.apache.storm.daemon.acker :as acker])
 -  (:use [org.apache.storm cluster util thrift config log local-state]))
 +  (:use [org.apache.storm util thrift config log local-state converter]))
  
  (defn feeder-spout
    [fields]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/thrift.clj
index 4dc21f9,779c1d1..7aab729
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@@ -29,8 -29,9 +29,9 @@@
    (:import [org.apache.storm.grouping CustomStreamGrouping])
    (:import [org.apache.storm.topology TopologyBuilder])
    (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
-   (:import [org.apache.thrift.transport TTransport])
+   (:import [org.apache.thrift.transport TTransport]
+            (org.json.simple JSONValue))
 -  (:use [org.apache.storm util config log zookeeper]))
 +  (:use [org.apache.storm util config log]))
  
  (defn instantiate-java-object
    [^JavaObject obj]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/util.clj
index 165d8ee,f685d12..72778bb
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@@ -20,9 -20,8 +20,9 @@@
    (:import [java.io FileReader FileNotFoundException])
    (:import [java.nio.file Paths])
    (:import [org.apache.storm Config])
-   (:import [org.apache.storm.generated ErrorInfo])
-   (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
-             MutableObject MutableInt])
++ (:import [org.apache.storm.generated ErrorInfo])
+   (:import [org.apache.storm.utils Time ClojureTimerTask Utils
+             MutableObject])
    (:import [org.apache.storm.security.auth NimbusPrincipal])
    (:import [javax.security.auth Subject])
    (:import [java.util UUID Random ArrayList List Collections])
@@@ -262,58 -163,9 +164,19 @@@
                     (instance? Boolean x) (boolean x)
                     true x))
             s))
 +; move this func form convert.clj due to cyclic load dependency
 +(defn clojurify-error [^ErrorInfo error]
 +  (if error
 +    {
 +      :error (.get_error error)
 +      :time-secs (.get_error_time_secs error)
 +      :host (.get_host error)
 +      :port (.get_port error)
 +      }
 +    ))
  
- (defmacro with-file-lock
-   [path & body]
-   `(let [f# (File. ~path)
-          _# (.createNewFile f#)
-          rf# (RandomAccessFile. f# "rw")
-          lock# (.. rf# (getChannel) (lock))]
-      (try
-        ~@body
-        (finally
-          (.release lock#)
-          (.close rf#)))))
- 
- (defn tokenize-path
-   [^String path]
-   (let [toks (.split path "/")]
-     (vec (filter (complement empty?) toks))))
- 
- (defn assoc-conj
-   [m k v]
-   (merge-with concat m {k [v]}))
- 
- ;; returns [ones in first set not in second, ones in second set not in first]
- (defn set-delta
-   [old curr]
-   (let [s1 (set old)
-         s2 (set curr)]
-     [(set/difference s1 s2) (set/difference s2 s1)]))
- 
- (defn parent-path
-   [path]
-   (let [toks (tokenize-path path)]
-     (str "/" (str/join "/" (butlast toks)))))
- 
- (defn toks->path
-   [toks]
-   (str "/" (str/join "/" toks)))
- 
- (defn normalize-path
-   [^String path]
-   (toks->path (tokenize-path path)))
- 
+ ;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+ ;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
  (defn map-val
    [afn amap]
    (into {}

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index 1226c55,0000000..a9c4d89
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@@ -1,212 -1,0 +1,212 @@@
 +/**
 + * 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.APersistentMap;
 +import org.apache.curator.framework.state.ConnectionStateListener;
 +import org.apache.storm.callback.ZKStateChangedCallback;
 +import org.apache.storm.generated.*;
 +import org.apache.storm.pacemaker.PacemakerClient;
 +import org.apache.storm.utils.Utils;
 +import org.apache.zookeeper.data.ACL;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import java.util.List;
 +
 +public class PaceMakerStateStorage implements IStateStorage {
 +
 +    private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
 +
 +    private PacemakerClient pacemakerClient;
 +    private IStateStorage stateStorage;
 +    private static final int maxRetries = 10;
 +
 +    public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
 +        this.pacemakerClient = pacemakerClient;
 +        this.stateStorage = stateStorage;
 +    }
 +
 +    @Override
 +    public String register(ZKStateChangedCallback callback) {
 +        return stateStorage.register(callback);
 +    }
 +
 +    @Override
 +    public void unregister(String id) {
 +        stateStorage.unregister(id);
 +    }
 +
 +    @Override
 +    public String create_sequential(String path, byte[] data, List<ACL> acls) {
 +        return stateStorage.create_sequential(path, data, acls);
 +    }
 +
 +    @Override
 +    public void mkdirs(String path, List<ACL> acls) {
 +        stateStorage.mkdirs(path, acls);
 +    }
 +
 +    @Override
 +    public void delete_node(String path) {
 +        stateStorage.delete_node(path);
 +    }
 +
 +    @Override
 +    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
 +        stateStorage.set_ephemeral_node(path, data, acls);
 +    }
 +
 +    @Override
 +    public Integer get_version(String path, boolean watch) throws Exception {
 +        return stateStorage.get_version(path, watch);
 +    }
 +
 +    @Override
 +    public boolean node_exists(String path, boolean watch) {
 +        return stateStorage.node_exists(path, watch);
 +    }
 +
 +    @Override
 +    public List<String> get_children(String path, boolean watch) {
 +        return stateStorage.get_children(path, watch);
 +    }
 +
 +    @Override
 +    public void close() {
 +        stateStorage.close();
 +        pacemakerClient.close();
 +    }
 +
 +    @Override
 +    public void set_data(String path, byte[] data, List<ACL> acls) {
 +        stateStorage.set_data(path, data, acls);
 +    }
 +
 +    @Override
 +    public byte[] get_data(String path, boolean watch) {
 +        return stateStorage.get_data(path, watch);
 +    }
 +
 +    @Override
 +    public APersistentMap get_data_with_version(String path, boolean watch) {
 +        return stateStorage.get_data_with_version(path, watch);
 +    }
 +
 +    @Override
 +    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
 +                HBPulse hbPulse = new HBPulse();
 +                hbPulse.set_id(path);
 +                hbPulse.set_details(data);
 +                HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful set_worker_hb");
 +                break;
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public byte[] get_worker_hb(String path, boolean watch) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
 +                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                return response.get_data().get_pulse().get_details();
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public List<String> get_worker_hb_children(String path, boolean watch) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
-                 HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++                HBMessage message = new HBMessage(HBServerMessageType.GET_ALL_NODES_FOR_PATH, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                return response.get_data().get_nodes().get_pulseIds();
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public void delete_worker_hb(String path) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
-                 HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++                HBMessage message = new HBMessage(HBServerMessageType.DELETE_PATH, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                break;
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public void add_listener(ConnectionStateListener listener) {
 +        stateStorage.add_listener(listener);
 +    }
 +
 +    @Override
 +    public void sync_path(String path) {
 +        stateStorage.sync_path(path);
 +    }
 +
 +    @Override
 +    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
 +        stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index af0e8f3,34f3665..20d6deb
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@@ -60,6 -60,6 +60,10 @@@ public class PacemakerClient implement
      private StormBoundedExponentialBackoffRetry backoff = new StormBoundedExponentialBackoffRetry(100, 5000, 20);
      private int retryTimes = 0;
  
++    //the constructor is invoked by pacemaker-state-factory-test
++    public PacemakerClient() {
++        bootstrap = new ClientBootstrap();
++    }
      public PacemakerClient(Map config) {
  
          String host = (String)config.get(Config.PACEMAKER_HOST);
@@@ -157,6 -157,7 +161,7 @@@
      public String secretKey() {
          return secret;
      }
 -
++    public HBMessage  checkCaptured() {return null;}
      public HBMessage send(HBMessage m) {
          waitUntilReady();
          LOG.debug("Sending message: {}", m.toString());

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 39adb9e,b146cb0..6c32d54
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -22,11 -22,11 +22,11 @@@
    (:import [org.mockito Mockito])
    (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
-   (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils ZookeeperAuthInfo ConfigUtils])
 -  (:import [org.apache.storm.cluster ClusterState])
 +  (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
    (:import [org.apache.storm.zookeeper Zookeeper])
 -  (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
 -  (:require [org.apache.storm [zookeeper :as zk]])
 +  (:import [org.apache.storm.callback ZKStateChangedCallback])
 +  (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
    (:require [conjure.core])
    (:use [conjure core])
    (:use [clojure test])
@@@ -39,14 -39,18 +39,18 @@@
  
  (defn mk-state
    ([zk-port] (let [conf (mk-config zk-port)]
 -               (mk-distributed-cluster-state conf :auth-conf conf)))
 +               (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
    ([zk-port cb]
 -     (let [ret (mk-state zk-port)]
 -       (.register ret cb)
 -       ret )))
 +    (let [ret (mk-state zk-port)]
 +      (.register ret cb)
 +      ret)))
  
 -(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
 +(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
  
+ (defn barr
+   [& vals]
+   (byte-array (map byte vals)))
+ 
  (deftest test-basics
    (with-inprocess-zookeeper zk-port
      (let [state (mk-state zk-port)]
@@@ -242,27 -244,26 +246,32 @@@
        (is (.contains (:error error) target))
        )))
  
++(defn- stringify-error [error]
++  (let [result (java.io.StringWriter.)
++        printer (java.io.PrintWriter. result)]
++    (.printStackTrace error printer)
++    (.toString result)))
  
  (deftest test-storm-cluster-state-errors
    (with-inprocess-zookeeper zk-port
      (with-simulated-time
        (let [state (mk-storm-state zk-port)]
-         (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
 -        (.report-error state "a" "1" (Utils/localHostname) 6700 (RuntimeException.))
++        (.reportError state "a" "1" (Utils/localHostname) 6700 (stringify-error (RuntimeException.)))
          (validate-errors! state "a" "1" ["RuntimeException"])
          (advance-time-secs! 1)
-         (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
 -        (.report-error state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
++        (.reportError state "a" "1" (Utils/localHostname) 6700 (stringify-error (IllegalArgumentException.)))
          (validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
          (doseq [i (range 10)]
-           (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
 -          (.report-error state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
++          (.reportError state "a" "2" (Utils/localHostname) 6700 (stringify-error (RuntimeException.)))
            (advance-time-secs! 2))
          (validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
          (doseq [i (range 5)]
-           (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
 -          (.report-error state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
++          (.reportError state "a" "2" (Utils/localHostname) 6700 (stringify-error (IllegalArgumentException.)))
            (advance-time-secs! 2))
          (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
 -                                                (repeat 5 "RuntimeException")
 -                                                ))
 +                                          (repeat 5 "RuntimeException")
 +                                          ))
 +
          (.disconnect state)
          ))))
  
@@@ -300,12 -301,12 +309,12 @@@
        (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
        (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
        (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
-       (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
+       (Utils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
        (is (nil?
 -           (try
 -             (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
 -             (catch MockitoAssertionError e
 -               e)))))))
 +            (try
 +              (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
 +              (catch MockitoAssertionError e
 +                e)))))))
  
  (deftest test-storm-state-callbacks
    ;; TODO finish
@@@ -313,17 -314,15 +322,17 @@@
  
  (deftest test-cluster-state-default-acls
    (testing "The default ACLs are empty."
 -    (let [zk-mock (Mockito/mock Zookeeper)]
 +    (let [zk-mock (Mockito/mock Zookeeper)
 +          curator-frameworke (reify CuratorFramework (^void close [this] nil))]
        ;; No need for when clauses because we just want to return nil
        (with-open [_ (MockedZookeeper. zk-mock)]
 -        (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
 -          (mk-distributed-cluster-state {})
 -          (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
 -    (stubbing [mk-distributed-cluster-state (reify ClusterState
 -                                              (register [this callback] nil)
 -                                              (mkdirs [this path acls] nil))]
 -     (mk-storm-cluster-state {})
 -     (verify-call-times-for mk-distributed-cluster-state 1)
 -     (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
 +        (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
 +        (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
 +        (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
 +    (let [distributed-state-storage (reify IStateStorage
 +                                      (register [this callback] nil)
 +                                      (mkdirs [this path acls] nil))
 +          cluster-utils (Mockito/mock ClusterUtils)]
 +      (with-open [mocked-cluster (MockedCluster. cluster-utils)]
-         (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
++        (. (Mockito/when (.mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
 +        (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 772a232,70cb885..2a65efc
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -23,27 -23,33 +23,36 @@@
             [org.apache.storm.nimbus InMemoryTopologyActionNotifier])
    (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
    (:import [org.apache.storm.scheduler INimbus])
 +  (:import [org.mockito Mockito])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
-   (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
++  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
              TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
              InvalidTopologyException AuthorizationException
              LogConfig LogLevel LogLevelAction])
    (:import [java.util HashMap])
    (:import [java.io File])
-   (:import [org.apache.storm.utils Time Utils ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils Utils$UptimeComputer ConfigUtils IPredicate]
+            [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
    (:import [org.apache.storm.zookeeper Zookeeper])
-   (:import [org.apache.commons.io FileUtils])
+   (:import [org.apache.commons.io FileUtils]
+            [org.json.simple JSONValue])
 -  (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
 +  (:use [org.apache.storm testing MockAutoCred util config log timer converter])
    (:use [org.apache.storm.daemon common])
    (:require [conjure.core])
    (:require [org.apache.storm
 -             [thrift :as thrift]
 -             [cluster :as cluster]])
 +             [thrift :as thrift]])
    (:use [conjure core]))
  
+ (defn- from-json
+        [^String str]
+        (if str
+          (clojurify-structure
+            (JSONValue/parse str))
+          nil))
+ 
  (defn storm-component->task-info [cluster storm-name]
    (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
          nimbus (:nimbus cluster)]
@@@ -72,8 -80,8 +83,8 @@@
  
  (defn storm-num-workers [state storm-name]
    (let [storm-id (get-storm-id state storm-name)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (count (reverse-map (:executor->node+port assignment)))
+     (count (clojurify-structure (Utils/reverseMap (:executor->node+port assignment))))
      ))
  
  (defn topology-nodes [state storm-name]
@@@ -95,9 -103,11 +106,11 @@@
           set         
           )))
  
+ ;TODO: when translating this function, don't call map-val, but instead use an inline for loop.
+ ; map-val is a temporary kluge for clojure.
  (defn topology-node-distribution [state storm-name]
    (let [storm-id (get-storm-id state storm-name)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
      (->> assignment
           :executor->node+port
           vals
@@@ -124,19 -134,18 +137,18 @@@
  
  (defn do-executor-heartbeat [cluster storm-id executor]
    (let [state (:storm-cluster-state cluster)
 -        executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
 +        executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
          [node port] (get executor->node+port executor)
 -        curr-beat (.get-worker-heartbeat state storm-id node port)
 +        curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
          stats (:executor-stats curr-beat)]
 -    (.worker-heartbeat! state storm-id node port
 -      {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
 +    (.workerHeartbeat state storm-id node port
-       (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
++      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
        )))
  
  (defn slot-assignments [cluster storm-id]
    (let [state (:storm-cluster-state cluster)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (reverse-map (:executor->node+port assignment))
-     ))
+         (clojurify-structure (Utils/reverseMap (:executor->node+port assignment)))))
  
  (defn task-ids [cluster storm-id]
    (let [nimbus (:nimbus cluster)]
@@@ -146,8 -155,10 +158,10 @@@
  
  (defn topology-executors [cluster storm-id]
    (let [state (:storm-cluster-state cluster)
-         assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (keys (:executor->node+port assignment))
 -        assignment (.assignment-info state storm-id nil)
 -        ret-keys (keys (:executor->node+port assignment))
++        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
++    ret-keys (keys (:executor->node+port assignment))
+         _ (log-message "ret-keys: " (pr-str ret-keys)) ]
+     ret-keys
      ))
  
  (defn check-distribution [items distribution]
@@@ -1350,23 -1399,27 +1402,29 @@@
                       NIMBUS-THRIFT-PORT 6666})
            expected-acls nimbus/NIMBUS-ZK-ACLS
            fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+           fake-cu (proxy [ConfigUtils] []
 -                      (nimbusTopoHistoryStateImpl [conf] nil))
++                    (nimbusTopoHistoryStateImpl [conf] nil))
+           fake-utils (proxy [Utils] []
+                        (newInstanceImpl [_])
+                        (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
 -                                                (upTime [] 0))))]
++                                                (upTime [] 0))))
 +          cluster-utils (Mockito/mock ClusterUtils)]
-       (with-open [_ (proxy [MockedConfigUtils] []
+       (with-open [_ (ConfigUtilsInstaller. fake-cu)
+                   _ (UtilsInstaller. fake-utils)
++                  _ (proxy [ConfigUtils] []
 +                      (nimbusTopoHistoryStateImpl [conf] nil))
                    zk-le (MockedZookeeper. (proxy [Zookeeper] []
 -                          (zkLeaderElectorImpl [conf] nil)))]
 +                          (zkLeaderElectorImpl [conf] nil)))
 +                  mocked-cluster (MockedCluster. cluster-utils)]
          (stubbing [mk-authorization-handler nil
 -                   cluster/mk-storm-cluster-state nil
 -                   nimbus/file-cache-map nil
 -                   nimbus/mk-blob-cache-map nil
 -                   nimbus/mk-bloblist-cache-map nil
 -                   mk-timer nil
 -                   nimbus/mk-scheduler nil]
 -                  (nimbus/nimbus-data auth-conf fake-inimbus)
 -                  (verify-call-times-for cluster/mk-storm-cluster-state 1)
 -                  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
 -                                                      expected-acls))))))
 +                 nimbus/file-cache-map nil
 +                 nimbus/mk-blob-cache-map nil
 +                 nimbus/mk-bloblist-cache-map nil
-                  uptime-computer nil
-                  new-instance nil
 +                 mk-timer nil
 +                 nimbus/mk-scheduler nil]
 +          (nimbus/nimbus-data auth-conf fake-inimbus)
 +          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
 +          )))))
  
  (deftest test-file-bogus-download
    (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1397,9 -1450,9 +1455,9 @@@
                        STORM-CLUSTER-MODE "local"
                        STORM-ZOOKEEPER-PORT zk-port
                        STORM-LOCAL-DIR nimbus-dir}))
 -        (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +        (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-         (sleep-secs 1)
+         (Time/sleepSecs 1)
          (bind topology (thrift/mk-topology
                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                           {}))
@@@ -1429,10 -1482,10 +1487,10 @@@
                          STORM-ZOOKEEPER-PORT zk-port
                          STORM-LOCAL-DIR nimbus-dir
                          NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
 -          (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
            (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
            (bind notifier (InMemoryTopologyActionNotifier.))
-           (sleep-secs 1)
+           (Time/sleepSecs 1)
            (bind topology (thrift/mk-topology
                             {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                             {}))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 1a7bd2c,0925237..1c45266
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@@ -19,10 -20,8 +19,11 @@@
    (:import [org.apache.storm.generated
              HBExecutionException HBNodes HBRecords
              HBServerMessageType HBMessage HBMessageData HBPulse]
-            [org.apache.storm.cluster ClusterStateContext  PaceMakerStateStorageFactory]
 -           [org.apache.storm.cluster ClusterStateContext]
 -           [org.mockito Mockito Matchers]))
++           [org.apache.storm.cluster ClusterStateContext  PaceMakerStateStorageFactory PaceMakerStateStorage]
 +           [org.mockito Mockito Matchers])
 +(:import [org.mockito.exceptions.base MockitoAssertionError])
++(:import [org.apache.storm.pacemaker PacemakerClient])
 +(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
  
  (defn- string-to-bytes [string]
    (byte-array (map int string)))
@@@ -30,26 -29,24 +31,23 @@@
  (defn- bytes-to-string [bytez]
    (apply str (map char bytez)))
  
--(defprotocol send-capture
--  (send [this something])
--  (check-captured [this]))
--
  (defn- make-send-capture [response]
    (let [captured (atom nil)]
--    (reify send-capture
--      (send [this something] (reset! captured something) response)
--      (check-captured [this] @captured))))
 -
 -(defmacro with-mock-pacemaker-client-and-state [client state response & body]
 -  `(let [~client (make-send-capture ~response)]
 -     (stubbing [psf/makeZKState nil
 -                psf/makeClient ~client]
 -               (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
++    (proxy [PacemakerClient] []
++      (send [m] (reset! captured m) response)
++      (checkCaptured [] @captured))))
 +
 +(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
 +  `(let [~client (make-send-capture ~response)
 +         ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
 +
 +     (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
 +       (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
 +       (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
-                (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
++               (let [~state (PaceMakerStateStorage. (PaceMakerStateStorageFactory/initMakeClient nil)
++                   (PaceMakerStateStorageFactory/initZKstate nil  nil nil nil))]
                   ~@body))))
  
 -
  (deftest pacemaker_state_set_worker_hb
    (testing "set_worker_hb"
      (with-mock-pacemaker-client-and-state
@@@ -57,7 -54,7 +55,7 @@@
        (HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
  
        (.set_worker_hb state "/foo" (string-to-bytes "data") nil)
--      (let [sent (.check-captured client)
++      (let [sent (.checkCaptured client)
              pulse (.get_pulse (.get_data sent))]
          (is (= (.get_type sent) HBServerMessageType/SEND_PULSE))
          (is (= (.get_id pulse) "/foo"))
@@@ -65,13 -62,13 +63,12 @@@
  
    (testing "set_worker_hb"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/SEND_PULSE nil)
  
-       (is (thrown? RuntimeException      
 -      (is (thrown? HBExecutionException      
--                   (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
++      (is (thrown? RuntimeException
++            (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
  
--      
  
  (deftest pacemaker_state_delete_worker_hb
    (testing "delete_worker_hb"
@@@ -80,74 -77,74 +77,75 @@@
        (HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
  
        (.delete_worker_hb state "/foo/bar")
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/DELETE_PATH))
          (is (= (.get_path (.get_data sent)) "/foo/bar")))))
  
--    (testing "delete_worker_hb"
--      (with-mock-pacemaker-client-and-state
-         client state pacefactory mock
 -        client state
--        (HBMessage. HBServerMessageType/DELETE_PATH nil)
--        
-         (is (thrown? RuntimeException
 -        (is (thrown? HBExecutionException
--                     (.delete_worker_hb state "/foo/bar"))))))
++  (testing "delete_worker_hb"
++    (with-mock-pacemaker-client-and-state
++      client state pacefactory mock
++      (HBMessage. HBServerMessageType/DELETE_PATH nil)
++
++      (is (thrown? RuntimeException
++            (.delete_worker_hb state "/foo/bar"))))))
  
  (deftest pacemaker_state_get_worker_hb
    (testing "get_worker_hb"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
--                (HBMessageData/pulse
--                 (doto (HBPulse.)
--                   (.set_id "/foo")
--                   (.set_details (string-to-bytes "some data")))))
++        (HBMessageData/pulse
++          (doto (HBPulse.)
++            (.set_id "/foo")
++            (.set_details (string-to-bytes "some data")))))
  
        (.get_worker_hb state "/foo" false)
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/GET_PULSE))
          (is (= (.get_path (.get_data sent)) "/foo")))))
  
    (testing "get_worker_hb - fail (bad response)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE nil)
--      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb state "/foo" false)))))
 -  
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb state "/foo" false)))))
-   
++            (.get_worker_hb state "/foo" false)))))
++
    (testing "get_worker_hb - fail (bad data)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
--      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb state "/foo" false))))))
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb state "/foo" false))))))
++            (.get_worker_hb state "/foo" false))))))
  
  (deftest pacemaker_state_get_worker_hb_children
    (testing "get_worker_hb_children"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
--                (HBMessageData/nodes
--                 (HBNodes. [])))
++        (HBMessageData/nodes
++          (HBNodes. [])))
  
        (.get_worker_hb_children state "/foo" false)
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/GET_ALL_NODES_FOR_PATH))
          (is (= (.get_path (.get_data sent)) "/foo")))))
  
    (testing "get_worker_hb_children - fail (bad response)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/DELETE_PATH nil)
  
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb_children state "/foo" false)))))
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb_children state "/foo" false)))))
++            (.get_worker_hb_children state "/foo" false)))))
  
--    (testing "get_worker_hb_children - fail (bad data)"
++  (testing "get_worker_hb_children - fail (bad data)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
-       ;need been update due to HBExecutionException
 -      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb_children state "/foo" false))))))
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb_children state "/foo" false))))))
++            (.get_worker_hb_children state "/foo" false))))))
++


[04/27] storm git commit: delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass

Posted by bo...@apache.org.
delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/9a79fb7d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/9a79fb7d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/9a79fb7d

Branch: refs/heads/master
Commit: 9a79fb7de0e824e73c294738521e892f1d81fbb0
Parents: 682d31c
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 20:28:05 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 20:28:05 2016 +0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +-
 storm-core/src/clj/org/apache/storm/cluster.clj | 691 -------------------
 .../cluster_state/zookeeper_state_factory.clj   | 163 -----
 .../org/apache/storm/command/dev_zookeeper.clj  |   2 +-
 .../clj/org/apache/storm/command/heartbeats.clj |   6 +-
 .../apache/storm/command/shell_submission.clj   |   2 +-
 .../src/clj/org/apache/storm/converter.clj      |  14 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |  13 +-
 .../clj/org/apache/storm/daemon/executor.clj    |  12 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 138 ++--
 .../clj/org/apache/storm/daemon/supervisor.clj  |  35 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  43 +-
 .../storm/pacemaker/pacemaker_state_factory.clj |  12 +-
 storm-core/src/clj/org/apache/storm/stats.clj   |   3 +-
 storm-core/src/clj/org/apache/storm/testing.clj |  14 +-
 storm-core/src/clj/org/apache/storm/thrift.clj  |   2 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |   2 +-
 storm-core/src/clj/org/apache/storm/util.clj    |  11 +
 .../src/clj/org/apache/storm/zookeeper.clj      |  75 --
 .../jvm/org/apache/storm/callback/Callback.java |   3 +
 .../jvm/org/apache/storm/cluster/Cluster.java   |  38 +-
 .../org/apache/storm/cluster/ClusterState.java  |   2 +-
 .../storm/cluster/DistributedClusterState.java  |   7 +-
 .../apache/storm/cluster/StormClusterState.java |  34 +-
 .../storm/cluster/StormZkClusterState.java      | 109 +--
 .../testing/staticmocking/MockedCluster.java    |  31 +
 .../org/apache/storm/integration_test.clj       |  15 +-
 .../test/clj/org/apache/storm/cluster_test.clj  | 103 ++-
 .../test/clj/org/apache/storm/nimbus_test.clj   | 148 ++--
 .../storm/security/auth/nimbus_auth_test.clj    |   3 +-
 .../clj/org/apache/storm/supervisor_test.clj    |  29 +-
 .../test/jvm/org/apache/storm/ClusterTest.java  |  22 +
 32 files changed, 488 insertions(+), 1296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 8873d12..74605bb 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,7 +51,7 @@ storm.auth.simple-white-list.users: []
 storm.auth.simple-acl.users: []
 storm.auth.simple-acl.users.commands: []
 storm.auth.simple-acl.admins: []
-storm.cluster.state.store: "org.apache.storm.cluster_state.zookeeper_state_factory"
+storm.cluster.state.store: "org.apache.storm.cluster.StormZkClusterState"
 storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate"
 storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor"
 storm.workers.artifacts.dir: "workers-artifacts"

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/cluster.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/cluster.clj b/storm-core/src/clj/org/apache/storm/cluster.clj
deleted file mode 100644
index 152423a..0000000
--- a/storm-core/src/clj/org/apache/storm/cluster.clj
+++ /dev/null
@@ -1,691 +0,0 @@
-;; 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.
-
-(ns org.apache.storm.cluster
-  (:import [org.apache.zookeeper.data Stat ACL Id]
-           [org.apache.storm.generated SupervisorInfo Assignment StormBase ClusterWorkerHeartbeat ErrorInfo Credentials NimbusSummary
-            LogConfig ProfileAction ProfileRequest NodeInfo]
-           [java.io Serializable])
-  (:import [org.apache.zookeeper KeeperException KeeperException$NoNodeException ZooDefs ZooDefs$Ids ZooDefs$Perms])
-  (:import [org.apache.curator.framework CuratorFramework])
-  (:import [org.apache.storm.utils Utils])
-  (:import [org.apache.storm.cluster ClusterState ClusterStateContext ClusterStateListener ConnectionState])
-  (:import [java.security MessageDigest])
-  (:import [org.apache.zookeeper.server.auth DigestAuthenticationProvider])
-  (:import [org.apache.storm.nimbus NimbusInfo])
-  (:use [org.apache.storm util log config converter])
-  (:require [org.apache.storm [zookeeper :as zk]])
-  (:require [org.apache.storm.daemon [common :as common]]))
-
-(defn mk-topo-only-acls
-  [topo-conf]
-  (let [payload (.get topo-conf STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD)]
-    (when (Utils/isZkAuthenticationConfiguredTopology topo-conf)
-      [(first ZooDefs$Ids/CREATOR_ALL_ACL)
-       (ACL. ZooDefs$Perms/READ (Id. "digest" (DigestAuthenticationProvider/generateDigest payload)))])))
- 
-(defnk mk-distributed-cluster-state
-  [conf :auth-conf nil :acls nil :context (ClusterStateContext.)]
-  (let [clazz (Class/forName (or (conf STORM-CLUSTER-STATE-STORE)
-                                 "org.apache.storm.cluster_state.zookeeper_state_factory"))
-        state-instance (.newInstance clazz)]
-    (log-debug "Creating cluster state: " (.toString clazz))
-    (or (.mkState state-instance conf auth-conf acls context)
-        nil)))
-
-(defprotocol StormClusterState
-  (assignments [this callback])
-  (assignment-info [this storm-id callback])
-  (assignment-info-with-version [this storm-id callback])
-  (assignment-version [this storm-id callback])
-  ;returns key information under /storm/blobstore/key
-  (blobstore-info [this blob-key])
-  ;returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
-  (nimbuses [this])
-  ;adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
-  (add-nimbus-host! [this nimbus-id nimbus-summary])
-
-  (active-storms [this])
-  (storm-base [this storm-id callback])
-  (get-worker-heartbeat [this storm-id node port])
-  (get-worker-profile-requests [this storm-id nodeinfo thrift?])
-  (get-topology-profile-requests [this storm-id thrift?])
-  (set-worker-profile-request [this storm-id profile-request])
-  (delete-topology-profile-requests [this storm-id profile-request])
-  (executor-beats [this storm-id executor->node+port])
-  (supervisors [this callback])
-  (supervisor-info [this supervisor-id]) ;; returns nil if doesn't exist
-  (setup-heartbeats! [this storm-id])
-  (teardown-heartbeats! [this storm-id])
-  (teardown-topology-errors! [this storm-id])
-  (heartbeat-storms [this])
-  (error-topologies [this])
-  (set-topology-log-config! [this storm-id log-config])
-  (topology-log-config [this storm-id cb])
-  (worker-heartbeat! [this storm-id node port info])
-  (remove-worker-heartbeat! [this storm-id node port])
-  (supervisor-heartbeat! [this supervisor-id info])
-  (worker-backpressure! [this storm-id node port info])
-  (topology-backpressure [this storm-id callback])
-  (setup-backpressure! [this storm-id])
-  (remove-worker-backpressure! [this storm-id node port])
-  (activate-storm! [this storm-id storm-base])
-  (update-storm! [this storm-id new-elems])
-  (remove-storm-base! [this storm-id])
-  (set-assignment! [this storm-id info])
-  ;; sets up information related to key consisting of nimbus
-  ;; host:port and version info of the blob
-  (setup-blobstore! [this key nimbusInfo versionInfo])
-  (active-keys [this])
-  (blobstore [this callback])
-  (remove-storm! [this storm-id])
-  (remove-blobstore-key! [this blob-key])
-  (remove-key-version! [this blob-key])
-  (report-error [this storm-id component-id node port error])
-  (errors [this storm-id component-id])
-  (last-error [this storm-id component-id])
-  (set-credentials! [this storm-id creds topo-conf])
-  (credentials [this storm-id callback])
-  (disconnect [this]))
-
-(def ASSIGNMENTS-ROOT "assignments")
-(def CODE-ROOT "code")
-(def STORMS-ROOT "storms")
-(def SUPERVISORS-ROOT "supervisors")
-(def WORKERBEATS-ROOT "workerbeats")
-(def BACKPRESSURE-ROOT "backpressure")
-(def ERRORS-ROOT "errors")
-(def BLOBSTORE-ROOT "blobstore")
-; Stores the latest update sequence for a blob
-(def BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-ROOT "blobstoremaxkeysequencenumber")
-(def NIMBUSES-ROOT "nimbuses")
-(def CREDENTIALS-ROOT "credentials")
-(def LOGCONFIG-ROOT "logconfigs")
-(def PROFILERCONFIG-ROOT "profilerconfigs")
-
-(def ASSIGNMENTS-SUBTREE (str "/" ASSIGNMENTS-ROOT))
-(def STORMS-SUBTREE (str "/" STORMS-ROOT))
-(def SUPERVISORS-SUBTREE (str "/" SUPERVISORS-ROOT))
-(def WORKERBEATS-SUBTREE (str "/" WORKERBEATS-ROOT))
-(def BACKPRESSURE-SUBTREE (str "/" BACKPRESSURE-ROOT))
-(def ERRORS-SUBTREE (str "/" ERRORS-ROOT))
-;; Blobstore subtree /storm/blobstore
-(def BLOBSTORE-SUBTREE (str "/" BLOBSTORE-ROOT))
-(def BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-SUBTREE (str "/" BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-ROOT))
-(def NIMBUSES-SUBTREE (str "/" NIMBUSES-ROOT))
-(def CREDENTIALS-SUBTREE (str "/" CREDENTIALS-ROOT))
-(def LOGCONFIG-SUBTREE (str "/" LOGCONFIG-ROOT))
-(def PROFILERCONFIG-SUBTREE (str "/" PROFILERCONFIG-ROOT))
-
-(defn supervisor-path
-  [id]
-  (str SUPERVISORS-SUBTREE "/" id))
-
-(defn assignment-path
-  [id]
-  (str ASSIGNMENTS-SUBTREE "/" id))
-
-(defn blobstore-path
-  [key]
-  (str BLOBSTORE-SUBTREE "/" key))
-
-(defn blobstore-max-key-sequence-number-path
-  [key]
-  (str BLOBSTORE-MAX-KEY-SEQUENCE-NUMBER-SUBTREE "/" key))
-
-(defn nimbus-path
-  [id]
-  (str NIMBUSES-SUBTREE "/" id))
-
-(defn storm-path
-  [id]
-  (str STORMS-SUBTREE "/" id))
-
-(defn workerbeat-storm-root
-  [storm-id]
-  (str WORKERBEATS-SUBTREE "/" storm-id))
-
-(defn workerbeat-path
-  [storm-id node port]
-  (str (workerbeat-storm-root storm-id) "/" node "-" port))
-
-(defn backpressure-storm-root
-  [storm-id]
-  (str BACKPRESSURE-SUBTREE "/" storm-id))
-
-(defn backpressure-path
-  [storm-id node port]
-  (str (backpressure-storm-root storm-id) "/" node "-" port))
-
-(defn error-storm-root
-  [storm-id]
-  (str ERRORS-SUBTREE "/" storm-id))
-
-(defn error-path
-  [storm-id component-id]
-  (str (error-storm-root storm-id) "/" (url-encode component-id)))
-
-(def last-error-path-seg "last-error")
-
-(defn last-error-path
-  [storm-id component-id]
-  (str (error-storm-root storm-id)
-       "/"
-       (url-encode component-id)
-       "-"
-       last-error-path-seg))
-
-(defn credentials-path
-  [storm-id]
-  (str CREDENTIALS-SUBTREE "/" storm-id))
-
-(defn log-config-path
-  [storm-id]
-  (str LOGCONFIG-SUBTREE "/" storm-id))
-
-(defn profiler-config-path
-  ([storm-id]
-   (str PROFILERCONFIG-SUBTREE "/" storm-id))
-  ([storm-id host port request-type]
-   (str (profiler-config-path storm-id) "/" host "_" port "_" request-type)))
-
-(defn- issue-callback!
-  [cb-atom]
-  (let [cb @cb-atom]
-    (reset! cb-atom nil)
-    (when cb
-      (cb))))
-
-(defn- issue-map-callback!
-  [cb-atom id]
-  (let [cb (@cb-atom id)]
-    (swap! cb-atom dissoc id)
-    (when cb
-      (cb id))))
-
-(defn- maybe-deserialize
-  [ser clazz]
-  (when ser
-    (Utils/deserialize ser clazz)))
-
-(defrecord TaskError [error time-secs host port])
-
-(defn- parse-error-path
-  [^String p]
-  (Long/parseLong (.substring p 1)))
-
-(defn convert-executor-beats
-  "Ensures that we only return heartbeats for executors assigned to
-  this worker."
-  [executors worker-hb]
-  (let [executor-stats (:executor-stats worker-hb)]
-    (->> executors
-         (map (fn [t]
-                (if (contains? executor-stats t)
-                  {t {:time-secs (:time-secs worker-hb)
-                      :uptime (:uptime worker-hb)
-                      :stats (get executor-stats t)}})))
-         (into {}))))
-
-;; Watches should be used for optimization. When ZK is reconnecting, they're not guaranteed to be called.
-(defnk mk-storm-cluster-state
-  [cluster-state-spec :acls nil :context (ClusterStateContext.)]
-  (let [[solo? cluster-state] (if (instance? ClusterState cluster-state-spec)
-                                [false cluster-state-spec]
-                                [true (mk-distributed-cluster-state cluster-state-spec :auth-conf cluster-state-spec :acls acls :context context)])
-        assignment-info-callback (atom {})
-        assignment-info-with-version-callback (atom {})
-        assignment-version-callback (atom {})
-        supervisors-callback (atom nil)
-        backpressure-callback (atom {})   ;; we want to reigister a topo directory getChildren callback for all workers of this dir
-        assignments-callback (atom nil)
-        storm-base-callback (atom {})
-        blobstore-callback (atom nil)
-        credentials-callback (atom {})
-        log-config-callback (atom {})
-        state-id (.register
-                  cluster-state
-                  (fn [type path]
-                    (let [[subtree & args] (tokenize-path path)]
-                      (condp = subtree
-                         ASSIGNMENTS-ROOT (if (empty? args)
-                                             (issue-callback! assignments-callback)
-                                             (do
-                                               (issue-map-callback! assignment-info-callback (first args))
-                                               (issue-map-callback! assignment-version-callback (first args))
-                                               (issue-map-callback! assignment-info-with-version-callback (first args))))
-                         SUPERVISORS-ROOT (issue-callback! supervisors-callback)
-                         BLOBSTORE-ROOT (issue-callback! blobstore-callback) ;; callback register for blobstore
-                         STORMS-ROOT (issue-map-callback! storm-base-callback (first args))
-                         CREDENTIALS-ROOT (issue-map-callback! credentials-callback (first args))
-                         LOGCONFIG-ROOT (issue-map-callback! log-config-callback (first args))
-                         BACKPRESSURE-ROOT (issue-map-callback! backpressure-callback (first args))
-                         ;; this should never happen
-                         (exit-process! 30 "Unknown callback for subtree " subtree args)))))]
-    (doseq [p [ASSIGNMENTS-SUBTREE STORMS-SUBTREE SUPERVISORS-SUBTREE WORKERBEATS-SUBTREE ERRORS-SUBTREE BLOBSTORE-SUBTREE NIMBUSES-SUBTREE
-               LOGCONFIG-SUBTREE]]
-      (.mkdirs cluster-state p acls))
-    (reify
-      StormClusterState
-
-      (assignments
-        [this callback]
-        (when callback
-          (reset! assignments-callback callback))
-        (.get_children cluster-state ASSIGNMENTS-SUBTREE (not-nil? callback)))
-
-      (assignment-info
-        [this storm-id callback]
-        (when callback
-          (swap! assignment-info-callback assoc storm-id callback))
-        (clojurify-assignment (maybe-deserialize (.get_data cluster-state (assignment-path storm-id) (not-nil? callback)) Assignment)))
-
-      (assignment-info-with-version 
-        [this storm-id callback]
-        (when callback
-          (swap! assignment-info-with-version-callback assoc storm-id callback))
-        (let [{data :data version :version} 
-              (.get_data_with_version cluster-state (assignment-path storm-id) (not-nil? callback))]
-        {:data (clojurify-assignment (maybe-deserialize data Assignment))
-         :version version}))
-
-      (assignment-version 
-        [this storm-id callback]
-        (when callback
-          (swap! assignment-version-callback assoc storm-id callback))
-        (.get_version cluster-state (assignment-path storm-id) (not-nil? callback)))
-
-      ;; blobstore state
-      (blobstore
-        [this callback]
-        (when callback
-          (reset! blobstore-callback callback))
-        (.sync_path cluster-state BLOBSTORE-SUBTREE)
-        (.get_children cluster-state BLOBSTORE-SUBTREE (not-nil? callback)))
-
-      (nimbuses
-        [this]
-        (map #(maybe-deserialize (.get_data cluster-state (nimbus-path %1) false) NimbusSummary)
-          (.get_children cluster-state NIMBUSES-SUBTREE false)))
-
-      (add-nimbus-host!
-        [this nimbus-id nimbus-summary]
-        ;explicit delete for ephmeral node to ensure this session creates the entry.
-        (.delete_node cluster-state (nimbus-path nimbus-id))
-
-        (.add_listener cluster-state (reify ClusterStateListener
-                        (^void stateChanged[this ^ConnectionState newState]
-                          (log-message "Connection state listener invoked, zookeeper connection state has changed to " newState)
-                          (if (.equals newState ConnectionState/RECONNECTED)
-                            (do
-                              (log-message "Connection state has changed to reconnected so setting nimbuses entry one more time")
-                              (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))))))
-        
-        (.set_ephemeral_node cluster-state (nimbus-path nimbus-id) (Utils/serialize nimbus-summary) acls))
-
-      (setup-blobstore!
-        [this key nimbusInfo versionInfo]
-        (let [path (str (blobstore-path key) "/" (.toHostPortString nimbusInfo) "-" versionInfo)]
-          (log-message "setup-path: " path)
-          (.mkdirs cluster-state (blobstore-path key) acls)
-          ;we delete the node first to ensure the node gets created as part of this session only.
-          (.delete_node_blobstore cluster-state (str (blobstore-path key)) (.toHostPortString nimbusInfo))
-          (.set_ephemeral_node cluster-state path nil acls)))
-
-      (blobstore-info
-        [this blob-key]
-        (let [path (blobstore-path blob-key)]
-          (.sync_path cluster-state path)
-          (.get_children cluster-state path false)))
-
-      (active-storms
-        [this]
-        (.get_children cluster-state STORMS-SUBTREE false))
-
-      (active-keys
-        [this]
-        (.get_children cluster-state BLOBSTORE-SUBTREE false))
-
-      (heartbeat-storms
-        [this]
-        (.get_worker_hb_children cluster-state WORKERBEATS-SUBTREE false))
-
-      (error-topologies
-        [this]
-        (.get_children cluster-state ERRORS-SUBTREE false))
-
-      (get-worker-heartbeat
-        [this storm-id node port]
-        (let [worker-hb (.get_worker_hb cluster-state (workerbeat-path storm-id node port) false)]
-          (if worker-hb
-            (-> worker-hb
-              (maybe-deserialize ClusterWorkerHeartbeat)
-              clojurify-zk-worker-hb))))
-
-      (executor-beats
-        [this storm-id executor->node+port]
-        ;; 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
-        (let [node+port->executors (reverse-map executor->node+port)
-              all-heartbeats (for [[[node port] executors] node+port->executors]
-                               (->> (get-worker-heartbeat this storm-id node port)
-                                    (convert-executor-beats executors)
-                                    ))]
-          (apply merge all-heartbeats)))
-
-      (supervisors
-        [this callback]
-        (when callback
-          (reset! supervisors-callback callback))
-        (.get_children cluster-state SUPERVISORS-SUBTREE (not-nil? callback)))
-
-      (supervisor-info
-        [this supervisor-id]
-        (clojurify-supervisor-info (maybe-deserialize (.get_data cluster-state (supervisor-path supervisor-id) false) SupervisorInfo)))
-
-      (topology-log-config
-        [this storm-id cb]
-        (when cb
-          (swap! log-config-callback assoc storm-id cb))
-        (maybe-deserialize (.get_data cluster-state (log-config-path storm-id) (not-nil? cb)) LogConfig))
-
-      (set-topology-log-config!
-        [this storm-id log-config]
-        (.set_data cluster-state (log-config-path storm-id) (Utils/serialize log-config) acls))
-
-      (set-worker-profile-request
-        [this storm-id profile-request]
-        (let [request-type (.get_action profile-request)
-              host (.get_node (.get_nodeInfo profile-request))
-              port (first (.get_port (.get_nodeInfo profile-request)))]
-          (.set_data cluster-state
-                     (profiler-config-path storm-id host port request-type)
-                     (Utils/serialize profile-request)
-                     acls)))
-
-      (get-topology-profile-requests
-        [this storm-id thrift?]
-        (let [path (profiler-config-path storm-id)
-              requests (if (.node_exists cluster-state path false)
-                         (dofor [c (.get_children cluster-state path false)]
-                                (let [raw (.get_data cluster-state (str path "/" c) false)
-                                      request (maybe-deserialize raw ProfileRequest)]
-                                      (if thrift?
-                                        request
-                                        (clojurify-profile-request request)))))]
-          requests))
-
-      (delete-topology-profile-requests
-        [this storm-id profile-request]
-        (let [profile-request-inst (thriftify-profile-request profile-request)
-              action (:action profile-request)
-              host (:host profile-request)
-              port (:port profile-request)]
-          (.delete_node cluster-state
-           (profiler-config-path storm-id host port action))))
-          
-      (get-worker-profile-requests
-        [this storm-id node-info thrift?]
-        (let [host (:host node-info)
-              port (:port node-info)
-              profile-requests (get-topology-profile-requests this storm-id thrift?)]
-          (if thrift?
-            (filter #(and (= host (.get_node (.get_nodeInfo %))) (= port (first (.get_port (.get_nodeInfo  %)))))
-                    profile-requests)
-            (filter #(and (= host (:host %)) (= port (:port %)))
-                    profile-requests))))
-      
-      (worker-heartbeat!
-        [this storm-id node port info]
-        (let [thrift-worker-hb (thriftify-zk-worker-hb info)]
-          (if thrift-worker-hb
-            (.set_worker_hb cluster-state (workerbeat-path storm-id node port) (Utils/serialize thrift-worker-hb) acls))))
-
-      (remove-worker-heartbeat!
-        [this storm-id node port]
-        (.delete_worker_hb cluster-state (workerbeat-path storm-id node port)))
-
-      (setup-heartbeats!
-        [this storm-id]
-        (.mkdirs cluster-state (workerbeat-storm-root storm-id) acls))
-
-      (teardown-heartbeats!
-        [this storm-id]
-        (try-cause
-          (.delete_worker_hb cluster-state (workerbeat-storm-root storm-id))
-          (catch KeeperException e
-            (log-warn-error e "Could not teardown heartbeats for " storm-id))))
-
-      (worker-backpressure!
-        [this storm-id node port on?]
-        "if znode exists and to be not on?, delete; if exists and on?, do nothing;
-        if not exists and to be on?, create; if not exists and not on?, do nothing"
-        (let [path (backpressure-path storm-id node port)
-              existed (.node_exists cluster-state path false)]
-          (if existed
-            (if (not on?)
-              (.delete_node cluster-state path))   ;; delete the znode since the worker is not congested
-            (if on?
-              (.set_ephemeral_node cluster-state path nil acls))))) ;; create the znode since worker is congested
-    
-      (topology-backpressure
-        [this storm-id callback]
-        "if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not."
-        (when callback
-          (swap! backpressure-callback assoc storm-id callback))
-        (let [path (backpressure-storm-root storm-id)
-              children (.get_children cluster-state path (not-nil? callback))]
-              (> (count children) 0)))
-      
-      (setup-backpressure!
-        [this storm-id]
-        (.mkdirs cluster-state (backpressure-storm-root storm-id) acls))
-
-      (remove-worker-backpressure!
-        [this storm-id node port]
-        (.delete_node cluster-state (backpressure-path storm-id node port)))
-
-      (teardown-topology-errors!
-        [this storm-id]
-        (try-cause
-          (.delete_node cluster-state (error-storm-root storm-id))
-          (catch KeeperException e
-            (log-warn-error e "Could not teardown errors for " storm-id))))
-
-      (supervisor-heartbeat!
-        [this supervisor-id info]
-        (let [thrift-supervisor-info (thriftify-supervisor-info info)]
-          (.set_ephemeral_node cluster-state (supervisor-path supervisor-id) (Utils/serialize thrift-supervisor-info) acls)))
-
-      (activate-storm!
-        [this storm-id storm-base]
-        (let [thrift-storm-base (thriftify-storm-base storm-base)]
-          (.set_data cluster-state (storm-path storm-id) (Utils/serialize thrift-storm-base) acls)))
-
-      (update-storm!
-        [this storm-id new-elems]
-        (let [base (storm-base this storm-id nil)
-              executors (:component->executors base)
-              component->debug (:component->debug base)
-              new-elems (update new-elems :component->executors (partial merge executors))
-              new-elems (update new-elems :component->debug (partial merge-with merge component->debug))]
-          (.set_data cluster-state (storm-path storm-id)
-                    (-> base
-                        (merge new-elems)
-                        thriftify-storm-base
-                        Utils/serialize)
-                    acls)))
-
-      (storm-base
-        [this storm-id callback]
-        (when callback
-          (swap! storm-base-callback assoc storm-id callback))
-        (clojurify-storm-base (maybe-deserialize (.get_data cluster-state (storm-path storm-id) (not-nil? callback)) StormBase)))
-
-      (remove-storm-base!
-        [this storm-id]
-        (.delete_node cluster-state (storm-path storm-id)))
-
-      (set-assignment!
-        [this storm-id info]
-        (let [thrift-assignment (thriftify-assignment info)]
-          (.set_data cluster-state (assignment-path storm-id) (Utils/serialize thrift-assignment) acls)))
-
-      (remove-blobstore-key!
-        [this blob-key]
-        (log-debug "removing key" blob-key)
-        (.delete_node cluster-state (blobstore-path blob-key)))
-
-      (remove-key-version!
-        [this blob-key]
-        (.delete_node cluster-state (blobstore-max-key-sequence-number-path blob-key)))
-
-      (remove-storm!
-        [this storm-id]
-        (.delete_node cluster-state (assignment-path storm-id))
-        (.delete_node cluster-state (credentials-path storm-id))
-        (.delete_node cluster-state (log-config-path storm-id))
-        (.delete_node cluster-state (profiler-config-path storm-id))
-        (remove-storm-base! this storm-id))
-
-      (set-credentials!
-         [this storm-id creds topo-conf]
-         (let [topo-acls (mk-topo-only-acls topo-conf)
-               path (credentials-path storm-id)
-               thriftified-creds (thriftify-credentials creds)]
-           (.set_data cluster-state path (Utils/serialize thriftified-creds) topo-acls)))
-
-      (credentials
-        [this storm-id callback]
-        (when callback
-          (swap! credentials-callback assoc storm-id callback))
-        (clojurify-crdentials (maybe-deserialize (.get_data cluster-state (credentials-path storm-id) (not-nil? callback)) Credentials)))
-
-      (report-error
-         [this storm-id component-id node port error]
-         (let [path (error-path storm-id component-id)
-               last-error-path (last-error-path storm-id component-id)
-               data (thriftify-error {:time-secs (current-time-secs) :error (stringify-error error) :host node :port port})
-               _ (.mkdirs cluster-state path acls)
-               ser-data (Utils/serialize data)
-               _ (.mkdirs cluster-state path acls)
-               _ (.create_sequential cluster-state (str path "/e") ser-data acls)
-               _ (.set_data cluster-state last-error-path ser-data acls)
-               to-kill (->> (.get_children cluster-state path false)
-                            (sort-by parse-error-path)
-                            reverse
-                            (drop 10))]
-           (doseq [k to-kill]
-             (.delete_node cluster-state (str path "/" k)))))
-
-      (errors
-         [this storm-id component-id]
-         (let [path (error-path storm-id component-id)
-               errors (if (.node_exists cluster-state path false)
-                        (dofor [c (.get_children cluster-state path false)]
-                          (if-let [data (-> (.get_data cluster-state
-                                                      (str path "/" c)
-                                                      false)
-                                          (maybe-deserialize ErrorInfo)
-                                          clojurify-error)]
-                            (map->TaskError data)))
-                        ())]
-           (->> (filter not-nil? errors)
-                (sort-by (comp - :time-secs)))))
-
-      (last-error
-        [this storm-id component-id]
-        (let [path (last-error-path storm-id component-id)]
-          (if (.node_exists cluster-state path false)
-            (if-let [data (-> (.get_data cluster-state path false)
-                              (maybe-deserialize ErrorInfo)
-                              clojurify-error)]
-              (map->TaskError data)))))
-      
-      (disconnect
-         [this]
-        (.unregister cluster-state state-id)
-        (when solo?
-          (.close cluster-state))))))
-
-;; daemons have a single thread that will respond to events
-;; start with initialize event
-;; callbacks add events to the thread's queue
-
-;; keeps in memory cache of the state, only for what client subscribes to. Any subscription is automatically kept in sync, and when there are changes, client is notified.
-;; master gives orders through state, and client records status in state (ephemerally)
-
-;; master tells nodes what workers to launch
-
-;; master writes this. supervisors and workers subscribe to this to understand complete topology. each storm is a map from nodes to workers to tasks to ports whenever topology changes everyone will be notified
-;; master includes timestamp of each assignment so that appropriate time can be given to each worker to start up
-;; /assignments/{storm id}
-
-;; which tasks they talk to, etc. (immutable until shutdown)
-;; everyone reads this in full to understand structure
-;; /tasks/{storm id}/{task id} ; just contains bolt id
-
-;; supervisors send heartbeats here, master doesn't subscribe but checks asynchronously
-;; /supervisors/status/{ephemeral node ids}  ;; node metadata such as port ranges are kept here
-
-;; tasks send heartbeats here, master doesn't subscribe, just checks asynchronously
-;; /taskbeats/{storm id}/{ephemeral task id}
-
-;; contains data about whether it's started or not, tasks and workers subscribe to specific storm here to know when to shutdown
-;; master manipulates
-;; /storms/{storm id}
-
-;; Zookeeper flows:
-
-;; Master:
-;; job submit:
-;; 1. read which nodes are available
-;; 2. set up the worker/{storm}/{task} stuff (static)
-;; 3. set assignments
-;; 4. start storm - necessary in case master goes down, when goes back up can remember to take down the storm (2 states: on or off)
-
-;; Monitoring (or by checking when nodes go down or heartbeats aren't received):
-;; 1. read assignment
-;; 2. see which tasks/nodes are up
-;; 3. make new assignment to fix any problems
-;; 4. if a storm exists but is not taken down fully, ensure that storm takedown is launched (step by step remove tasks and finally remove assignments)
-
-;; masters only possible watches is on ephemeral nodes and tasks, and maybe not even
-
-;; Supervisor:
-;; 1. monitor /storms/* and assignments
-;; 2. local state about which workers are local
-;; 3. when storm is on, check that workers are running locally & start/kill if different than assignments
-;; 4. when storm is off, monitor tasks for workers - when they all die or don't hearbeat, kill the process and cleanup
-
-;; Worker:
-;; 1. On startup, start the tasks if the storm is on
-
-;; Task:
-;; 1. monitor assignments, reroute when assignments change
-;; 2. monitor storm (when storm turns off, error if assignments change) - take down tasks as master turns them off
-
-;; locally on supervisor: workers write pids locally on startup, supervisor deletes it on shutdown (associates pid with worker name)
-;; supervisor periodically checks to make sure processes are alive
-;; {rootdir}/workers/{storm id}/{worker id}   ;; contains pid inside
-
-;; all tasks in a worker share the same cluster state
-;; workers, supervisors, and tasks subscribes to storm to know when it's started or stopped
-;; on stopped, master removes records in order (tasks need to subscribe to themselves to see if they disappear)
-;; when a master removes a worker, the supervisor should kill it (and escalate to kill -9)
-;; on shutdown, tasks subscribe to tasks that send data to them to wait for them to die. when node disappears, they can die

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj b/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
deleted file mode 100644
index dcfa8d8..0000000
--- a/storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
+++ /dev/null
@@ -1,163 +0,0 @@
-;; 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.
-
-(ns org.apache.storm.cluster-state.zookeeper-state-factory
-  (:import [org.apache.curator.framework.state ConnectionStateListener]
-           [org.apache.storm.zookeeper Zookeeper])
-  (:import [org.apache.zookeeper KeeperException$NoNodeException CreateMode
-             Watcher$Event$EventType Watcher$Event$KeeperState]
-           [org.apache.storm.cluster ClusterState DaemonType])
-  (:use [org.apache.storm cluster config log util])
-  (:require [org.apache.storm [zookeeper :as zk]])
-  (:gen-class
-   :implements [org.apache.storm.cluster.ClusterStateFactory]))
-
-(defn -mkState [this conf auth-conf acls context]
-  (let [zk (zk/mk-client conf (conf STORM-ZOOKEEPER-SERVERS) (conf STORM-ZOOKEEPER-PORT) :auth-conf auth-conf)]
-    (Zookeeper/mkdirs zk (conf STORM-ZOOKEEPER-ROOT) acls)
-    (.close zk))
-  (let [callbacks (atom {})
-        active (atom true)
-        zk-writer (zk/mk-client conf
-                         (conf STORM-ZOOKEEPER-SERVERS)
-                         (conf STORM-ZOOKEEPER-PORT)
-                         :auth-conf auth-conf
-                         :root (conf STORM-ZOOKEEPER-ROOT)
-                         :watcher (fn [state type path]
-                                    (when @active
-                                      (when-not (= Watcher$Event$KeeperState/SyncConnected state)
-                                        (log-warn "Received event " state ":" type ":" path " with disconnected Writer Zookeeper."))
-                                      (when-not (= Watcher$Event$EventType/None type)
-                                        (doseq [callback (vals @callbacks)]
-                                          (callback type path))))))
-        is-nimbus? (= (.getDaemonType context) DaemonType/NIMBUS)
-        zk-reader (if is-nimbus?
-                    (zk/mk-client conf
-                         (conf STORM-ZOOKEEPER-SERVERS)
-                         (conf STORM-ZOOKEEPER-PORT)
-                         :auth-conf auth-conf
-                         :root (conf STORM-ZOOKEEPER-ROOT)
-                         :watcher (fn [state type path]
-                                    (when @active
-                                      (when-not (= Watcher$Event$KeeperState/SyncConnected state)
-                                        (log-warn "Received event " state ":" type ":" path " with disconnected Reader Zookeeper."))
-                                      (when-not (= Watcher$Event$EventType/None type)
-                                        (doseq [callback (vals @callbacks)]
-                                          (callback type path))))))
-                    zk-writer)]
-    (reify
-     ClusterState
-
-     (register
-       [this callback]
-       (let [id (uuid)]
-         (swap! callbacks assoc id callback)
-         id))
-
-     (unregister
-       [this id]
-       (swap! callbacks dissoc id))
-
-     (set-ephemeral-node
-       [this path data acls]
-       (Zookeeper/mkdirs zk-writer (parent-path path) acls)
-       (if (Zookeeper/exists zk-writer path false)
-         (try-cause
-           (Zookeeper/setData zk-writer path data) ; should verify that it's ephemeral
-           (catch KeeperException$NoNodeException e
-             (log-warn-error e "Ephemeral node disappeared between checking for existing and setting data")
-             (Zookeeper/createNode zk-writer path data CreateMode/EPHEMERAL acls)))
-         (Zookeeper/createNode zk-writer path data CreateMode/EPHEMERAL acls)))
-
-     (create-sequential
-       [this path data acls]
-       (Zookeeper/createNode zk-writer path data CreateMode/PERSISTENT_SEQUENTIAL acls))
-
-     (set-data
-       [this path data acls]
-       ;; note: this does not turn off any existing watches
-       (if (Zookeeper/exists zk-writer path false)
-         (Zookeeper/setData zk-writer path data)
-         (do
-           (Zookeeper/mkdirs zk-writer (parent-path path) acls)
-           (Zookeeper/createNode zk-writer path data CreateMode/PERSISTENT acls))))
-
-     (set-worker-hb
-       [this path data acls]
-       (.set_data this path data acls))
-
-     (delete-node
-       [this path]
-       (Zookeeper/deleteNode zk-writer path))
-
-     (delete-worker-hb
-       [this path]
-       (.delete_node this path))
-
-     (get-data
-       [this path watch?]
-       (Zookeeper/getData zk-reader path watch?))
-
-     (get-data-with-version
-       [this path watch?]
-       (Zookeeper/getDataWithVersion zk-reader path watch?))
-
-     (get-version
-       [this path watch?]
-       (Zookeeper/getVersion zk-reader path watch?))
-
-     (get-worker-hb
-       [this path watch?]
-       (.get_data this path watch?))
-
-     (get-children
-       [this path watch?]
-       (Zookeeper/getChildren zk-reader path watch?))
-
-     (get-worker-hb-children
-       [this path watch?]
-       (.get_children this path watch?))
-
-     (mkdirs
-       [this path acls]
-       (Zookeeper/mkdirs zk-writer path acls))
-
-     (node-exists
-       [this path watch?]
-       (Zookeeper/existsNode zk-reader path watch?))
-
-     (add-listener
-       [this listener]
-       (let [curator-listener (reify ConnectionStateListener
-                                (stateChanged
-                                  [this client newState]
-                                  (.stateChanged listener client newState)))]
-         (Zookeeper/addListener zk-reader curator-listener)))
-
-     (sync-path
-       [this path]
-       (Zookeeper/syncPath zk-writer path))
-
-      (delete-node-blobstore
-        [this path nimbus-host-port-info]
-        (Zookeeper/deleteNodeBlobstore zk-writer path nimbus-host-port-info))
-
-     (close
-       [this]
-       (reset! active false)
-       (.close zk-writer)
-       (if is-nimbus?
-         (.close zk-reader))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index ef9ecbb..7be526d 100644
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@ -14,7 +14,7 @@
 ;; See the License for the specific language governing permissions and
 ;; limitations under the License.
 (ns org.apache.storm.command.dev-zookeeper
-  (:use [org.apache.storm zookeeper util config])
+  (:use [org.apache.storm util config])
   (:import [org.apache.storm.utils ConfigUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index be8d030..954042f 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -18,16 +18,16 @@
              [config :refer :all]
              [log :refer :all]
              [util :refer :all]
-             [cluster :refer :all]
              [converter :refer :all]]
             [clojure.string :as string])
   (:import [org.apache.storm.generated ClusterWorkerHeartbeat]
-           [org.apache.storm.utils Utils ConfigUtils])
+           [org.apache.storm.utils Utils ConfigUtils]
+           [org.apache.storm.cluster DistributedClusterState ClusterStateContext])
   (:gen-class))
 
 (defn -main [command path & args]
   (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
-        cluster (mk-distributed-cluster-state conf :auth-conf conf)]
+        cluster (DistributedClusterState. conf conf nil (ClusterStateContext.))]
     (println "Command: [" command "]")
     (condp = command
       "list"

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 8a5eb21..3978d2f 100644
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@ -16,7 +16,7 @@
 (ns org.apache.storm.command.shell-submission
   (:import [org.apache.storm StormSubmitter]
            [org.apache.storm.zookeeper Zookeeper])
-  (:use [org.apache.storm thrift util config log zookeeper])
+  (:use [org.apache.storm thrift util config log])
   (:require [clojure.string :as str])
   (:import [org.apache.storm.utils ConfigUtils])
   (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index bb2dc87..d169301 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -181,9 +181,9 @@
 (defn thriftify-storm-base [storm-base]
   (doto (StormBase.)
     (.set_name (:storm-name storm-base))
-    (.set_launch_time_secs (int (:launch-time-secs storm-base)))
+    (.set_launch_time_secs (if (:launch-time-secs storm-base) (int (:launch-time-secs storm-base)) 0))
     (.set_status (convert-to-status-from-symbol (:status storm-base)))
-    (.set_num_workers (int (:num-workers storm-base)))
+    (.set_num_workers (if (:num-workers storm-base) (int (:num-workers storm-base)) 0))
     (.set_component_executors (map-val int (:component->executors storm-base)))
     (.set_owner (:owner storm-base))
     (.set_topology_action_options (thriftify-topology-action-options storm-base))
@@ -234,16 +234,6 @@
       (.set_executor_stats (thriftify-stats (filter second (:executor-stats worker-hb))))
       (.set_time_secs (:time-secs worker-hb)))))
 
-(defn clojurify-error [^ErrorInfo error]
-  (if error
-    {
-      :error (.get_error error)
-      :time-secs (.get_error_time_secs error)
-      :host (.get_host error)
-      :port (.get_port error)
-      }
-    ))
-
 (defn thriftify-error [error]
   (doto (ErrorInfo. (:error error) (:time-secs error))
     (.set_host (:host error))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj
index 6c184fd..c9534f4 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@ -13,14 +13,16 @@
 ;; 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.
+;TopologyActionOptions TopologyStatus StormBase RebalanceOptions KillOptions
 (ns org.apache.storm.daemon.common
   (:use [org.apache.storm log config util])
-  (:import [org.apache.storm.generated StormTopology
+  (:import [org.apache.storm.generated StormTopology NodeInfo
             InvalidTopologyException GlobalStreamId]
            [org.apache.storm.utils ThriftTopologyUtils])
   (:import [org.apache.storm.utils Utils ConfigUtils])
   (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [org.apache.storm Constants])
+  (:import [org.apache.storm.cluster StormZkClusterState])
   (:import [org.apache.storm.metric SystemBolt])
   (:import [org.apache.storm.metric EventLoggerBolt])
   (:import [org.apache.storm.security.auth IAuthorizer]) 
@@ -72,18 +74,19 @@
 (defn new-executor-stats []
   (ExecutorStats. 0 0 0 0 0))
 
+
 (defn get-storm-id [storm-cluster-state storm-name]
-  (let [active-storms (.active-storms storm-cluster-state)]
+  (let [active-storms (.activeStorms storm-cluster-state)]
     (find-first
-      #(= storm-name (:storm-name (.storm-base storm-cluster-state % nil)))
+      #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
       active-storms)
     ))
 
 (defn topology-bases [storm-cluster-state]
-  (let [active-topologies (.active-storms storm-cluster-state)]
+  (let [active-topologies (.activeStorms storm-cluster-state)]
     (into {} 
           (dofor [id active-topologies]
-                 [id (.storm-base storm-cluster-state id nil)]
+                 [id  (.stormBase storm-cluster-state id nil)]
                  ))
     ))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 82d56a9..e50e150 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -34,11 +34,10 @@
   (:import [org.apache.storm.daemon Shutdownable])
   (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
   (:import [org.apache.storm Config Constants])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster])
   (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
-  (:require [org.apache.storm [thrift :as thrift]
-             [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
+  (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
   (:require [org.apache.storm.daemon [task :as task]])
   (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
   (:require [clojure.set :as set]))
@@ -207,7 +206,7 @@
       (swap! interval-errors inc)
 
       (when (<= @interval-errors max-per-interval)
-        (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
+        (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
                               (hostname storm-conf)
                               (.getThisWorkerPort (:worker-context executor)) error)
         ))))
@@ -252,9 +251,8 @@
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
-     :storm-cluster-state (cluster/mk-storm-cluster-state (:cluster-state worker) 
-                                                          :acls (Utils/getWorkerACL storm-conf)
-                                                          :context (ClusterStateContext. DaemonType/WORKER))
+     :storm-cluster-state (StormZkClusterState. (:cluster-state worker) (Utils/getWorkerACL storm-conf)
+                            (ClusterStateContext. DaemonType/WORKER))
      :type executor-type
      ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)
      :stats (mk-executor-stats <> (ConfigUtils/samplingRate storm-conf))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index de5a14e..9b00df3 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -40,7 +40,7 @@
   (:import [org.apache.storm.nimbus NimbusInfo])
   (:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
             BufferFileInputStream BufferInputStream])
-  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
+  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
             ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
             KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
             ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@ -48,10 +48,9 @@
             ProfileRequest ProfileAction NodeInfo])
   (:import [org.apache.storm.daemon Shutdownable])
   (:import [org.apache.storm.validation ConfigValidation])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
-  (:use [org.apache.storm util config log timer zookeeper local-state])
-  (:require [org.apache.storm [cluster :as cluster]
-                            [converter :as converter]
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState])
+  (:use [org.apache.storm util config log timer local-state converter])
+  (:require [org.apache.storm [converter :as converter]
                             [stats :as stats]])
   (:require [clojure.set :as set])
   (:import [org.apache.storm.daemon.common StormBase Assignment])
@@ -174,11 +173,11 @@
      :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
      :impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
      :submitted-count (atom 0)
-     :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
+     :storm-cluster-state (StormZkClusterState. conf  (when
                                                                        (Utils/isZkAuthenticationConfiguredStormServer
                                                                          conf)
                                                                        NIMBUS-ZK-ACLS)
-                                                          :context (ClusterStateContext. DaemonType/NIMBUS))
+                                                          (ClusterStateContext. DaemonType/NIMBUS))
      :submit-lock (Object.)
      :cred-update-lock (Object.)
      :log-update-lock (Object.)
@@ -275,11 +274,11 @@
 
 (defn do-rebalance [nimbus storm-id status storm-base]
   (let [rebalance-options (:topology-action-options storm-base)]
-    (.update-storm! (:storm-cluster-state nimbus)
+    (.updateStorm (:storm-cluster-state nimbus)
       storm-id
-        (-> {:topology-action-options nil}
+      (thriftify-storm-base (-> {:topology-action-options nil}
           (assoc-non-nil :component->executors (:component->executors rebalance-options))
-          (assoc-non-nil :num-workers (:num-workers rebalance-options)))))
+          (assoc-non-nil :num-workers (:num-workers rebalance-options))))))
   (mk-assignments nimbus :scratch-topology-id storm-id))
 
 (defn state-transitions [nimbus storm-id status storm-base]
@@ -303,12 +302,12 @@
             :kill (kill-transition nimbus storm-id)
             :remove (fn []
                       (log-message "Killing topology: " storm-id)
-                      (.remove-storm! (:storm-cluster-state nimbus)
+                      (.removeStorm (:storm-cluster-state nimbus)
                                       storm-id)
                       (when (instance? LocalFsBlobStore (:blob-store nimbus))
                         (doseq [blob-key (get-key-list-from-id (:conf nimbus) storm-id)]
-                          (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
-                          (.remove-key-version! (:storm-cluster-state nimbus) blob-key)))
+                          (.removeBlobstoreKey (:storm-cluster-state nimbus) blob-key)
+                          (.removeKeyVersion (:storm-cluster-state nimbus) blob-key)))
                       nil)
             }
    :rebalancing {:startup (fn [] (delay-event nimbus
@@ -332,7 +331,7 @@
     (locking (:submit-lock nimbus)
        (let [system-events #{:startup}
              [event & event-args] (if (keyword? event) [event] event)
-             storm-base (-> nimbus :storm-cluster-state  (.storm-base storm-id nil))
+             storm-base (clojurify-storm-base (-> nimbus :storm-cluster-state  (.stormBase storm-id nil)))
              status (:status storm-base)]
          ;; handles the case where event was scheduled but topology has been removed
          (if-not status
@@ -362,7 +361,7 @@
                                       storm-base-updates)]
 
              (when storm-base-updates
-               (.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
+               (.updateStorm (:storm-cluster-state nimbus) storm-id (thriftify-storm-base storm-base-updates))))))
        )))
 
 (defn transition-name! [nimbus storm-name event & args]
@@ -411,7 +410,7 @@
     (defaulted
       (apply merge-with set/union
              (for [a assignments
-                   [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
+                   [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
                {node #{port}}
                ))
       {})
@@ -424,7 +423,7 @@
        (into {}
              (mapcat
               (fn [id]
-                (if-let [info (.supervisor-info storm-cluster-state id)]
+                (if-let [info (clojurify-supervisor-info (.supervisorInfo storm-cluster-state id))]
                   [[id info]]
                   ))
               supervisor-ids))
@@ -469,13 +468,13 @@
     (when tmp-jar-location ;;in local mode there is no jar
       (.createBlob blob-store jar-key (FileInputStream. tmp-jar-location) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
       (if (instance? LocalFsBlobStore blob-store)
-        (.setup-blobstore! storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
+        (.setupBlobstore storm-cluster-state jar-key nimbus-host-port-info (get-version-for-key jar-key nimbus-host-port-info conf))))
     (.createBlob blob-store conf-key (Utils/toCompressedJsonConf storm-conf) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
     (if (instance? LocalFsBlobStore blob-store)
-      (.setup-blobstore! storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
+      (.setupBlobstore storm-cluster-state conf-key nimbus-host-port-info (get-version-for-key conf-key nimbus-host-port-info conf)))
     (.createBlob blob-store code-key (Utils/serialize topology) (SettableBlobMeta. BlobStoreAclHandler/DEFAULT) subject)
     (if (instance? LocalFsBlobStore blob-store)
-      (.setup-blobstore! storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
+      (.setupBlobstore storm-cluster-state code-key nimbus-host-port-info (get-version-for-key code-key nimbus-host-port-info conf)))))
 
 (defn- read-storm-topology [storm-id blob-store]
   (Utils/deserialize
@@ -540,7 +539,7 @@
 (defn read-topology-details [nimbus storm-id]
   (let [blob-store (:blob-store nimbus)
         storm-base (or
-                     (.storm-base (:storm-cluster-state nimbus) storm-id nil)
+                     (clojurify-storm-base (.stormBase (:storm-cluster-state nimbus) storm-id nil))
                      (throw (NotAliveException. storm-id)))
         topology-conf (read-storm-conf-as-nimbus storm-id blob-store)
         topology (read-storm-topology-as-nimbus storm-id blob-store)
@@ -587,7 +586,12 @@
 (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
   (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
   (let [storm-cluster-state (:storm-cluster-state nimbus)
-        executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment))
+        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))]
+                         (->> (clojurify-structure executor-stats-java-map)
+                           (map (fn [^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat]
+                                  {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)}))
+                         (into {})))
+
         cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
                                       executor-beats
                                       all-executors
@@ -637,7 +641,7 @@
 (defn- compute-executors [nimbus storm-id]
   (let [conf (:conf nimbus)
         blob-store (:blob-store nimbus)
-        storm-base (.storm-base (:storm-cluster-state nimbus) storm-id nil)
+        storm-base (clojurify-storm-base (.stormBase (:storm-cluster-state nimbus) storm-id nil))
         component->executors (:component->executors storm-base)
         storm-conf (read-storm-conf-as-nimbus storm-id blob-store)
         topology (read-storm-topology-as-nimbus storm-id blob-store)
@@ -897,7 +901,7 @@
         storm-cluster-state (:storm-cluster-state nimbus)
         ^INimbus inimbus (:inimbus nimbus)
         ;; read all the topologies
-        topology-ids (.active-storms storm-cluster-state)
+        topology-ids (.activeStorms storm-cluster-state)
         topologies (into {} (for [tid topology-ids]
                               {tid (read-topology-details nimbus tid)}))
         topologies (Topologies. topologies)
@@ -908,7 +912,7 @@
                                         ;; we exclude its assignment, meaning that all the slots occupied by its assignment
                                         ;; will be treated as free slot in the scheduler code.
                                         (when (or (nil? scratch-topology-id) (not= tid scratch-topology-id))
-                                          {tid (.assignment-info storm-cluster-state tid nil)})))
+                                          {tid (clojurify-assignment (.assignmentInfo storm-cluster-state tid nil))})))
         ;; make the new assignments for topologies
         new-scheduler-assignments (compute-new-scheduler-assignments
                                        nimbus
@@ -957,7 +961,7 @@
         (log-debug "Assignment for " topology-id " hasn't changed")
         (do
           (log-message "Setting new assignment for topology id " topology-id ": " (pr-str assignment))
-          (.set-assignment! storm-cluster-state topology-id assignment)
+          (.setAssignment storm-cluster-state topology-id (thriftify-assignment assignment))
           )))
     (->> new-assignments
           (map (fn [[topology-id assignment]]
@@ -984,9 +988,9 @@
         topology (system-topology! storm-conf (read-storm-topology storm-id blob-store))
         num-executors (->> (all-components topology) (map-val num-start-executors))]
     (log-message "Activating " storm-name ": " storm-id)
-    (.activate-storm! storm-cluster-state
+    (.activateStorm storm-cluster-state
                       storm-id
-                      (StormBase. storm-name
+      (thriftify-storm-base (StormBase. storm-name
                                   (current-time-secs)
                                   {:type topology-initial-status}
                                   (storm-conf TOPOLOGY-WORKERS)
@@ -994,7 +998,7 @@
                                   (storm-conf TOPOLOGY-SUBMITTER-USER)
                                   nil
                                   nil
-                                  {}))
+                                  {})))
     (notify-topology-action-listener nimbus storm-name "activate")))
 
 ;; Master:
@@ -1046,10 +1050,10 @@
     (set (.filterAndListKeys blob-store to-id))))
 
 (defn cleanup-storm-ids [conf storm-cluster-state blob-store]
-  (let [heartbeat-ids (set (.heartbeat-storms storm-cluster-state))
-        error-ids (set (.error-topologies storm-cluster-state))
+  (let [heartbeat-ids (set (.heartbeatStorms storm-cluster-state))
+        error-ids (set (.errorTopologies storm-cluster-state))
         code-ids (code-ids blob-store)
-        assigned-ids (set (.active-storms storm-cluster-state))]
+        assigned-ids (set (.activeStorms storm-cluster-state))]
     (set/difference (set/union heartbeat-ids error-ids code-ids) assigned-ids)
     ))
 
@@ -1113,7 +1117,7 @@
   (try
     (.deleteBlob blob-store key nimbus-subject)
     (if (instance? LocalFsBlobStore blob-store)
-      (.remove-blobstore-key! storm-cluster-state key))
+      (.removeBlobstoreKey storm-cluster-state key))
     (catch Exception e
       (log-message "Exception" e))))
 
@@ -1133,8 +1137,8 @@
         (when-not (empty? to-cleanup-ids)
           (doseq [id to-cleanup-ids]
             (log-message "Cleaning up " id)
-            (.teardown-heartbeats! storm-cluster-state id)
-            (.teardown-topology-errors! storm-cluster-state id)
+            (.teardownHeartbeats storm-cluster-state id)
+            (.teardownTopologyErrors storm-cluster-state id)
             (rmr (ConfigUtils/masterStormDistRoot conf id))
             (blob-rm-topology-keys id blob-store storm-cluster-state)
             (swap! (:heartbeats-cache nimbus) dissoc id)))))
@@ -1169,21 +1173,21 @@
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         blob-store (:blob-store nimbus)
         code-ids (set (code-ids blob-store))
-        active-topologies (set (.active-storms storm-cluster-state))
+        active-topologies (set (.activeStorms storm-cluster-state))
         corrupt-topologies (set/difference active-topologies code-ids)]
     (doseq [corrupt corrupt-topologies]
       (log-message "Corrupt topology " corrupt " has state on zookeeper but doesn't have a local dir on Nimbus. Cleaning up...")
-      (.remove-storm! storm-cluster-state corrupt)
+      (.removeStorm storm-cluster-state corrupt)
       (if (instance? LocalFsBlobStore blob-store)
         (doseq [blob-key (get-key-list-from-id (:conf nimbus) corrupt)]
-          (.remove-blobstore-key! storm-cluster-state blob-key))))))
+          (.removeBlobstoreKey storm-cluster-state blob-key))))))
 
 (defn setup-blobstore [nimbus]
   "Sets up blobstore state for all current keys."
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         blob-store (:blob-store nimbus)
         local-set-of-keys (set (get-key-seq-from-blob-store blob-store))
-        all-keys (set (.active-keys storm-cluster-state))
+        all-keys (set (.activeKeys storm-cluster-state))
         locally-available-active-keys (set/intersection local-set-of-keys all-keys)
         keys-to-delete (set/difference local-set-of-keys all-keys)
         conf (:conf nimbus)
@@ -1193,10 +1197,10 @@
       (.deleteBlob blob-store key nimbus-subject))
     (log-debug "Creating list of key entries for blobstore inside zookeeper" all-keys "local" locally-available-active-keys)
     (doseq [key locally-available-active-keys]
-      (.setup-blobstore! storm-cluster-state key (:nimbus-host-port-info nimbus) (get-version-for-key key nimbus-host-port-info conf)))))
+      (.setupBlobstore storm-cluster-state key (:nimbus-host-port-info nimbus) (get-version-for-key key nimbus-host-port-info conf)))))
 
 (defn- get-errors [storm-cluster-state storm-id component-id]
-  (->> (.errors storm-cluster-state storm-id component-id)
+  (->> (apply clojurify-error (.errors storm-cluster-state storm-id component-id))
        (map #(doto (ErrorInfo. (:error %) (:time-secs %))
                    (.set_host (:host %))
                    (.set_port (:port %))))))
@@ -1293,11 +1297,11 @@
           blob-store (:blob-store nimbus)
           renewers (:cred-renewers nimbus)
           update-lock (:cred-update-lock nimbus)
-          assigned-ids (set (.active-storms storm-cluster-state))]
+          assigned-ids (set (.activeStorms storm-cluster-state))]
       (when-not (empty? assigned-ids)
         (doseq [id assigned-ids]
           (locking update-lock
-            (let [orig-creds (.credentials storm-cluster-state id nil)
+            (let [orig-creds (clojurify-crdentials (.credentials storm-cluster-state id nil))
                   topology-conf (try-read-storm-conf (:conf nimbus) id blob-store)]
               (if orig-creds
                 (let [new-creds (HashMap. orig-creds)]
@@ -1305,7 +1309,7 @@
                     (log-message "Renewing Creds For " id " with " renewer)
                     (.renew renewer new-creds (Collections/unmodifiableMap topology-conf)))
                   (when-not (= orig-creds new-creds)
-                    (.set-credentials! storm-cluster-state id new-creds topology-conf)
+                    (.setCredentials storm-cluster-state id (thriftify-credentials new-creds) topology-conf)
                     ))))))))
     (log-message "not a leader skipping , credential renweal.")))
 
@@ -1370,11 +1374,11 @@
                                           operation)
                   topology (try-read-storm-topology storm-id blob-store)
                   task->component (storm-task-info topology topology-conf)
-                  base (.storm-base storm-cluster-state storm-id nil)
+                  base (clojurify-storm-base (.stormBase storm-cluster-state storm-id nil))
                   launch-time-secs (if base (:launch-time-secs base)
                                      (throw
                                        (NotAliveException. (str storm-id))))
-                  assignment (.assignment-info storm-cluster-state storm-id nil)
+                  assignment (clojurify-assignment (.assignmentInfo storm-cluster-state storm-id nil))
                   beats (map-val :heartbeat (get @(:heartbeats-cache nimbus)
                                                  storm-id))
                   all-components (set (vals task->component))]
@@ -1388,16 +1392,16 @@
                :task->component task->component
                :base base}))
         get-last-error (fn [storm-cluster-state storm-id component-id]
-                         (if-let [e (.last-error storm-cluster-state
+                         (if-let [e (clojurify-error  (.lastError storm-cluster-state
                                                  storm-id
-                                                 component-id)]
+                                                 component-id))]
                            (doto (ErrorInfo. (:error e) (:time-secs e))
                              (.set_host (:host e))
                              (.set_port (:port e)))))]
     (.prepare ^org.apache.storm.nimbus.ITopologyValidator (:validator nimbus) conf)
 
     ;add to nimbuses
-    (.add-nimbus-host! (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus))
+    (.addNimbusHost (:storm-cluster-state nimbus) (.toHostPortString (:nimbus-host-port-info nimbus))
       (NimbusSummary.
         (.getHost (:nimbus-host-port-info nimbus))
         (.getPort (:nimbus-host-port-info nimbus))
@@ -1413,7 +1417,7 @@
       (setup-blobstore nimbus))
 
     (when (is-leader nimbus :throw-exception false)
-      (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
+      (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))]
         (transition! nimbus storm-id :startup)))
     (schedule-recurring (:timer nimbus)
                         0
@@ -1520,12 +1524,12 @@
             (locking (:submit-lock nimbus)
               (check-storm-active! nimbus storm-name false)
               ;;cred-update-lock is not needed here because creds are being added for the first time.
-              (.set-credentials! storm-cluster-state storm-id credentials storm-conf)
+              (.setCredentials storm-cluster-state storm-id (thriftify-credentials credentials) storm-conf)
               (log-message "uploadedJar " uploadedJarLocation)
               (setup-storm-code nimbus conf storm-id uploadedJarLocation total-storm-conf topology)
               (wait-for-desired-code-replication nimbus total-storm-conf storm-id)
-              (.setup-heartbeats! storm-cluster-state storm-id)
-              (.setup-backpressure! storm-cluster-state storm-id)
+              (.setupHeatbeats storm-cluster-state storm-id)
+              (.setupBackpressure storm-cluster-state storm-id)
               (notify-topology-action-listener nimbus storm-name "submitTopology")
               (let [thrift-status->kw-status {TopologyInitialStatus/INACTIVE :inactive
                                               TopologyInitialStatus/ACTIVE :active}]
@@ -1613,7 +1617,7 @@
           (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
             (if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
           (locking (:submit-lock nimbus)
-            (.update-storm! storm-cluster-state storm-id storm-base-updates))))
+            (.updateStorm storm-cluster-state (thriftify-storm-base storm-id storm-base-updates)))))
 
       (^void setWorkerProfiler
         [this ^String id ^ProfileRequest profileRequest]
@@ -1622,7 +1626,7 @@
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "setWorkerProfiler")
               storm-cluster-state (:storm-cluster-state nimbus)]
-          (.set-worker-profile-request storm-cluster-state id profileRequest)))
+          (.setWorkerProfileRequest storm-cluster-state id profileRequest)))
 
       (^List getComponentPendingProfileActions
         [this ^String id ^String component_id ^ProfileAction action]
@@ -1635,7 +1639,7 @@
                                              [(node->host node) port])
                                     executor->node+port)
               nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
-              all-pending-actions-for-topology (.get-topology-profile-requests storm-cluster-state id true)
+              all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id true))
               latest-profile-actions (remove nil? (map (fn [nodeInfo]
                                                          (->> all-pending-actions-for-topology
                                                               (filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))
@@ -1653,7 +1657,7 @@
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "setLogConfig")
               storm-cluster-state (:storm-cluster-state nimbus)
-              merged-log-config (or (.topology-log-config storm-cluster-state id nil) (LogConfig.))
+              merged-log-config (or (.topologyLogConfig storm-cluster-state id nil) (LogConfig.))
               named-loggers (.get_named_logger_level merged-log-config)]
             (doseq [[_ level] named-loggers]
               (.set_action level LogLevelAction/UNCHANGED))
@@ -1671,7 +1675,7 @@
                                (.containsKey named-loggers logger-name))
                         (.remove named-loggers logger-name))))))
             (log-message "Setting log config for " storm-name ":" merged-log-config)
-            (.set-topology-log-config! storm-cluster-state id merged-log-config)))
+            (.setTopologyLogConfig storm-cluster-state id merged-log-config)))
 
       (uploadNewCredentials [this storm-name credentials]
         (mark! nimbus:num-uploadNewCredentials-calls)
@@ -1680,7 +1684,7 @@
               topology-conf (try-read-storm-conf conf storm-id blob-store)
               creds (when credentials (.get_creds credentials))]
           (check-authorization! nimbus storm-name topology-conf "uploadNewCredentials")
-          (locking (:cred-update-lock nimbus) (.set-credentials! storm-cluster-state storm-id creds topology-conf))))
+          (locking (:cred-update-lock nimbus) (.setCredentials storm-cluster-state storm-id (thriftify-credentials creds) topology-conf))))
 
       (beginFileUpload [this]
         (mark! nimbus:num-beginFileUpload-calls)
@@ -1755,7 +1759,7 @@
               storm-name (topology-conf TOPOLOGY-NAME)
               _ (check-authorization! nimbus storm-name topology-conf "getLogConfig")
              storm-cluster-state (:storm-cluster-state nimbus)
-             log-config (.topology-log-config storm-cluster-state id nil)]
+             log-config (.topologyLogConfig storm-cluster-state id nil)]
            (if log-config log-config (LogConfig.))))
 
       (^String getTopologyConf [this ^String id]
@@ -1800,7 +1804,8 @@
                                        (when-let [version (:version info)] (.set_version sup-sum version))
                                        sup-sum))
               nimbus-uptime ((:uptime nimbus))
-              bases (topology-bases storm-cluster-state)
+              javabases (topology-bases storm-cluster-state)
+              bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
               nimbuses (.nimbuses storm-cluster-state)
 
               ;;update the isLeader field for each nimbus summary
@@ -1812,7 +1817,7 @@
                     (.set_isLeader nimbus-summary (and (= leader-host (.get_host nimbus-summary)) (= leader-port (.get_port nimbus-summary))))))
 
               topology-summaries (dofor [[id base] bases :when base]
-                                   (let [assignment (.assignment-info storm-cluster-state id nil)
+                                   (let [assignment (clojurify-assignment (.assignmentInfo storm-cluster-state id nil))
                                          topo-summ (TopologySummary. id
                                                      (:storm-name base)
                                                      (->> (:executor->node+port assignment)
@@ -1939,7 +1944,7 @@
               nimbus-host-port-info (:nimbus-host-port-info nimbus)
               conf (:conf nimbus)]
           (if (instance? LocalFsBlobStore blob-store)
-              (.setup-blobstore! storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
+              (.setupBlobstore storm-cluster-state blob-key nimbus-host-port-info (get-version-for-key blob-key nimbus-host-port-info conf)))
           (log-debug "Created state in zookeeper" storm-cluster-state blob-store nimbus-host-port-info)))
 
       (^void uploadBlobChunk [this ^String session ^ByteBuffer blob-chunk]
@@ -2019,8 +2024,8 @@
                            (.subject))]
           (.deleteBlob (:blob-store nimbus) blob-key subject)
           (when (instance? LocalFsBlobStore blob-store)
-            (.remove-blobstore-key! (:storm-cluster-state nimbus) blob-key)
-            (.remove-key-version! (:storm-cluster-state nimbus) blob-key))
+            (.removeBlobstoreKey (:storm-cluster-state nimbus) blob-key)
+            (.removeKeyVersion (:storm-cluster-state nimbus) blob-key))
           (log-message "Deleted blob for key " blob-key)))
 
       (^ListBlobsResult listBlobs [this ^String session]
@@ -2157,7 +2162,8 @@
 
       (^TopologyHistoryInfo getTopologyHistory [this ^String user]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
-              bases (topology-bases storm-cluster-state)
+              javabases (topology-bases storm-cluster-state)
+              bases (into {} (dofor [[id  base] javabases][id (clojurify-storm-base base)]))
               assigned-topology-ids (.assignments storm-cluster-state nil)
               user-group-match-fn (fn [topo-id user conf]
                                     (let [topology-conf (try-read-storm-conf conf topo-id (:blob-store nimbus))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 337a1b4..079b221 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -19,11 +19,11 @@
            [org.apache.storm.utils LocalState Time Utils ConfigUtils]
            [org.apache.storm.daemon Shutdownable]
            [org.apache.storm Constants]
-           [org.apache.storm.cluster ClusterStateContext DaemonType]
+           [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster]
            [java.net JarURLConnection]
            [java.net URI]
            [org.apache.commons.io FileUtils])
-  (:use [org.apache.storm config util log timer local-state])
+  (:use [org.apache.storm config util log timer local-state converter])
   (:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
   (:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
   (:import [java.nio.file Files StandardCopyOption])
@@ -33,7 +33,7 @@
   (:use [org.apache.storm.daemon common])
   (:require [org.apache.storm.command [healthcheck :as healthcheck]])
   (:require [org.apache.storm.daemon [worker :as worker]]
-            [org.apache.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
+            [org.apache.storm [process-simulator :as psim] [event :as event]]
             [clojure.set :as set])
   (:import [org.apache.thrift.transport TTransportException])
   (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])
@@ -63,21 +63,22 @@
           (->>
            (dofor [sid storm-ids]
                   (let [recorded-version (:version (get assignment-versions sid))]
-                    (if-let [assignment-version (.assignment-version storm-cluster-state sid callback)]
+                    (if-let [assignment-version (.assignmentVersion storm-cluster-state sid callback)]
                       (if (= assignment-version recorded-version)
                         {sid (get assignment-versions sid)}
-                        {sid (.assignment-info-with-version storm-cluster-state sid callback)})
+                        {sid (.assignmentInfoWithVersion storm-cluster-state sid callback)})
                       {sid nil})))
            (apply merge)
            (filter-val not-nil?))
           new-profiler-actions
           (->>
             (dofor [sid (distinct storm-ids)]
-                   (if-let [topo-profile-actions (.get-topology-profile-requests storm-cluster-state sid false)]
+
+                   (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
                       {sid topo-profile-actions}))
            (apply merge))]
-         
-      {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
+
+      {:assignments (into {} (for [[k v] new-assignments] [k (clojurify-assignment (:data v))]))
        :profiler-actions new-profiler-actions
        :versions new-assignments})))
 
@@ -316,11 +317,9 @@
    :uptime (uptime-computer)
    :version STORM-VERSION
    :worker-thread-pids-atom (atom {})
-   :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
-                                                                     (Utils/isZkAuthenticationConfiguredStormServer
-                                                                       conf)
-                                                                     SUPERVISOR-ZK-ACLS)
-                                                        :context (ClusterStateContext. DaemonType/SUPERVISOR))
+   :storm-cluster-state (Cluster/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+                                                     SUPERVISOR-ZK-ACLS)
+                                                        (ClusterStateContext. DaemonType/SUPERVISOR))
    :local-state (ConfigUtils/supervisorState conf)
    :supervisor-id (.getSupervisorId isupervisor)
    :assignment-id (.getAssignmentId isupervisor)
@@ -675,7 +674,7 @@
 
 (defn- delete-topology-profiler-action [storm-cluster-state storm-id profile-action]
   (log-message "Deleting profiler action.." profile-action)
-  (.delete-topology-profile-requests storm-cluster-state storm-id profile-action))
+  (.deleteTopologyProfileRequests storm-cluster-state storm-id (thriftify-profile-request profile-action)))
 
 (defnk launch-profiler-action-for-worker
   "Launch profiler action for a worker"
@@ -743,7 +742,7 @@
                       action-on-exit (fn [exit-code]
                                        (log-message log-prefix " profile-action exited for code: " exit-code)
                                        (if (and (= exit-code 0) stop?)
-                                         (delete-topology-profiler-action storm-cluster-state storm-id pro-action)))
+                                         (delete-topology-profiler-action storm-cluster-state storm-id (thriftify-profile-request pro-action))))
                       command (->> command (map str) (filter (complement empty?)))]
 
                   (try
@@ -776,10 +775,10 @@
         synchronize-blobs-fn (update-blobs-for-all-topologies-fn supervisor)
         downloaded-storm-ids (set (read-downloaded-storm-ids conf))
         run-profiler-actions-fn (mk-run-profiler-actions-for-all-topologies supervisor)
-        heartbeat-fn (fn [] (.supervisor-heartbeat!
+        heartbeat-fn (fn [] (.supervisorHeartbeat
                                (:storm-cluster-state supervisor)
                                (:supervisor-id supervisor)
-                               (->SupervisorInfo (current-time-secs)
+                              (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
                                                  (:my-hostname supervisor)
                                                  (:assignment-id supervisor)
                                                  (keys @(:curr-assignment supervisor))
@@ -788,7 +787,7 @@
                                                  (conf SUPERVISOR-SCHEDULER-META)
                                                  ((:uptime supervisor))
                                                  (:version supervisor)
-                                                 (mk-supervisor-capacities conf))))]
+                                                 (mk-supervisor-capacities conf)))))]
     (heartbeat-fn)
 
     ;; should synchronize supervisor so it doesn't launch anything after being down (optimization)


[13/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 8df5885,0000000..17c8641
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@@ -1,664 -1,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.io.PrintWriter;
++import java.io.StringWriter;
 +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, IFn> assignmentInfoCallback;
 +    private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
 +    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
 +    private AtomicReference<IFn> supervisorsCallback;
 +    // we want to reigister a topo directory getChildren callback for all workers of this dir
 +    private ConcurrentHashMap<String, IFn> backPressureCallback;
 +    private AtomicReference<IFn> assignmentsCallback;
 +    private ConcurrentHashMap<String, IFn> stormBaseCallback;
 +    private AtomicReference<IFn> blobstoreCallback;
 +    private ConcurrentHashMap<String, IFn> credentialsCallback;
 +    private ConcurrentHashMap<String, IFn> 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 params = null;
 +                    String root = toks.get(0);
-                     IFn fn = null;
 +                    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<IFn> cb) {
 +        IFn callback = cb.getAndSet(null);
 +        if (callback != null)
 +            callback.invoke();
 +    }
 +
 +    protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
 +        IFn callback = callbackConcurrentHashMap.remove(key);
 +        if (callback != null)
 +            callback.invoke();
 +    }
 +
 +    @Override
 +    public List<String> assignments(IFn callback) {
 +        if (callback != null) {
 +            assignmentsCallback.set(callback);
 +        }
 +        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
 +    }
 +
 +    @Override
 +    public Assignment assignmentInfo(String stormId, IFn 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 APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
 +        if (callback != null) {
 +            assignmentInfoWithVersionCallback.put(stormId, callback);
 +        }
 +        Assignment assignment = null;
 +        Integer version = 0;
 +        APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
 +        if (aPersistentMap != null) {
 +            assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
 +            version = (Integer) aPersistentMap.get(RT.keyword(null, "version"));
 +        }
 +        APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
 +        return map;
 +    }
 +
 +    @Override
 +    public Integer assignmentVersion(String stormId, IFn 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, IFn 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
++    /**
++     * 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, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
-         Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
++    public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
++        Map<ExecutorInfo, APersistentMap> executorWhbs = new HashMap<>();
 +
 +        Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.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(IFn 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 (Zookeeper.exceptionCause(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 (Zookeeper.exceptionCause(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, IFn cb) {
 +        String path = ClusterUtils.logConfigPath(stormId);
 +        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
 +    }
 +
 +    @Override
 +    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
 +        if (info != null) {
 +            String path = ClusterUtils.workerbeatPath(stormId, node, port);
 +            stateStorage.set_worker_hb(path, Utils.serialize(info), acls);
 +        }
 +    }
 +
 +    @Override
 +    public void removeWorkerHeartbeat(String stormId, String node, Long port) {
 +        String path = ClusterUtils.workerbeatPath(stormId, node, port);
 +        stateStorage.delete_worker_hb(path);
 +    }
 +
 +    @Override
 +    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
 +        String path = ClusterUtils.supervisorPath(supervisorId);
 +        stateStorage.set_ephemeral_node(path, Utils.serialize(info), acls);
 +    }
 +
-     // if znode exists and to be not on?, delete; if exists and on?, do nothing;
-     // if not exists and to be on?, create; if not exists and not on?, do nothing;
++    /**
++     * if znode exists and to be not on?, delete; if exists and on?, do nothing; if not exists and to be on?, create; if not exists and not on?, do nothing;
++     * 
++     * @param stormId
++     * @param node
++     * @param port
++     * @param on
++     */
 +    @Override
 +    public void workerBackpressure(String stormId, String node, Long port, boolean on) {
 +        String path = ClusterUtils.backpressurePath(stormId, node, port);
 +        boolean existed = stateStorage.node_exists(path, false);
 +        if (existed) {
 +            if (on == false)
 +                stateStorage.delete_node(path);
 +
 +        } else {
 +            if (on == true) {
 +                stateStorage.set_ephemeral_node(path, null, acls);
 +            }
 +        }
 +    }
 +
-     // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
++    /**
++     * if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
++     * 
++     * @param stormId
++     * @param callback
++     * @return
++     */
 +    @Override
 +    public boolean topologyBackpressure(String stormId, IFn callback) {
 +        if (callback != null) {
 +            backPressureCallback.put(stormId, callback);
 +        }
 +        String path = ClusterUtils.backpressureStormRoot(stormId);
 +        List<String> childrens = stateStorage.get_children(path, callback != null);
 +        return childrens.size() > 0;
 +
 +    }
 +
 +    @Override
 +    public void setupBackpressure(String stormId) {
 +        stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls);
 +    }
 +
 +    @Override
 +    public void removeWorkerBackpressure(String stormId, String node, Long port) {
 +        stateStorage.delete_node(ClusterUtils.backpressurePath(stormId, node, port));
 +    }
 +
 +    @Override
 +    public void activateStorm(String stormId, StormBase stormBase) {
 +        String path = ClusterUtils.stormPath(stormId);
 +        stateStorage.set_data(path, Utils.serialize(stormBase), acls);
 +    }
 +
-     // To update this function due to APersistentMap/APersistentSet is clojure's structure
++    /**
++     * To update this function due to APersistentMap/APersistentSet is clojure's structure
++     * 
++     * @param stormId
++     * @param newElems
++     */
 +    @Override
 +    public void updateStorm(String stormId, StormBase newElems) {
 +
 +        StormBase stormBase = stormBase(stormId, null);
 +        if (stormBase.get_component_executors() != null) {
 +
 +            Map<String, Integer> newComponentExecutors = new HashMap<>();
 +            Map<String, Integer> componentExecutors = newElems.get_component_executors();
 +            // componentExecutors maybe be APersistentMap, which don't support "put"
 +            for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
 +                newComponentExecutors.put(entry.getKey(), entry.getValue());
 +            }
 +            for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
 +                if (!componentExecutors.containsKey(entry.getKey())) {
 +                    newComponentExecutors.put(entry.getKey(), entry.getValue());
 +                }
 +            }
 +            if (newComponentExecutors.size() > 0)
 +                newElems.set_component_executors(newComponentExecutors);
 +        }
 +
 +        Map<String, DebugOptions> ComponentDebug = new HashMap<>();
 +        Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
 +
 +        Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
 +        /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll
 +        Set<String> debugOptionsKeys = new HashSet<>();
 +        debugOptionsKeys.addAll(oldComponentDebug.keySet());
 +        debugOptionsKeys.addAll(newComponentDebug.keySet());
 +        for (String key : debugOptionsKeys) {
 +            boolean enable = false;
 +            double samplingpct = 0;
 +            if (oldComponentDebug.containsKey(key)) {
 +                enable = oldComponentDebug.get(key).is_enable();
 +                samplingpct = oldComponentDebug.get(key).get_samplingpct();
 +            }
 +            if (newComponentDebug.containsKey(key)) {
 +                enable = newComponentDebug.get(key).is_enable();
 +                samplingpct += newComponentDebug.get(key).get_samplingpct();
 +            }
 +            DebugOptions debugOptions = new DebugOptions();
 +            debugOptions.set_enable(enable);
 +            debugOptions.set_samplingpct(samplingpct);
 +            ComponentDebug.put(key, debugOptions);
 +        }
 +        if (ComponentDebug.size() > 0) {
 +            newElems.set_component_debug(ComponentDebug);
 +        }
 +
 +        if (StringUtils.isBlank(newElems.get_name())) {
 +            newElems.set_name(stormBase.get_name());
 +        }
 +        if (newElems.get_status() == null) {
 +            newElems.set_status(stormBase.get_status());
 +        }
 +        if (newElems.get_num_workers() == 0) {
 +            newElems.set_num_workers(stormBase.get_num_workers());
 +        }
 +        if (newElems.get_launch_time_secs() == 0) {
 +            newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
 +        }
 +        if (StringUtils.isBlank(newElems.get_owner())) {
 +            newElems.set_owner(stormBase.get_owner());
 +        }
 +        if (newElems.get_topology_action_options() == null) {
 +            newElems.set_topology_action_options(stormBase.get_topology_action_options());
 +        }
 +        if (newElems.get_status() == null) {
 +            newElems.set_status(stormBase.get_status());
 +        }
 +        stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls);
 +    }
 +
 +    @Override
 +    public void removeStormBase(String stormId) {
 +        stateStorage.delete_node(ClusterUtils.stormPath(stormId));
 +    }
 +
 +    @Override
 +    public void setAssignment(String stormId, Assignment info) {
 +        stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), acls);
 +    }
 +
 +    @Override
 +    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
 +        String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
 +        LOG.info("set-path: {}", path);
 +        stateStorage.mkdirs(ClusterUtils.blobstorePath(key), acls);
 +        stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString());
 +        stateStorage.set_ephemeral_node(path, null, acls);
 +    }
 +
 +    @Override
 +    public List<String> activeKeys() {
 +        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false);
 +    }
 +
 +    // blobstore state
 +    @Override
 +    public List<String> blobstore(IFn callback) {
 +        if (callback != null) {
 +            blobstoreCallback.set(callback);
 +        }
 +        stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE);
 +        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, callback != null);
 +
 +    }
 +
 +    @Override
 +    public void removeStorm(String stormId) {
 +        stateStorage.delete_node(ClusterUtils.assignmentPath(stormId));
 +        stateStorage.delete_node(ClusterUtils.credentialsPath(stormId));
 +        stateStorage.delete_node(ClusterUtils.logConfigPath(stormId));
 +        stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId));
 +        removeStormBase(stormId);
 +    }
 +
 +    @Override
 +    public void removeBlobstoreKey(String blobKey) {
 +        LOG.debug("remove key {}", blobKey);
 +        stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey));
 +    }
 +
 +    @Override
 +    public void removeKeyVersion(String blobKey) {
 +        stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey));
 +    }
 +
 +    @Override
-     public void reportError(String stormId, String componentId, String node, Long port, String error) {
++    public void reportError(String stormId, String componentId, String node, Long port, Throwable error) {
 +
 +        String path = ClusterUtils.errorPath(stormId, componentId);
 +        String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
-         ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
++        ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.StringifyError(error), Time.currentTimeSecs());
 +        errorInfo.set_host(node);
 +        errorInfo.set_port(port.intValue());
 +        byte[] serData = Utils.serialize(errorInfo);
 +        stateStorage.mkdirs(path, acls);
 +        stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, acls);
 +        stateStorage.set_data(lastErrorPath, serData, acls);
 +        List<String> childrens = stateStorage.get_children(path, false);
 +
 +        Collections.sort(childrens, new Comparator<String>() {
 +            public int compare(String arg0, String arg1) {
 +                return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1)));
 +            }
 +        });
 +
 +        while (childrens.size() > 10) {
 +            stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0));
 +        }
 +    }
 +
 +    @Override
 +    public List<ErrorInfo> errors(String stormId, String componentId) {
 +        List<ErrorInfo> errorInfos = new ArrayList<>();
 +        try {
 +            String path = ClusterUtils.errorPath(stormId, componentId);
 +            if (stateStorage.node_exists(path, false)) {
 +                List<String> childrens = stateStorage.get_children(path, false);
 +                for (String child : childrens) {
 +                    String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
 +                    ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
 +                    if (errorInfo != null)
 +                        errorInfos.add(errorInfo);
 +                }
 +            }
 +            Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
 +                public int compare(ErrorInfo arg0, ErrorInfo arg1) {
 +                    return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
 +                }
 +            });
 +        } catch (Exception e) {
 +            throw Utils.wrapInRuntime(e);
 +        }
 +
 +        return errorInfos;
 +    }
 +
 +    @Override
 +    public ErrorInfo lastError(String stormId, String componentId) {
 +
 +        String path = ClusterUtils.lastErrorPath(stormId, componentId);
 +        if (stateStorage.node_exists(path, false)) {
 +            ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class);
 +            return errorInfo;
 +        }
 +
 +        return null;
 +    }
 +
 +    @Override
 +    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
 +        List<ACL> aclList = ClusterUtils.mkTopoOnlyAcls(topoConf);
 +        String path = ClusterUtils.credentialsPath(stormId);
 +        stateStorage.set_data(path, Utils.serialize(creds), aclList);
 +
 +    }
 +
 +    @Override
 +    public Credentials credentials(String stormId, IFn callback) {
 +        if (callback != null) {
 +            credentialsCallback.put(stormId, callback);
 +        }
 +        String path = ClusterUtils.credentialsPath(stormId);
 +        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class);
 +
 +    }
 +
 +    @Override
 +    public void disconnect() {
 +        stateStorage.unregister(stateId);
 +        if (solo)
 +            stateStorage.close();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index f3b9253,0000000..956c20e
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@@ -1,36 -1,0 +1,36 @@@
 +/**
 + * 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.APersistentMap;
 +import org.apache.storm.utils.Utils;
 +import org.apache.zookeeper.data.ACL;
 +
 +import java.util.List;
 +
- public class ZKStateStorageFactory implements StateStorageFactory{
++public class ZKStateStorageFactory implements StateStorageFactory {
 +
 +    @Override
 +    public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
 +        try {
 +            return new ZKStateStorage(config, auth_conf, acls, context);
-         }catch (Exception e){
++        } catch (Exception e) {
 +            throw Utils.wrapInRuntime(e);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index af0e8f3,34f3665..20d6deb
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@@ -60,6 -60,6 +60,10 @@@ public class PacemakerClient implement
      private StormBoundedExponentialBackoffRetry backoff = new StormBoundedExponentialBackoffRetry(100, 5000, 20);
      private int retryTimes = 0;
  
++    //the constructor is invoked by pacemaker-state-factory-test
++    public PacemakerClient() {
++        bootstrap = new ClientBootstrap();
++    }
      public PacemakerClient(Map config) {
  
          String host = (String)config.get(Config.PACEMAKER_HOST);
@@@ -157,6 -157,7 +161,7 @@@
      public String secretKey() {
          return secret;
      }
 -
++    public HBMessage  checkCaptured() {return null;}
      public HBMessage send(HBMessage m) {
          waitUntilReady();
          LOG.debug("Sending message: {}", m.toString());

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 39adb9e,b146cb0..22c1f80
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -22,11 -22,11 +22,11 @@@
    (:import [org.mockito Mockito])
    (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
-   (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils ZookeeperAuthInfo ConfigUtils])
 -  (:import [org.apache.storm.cluster ClusterState])
 +  (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
    (:import [org.apache.storm.zookeeper Zookeeper])
 -  (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
 -  (:require [org.apache.storm [zookeeper :as zk]])
 +  (:import [org.apache.storm.callback ZKStateChangedCallback])
 +  (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
    (:require [conjure.core])
    (:use [conjure core])
    (:use [clojure test])
@@@ -39,14 -39,18 +39,18 @@@
  
  (defn mk-state
    ([zk-port] (let [conf (mk-config zk-port)]
 -               (mk-distributed-cluster-state conf :auth-conf conf)))
 +               (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
    ([zk-port cb]
 -     (let [ret (mk-state zk-port)]
 -       (.register ret cb)
 -       ret )))
 +    (let [ret (mk-state zk-port)]
 +      (.register ret cb)
 +      ret)))
  
 -(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
 +(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
  
+ (defn barr
+   [& vals]
+   (byte-array (map byte vals)))
+ 
  (deftest test-basics
    (with-inprocess-zookeeper zk-port
      (let [state (mk-state zk-port)]
@@@ -242,27 -244,26 +246,32 @@@
        (is (.contains (:error error) target))
        )))
  
++(defn- stringify-error [error]
++  (let [result (java.io.StringWriter.)
++        printer (java.io.PrintWriter. result)]
++    (.printStackTrace error printer)
++    (.toString result)))
  
  (deftest test-storm-cluster-state-errors
    (with-inprocess-zookeeper zk-port
      (with-simulated-time
        (let [state (mk-storm-state zk-port)]
-         (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
 -        (.report-error state "a" "1" (Utils/localHostname) 6700 (RuntimeException.))
++        (.reportError state "a" "1" (Utils/localHostname) 6700  (RuntimeException.))
          (validate-errors! state "a" "1" ["RuntimeException"])
          (advance-time-secs! 1)
-         (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
 -        (.report-error state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
++        (.reportError state "a" "1" (Utils/localHostname) 6700 (IllegalArgumentException.))
          (validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
          (doseq [i (range 10)]
-           (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
 -          (.report-error state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
++          (.reportError state "a" "2" (Utils/localHostname) 6700 (RuntimeException.))
            (advance-time-secs! 2))
          (validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
          (doseq [i (range 5)]
-           (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
 -          (.report-error state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
++          (.reportError state "a" "2" (Utils/localHostname) 6700 (IllegalArgumentException.))
            (advance-time-secs! 2))
          (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
 -                                                (repeat 5 "RuntimeException")
 -                                                ))
 +                                          (repeat 5 "RuntimeException")
 +                                          ))
 +
          (.disconnect state)
          ))))
  
@@@ -300,12 -301,12 +309,12 @@@
        (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
        (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
        (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
-       (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
+       (Utils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
        (is (nil?
 -           (try
 -             (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
 -             (catch MockitoAssertionError e
 -               e)))))))
 +            (try
 +              (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
 +              (catch MockitoAssertionError e
 +                e)))))))
  
  (deftest test-storm-state-callbacks
    ;; TODO finish
@@@ -313,17 -314,15 +322,17 @@@
  
  (deftest test-cluster-state-default-acls
    (testing "The default ACLs are empty."
 -    (let [zk-mock (Mockito/mock Zookeeper)]
 +    (let [zk-mock (Mockito/mock Zookeeper)
 +          curator-frameworke (reify CuratorFramework (^void close [this] nil))]
        ;; No need for when clauses because we just want to return nil
        (with-open [_ (MockedZookeeper. zk-mock)]
 -        (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
 -          (mk-distributed-cluster-state {})
 -          (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
 -    (stubbing [mk-distributed-cluster-state (reify ClusterState
 -                                              (register [this callback] nil)
 -                                              (mkdirs [this path acls] nil))]
 -     (mk-storm-cluster-state {})
 -     (verify-call-times-for mk-distributed-cluster-state 1)
 -     (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
 +        (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
 +        (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
 +        (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
 +    (let [distributed-state-storage (reify IStateStorage
 +                                      (register [this callback] nil)
 +                                      (mkdirs [this path acls] nil))
 +          cluster-utils (Mockito/mock ClusterUtils)]
 +      (with-open [mocked-cluster (MockedCluster. cluster-utils)]
-         (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
++        (. (Mockito/when (.mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
 +        (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 772a232,70cb885..09c4371
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -23,27 -23,33 +23,36 @@@
             [org.apache.storm.nimbus InMemoryTopologyActionNotifier])
    (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
    (:import [org.apache.storm.scheduler INimbus])
 +  (:import [org.mockito Mockito])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
-   (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
++  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
              TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
              InvalidTopologyException AuthorizationException
              LogConfig LogLevel LogLevelAction])
    (:import [java.util HashMap])
    (:import [java.io File])
-   (:import [org.apache.storm.utils Time Utils ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils Utils$UptimeComputer ConfigUtils IPredicate]
+            [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
    (:import [org.apache.storm.zookeeper Zookeeper])
-   (:import [org.apache.commons.io FileUtils])
+   (:import [org.apache.commons.io FileUtils]
+            [org.json.simple JSONValue])
 -  (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
 +  (:use [org.apache.storm testing MockAutoCred util config log timer converter])
    (:use [org.apache.storm.daemon common])
    (:require [conjure.core])
    (:require [org.apache.storm
 -             [thrift :as thrift]
 -             [cluster :as cluster]])
 +             [thrift :as thrift]])
    (:use [conjure core]))
  
+ (defn- from-json
+        [^String str]
+        (if str
+          (clojurify-structure
+            (JSONValue/parse str))
+          nil))
+ 
  (defn storm-component->task-info [cluster storm-name]
    (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
          nimbus (:nimbus cluster)]
@@@ -72,8 -80,8 +83,8 @@@
  
  (defn storm-num-workers [state storm-name]
    (let [storm-id (get-storm-id state storm-name)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (count (reverse-map (:executor->node+port assignment)))
+     (count (clojurify-structure (Utils/reverseMap (:executor->node+port assignment))))
      ))
  
  (defn topology-nodes [state storm-name]
@@@ -95,9 -103,11 +106,11 @@@
           set         
           )))
  
+ ;TODO: when translating this function, don't call map-val, but instead use an inline for loop.
+ ; map-val is a temporary kluge for clojure.
  (defn topology-node-distribution [state storm-name]
    (let [storm-id (get-storm-id state storm-name)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
      (->> assignment
           :executor->node+port
           vals
@@@ -124,19 -134,18 +137,18 @@@
  
  (defn do-executor-heartbeat [cluster storm-id executor]
    (let [state (:storm-cluster-state cluster)
 -        executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
 +        executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
          [node port] (get executor->node+port executor)
 -        curr-beat (.get-worker-heartbeat state storm-id node port)
 +        curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
          stats (:executor-stats curr-beat)]
 -    (.worker-heartbeat! state storm-id node port
 -      {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
 +    (.workerHeartbeat state storm-id node port
-       (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
++      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (Time/currentTimeSecs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
        )))
  
  (defn slot-assignments [cluster storm-id]
    (let [state (:storm-cluster-state cluster)
 -        assignment (.assignment-info state storm-id nil)]
 +        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (reverse-map (:executor->node+port assignment))
-     ))
+         (clojurify-structure (Utils/reverseMap (:executor->node+port assignment)))))
  
  (defn task-ids [cluster storm-id]
    (let [nimbus (:nimbus cluster)]
@@@ -146,8 -155,10 +158,10 @@@
  
  (defn topology-executors [cluster storm-id]
    (let [state (:storm-cluster-state cluster)
-         assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
-     (keys (:executor->node+port assignment))
 -        assignment (.assignment-info state storm-id nil)
 -        ret-keys (keys (:executor->node+port assignment))
++        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
++    ret-keys (keys (:executor->node+port assignment))
+         _ (log-message "ret-keys: " (pr-str ret-keys)) ]
+     ret-keys
      ))
  
  (defn check-distribution [items distribution]
@@@ -1350,23 -1399,27 +1402,27 @@@
                       NIMBUS-THRIFT-PORT 6666})
            expected-acls nimbus/NIMBUS-ZK-ACLS
            fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+           fake-cu (proxy [ConfigUtils] []
 -                      (nimbusTopoHistoryStateImpl [conf] nil))
++                    (nimbusTopoHistoryStateImpl [conf] nil))
+           fake-utils (proxy [Utils] []
+                        (newInstanceImpl [_])
+                        (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
 -                                                (upTime [] 0))))]
++                                                (upTime [] 0))))
 +          cluster-utils (Mockito/mock ClusterUtils)]
-       (with-open [_ (proxy [MockedConfigUtils] []
-                       (nimbusTopoHistoryStateImpl [conf] nil))
+       (with-open [_ (ConfigUtilsInstaller. fake-cu)
+                   _ (UtilsInstaller. fake-utils)
                    zk-le (MockedZookeeper. (proxy [Zookeeper] []
 -                          (zkLeaderElectorImpl [conf] nil)))]
 +                          (zkLeaderElectorImpl [conf] nil)))
 +                  mocked-cluster (MockedCluster. cluster-utils)]
          (stubbing [mk-authorization-handler nil
 -                   cluster/mk-storm-cluster-state nil
 -                   nimbus/file-cache-map nil
 -                   nimbus/mk-blob-cache-map nil
 -                   nimbus/mk-bloblist-cache-map nil
 -                   mk-timer nil
 -                   nimbus/mk-scheduler nil]
 -                  (nimbus/nimbus-data auth-conf fake-inimbus)
 -                  (verify-call-times-for cluster/mk-storm-cluster-state 1)
 -                  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
 -                                                      expected-acls))))))
 +                 nimbus/file-cache-map nil
 +                 nimbus/mk-blob-cache-map nil
 +                 nimbus/mk-bloblist-cache-map nil
-                  uptime-computer nil
-                  new-instance nil
 +                 mk-timer nil
 +                 nimbus/mk-scheduler nil]
 +          (nimbus/nimbus-data auth-conf fake-inimbus)
 +          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
 +          )))))
  
  (deftest test-file-bogus-download
    (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1397,9 -1450,9 +1453,9 @@@
                        STORM-CLUSTER-MODE "local"
                        STORM-ZOOKEEPER-PORT zk-port
                        STORM-LOCAL-DIR nimbus-dir}))
 -        (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +        (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-         (sleep-secs 1)
+         (Time/sleepSecs 1)
          (bind topology (thrift/mk-topology
                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                           {}))
@@@ -1429,10 -1482,10 +1485,10 @@@
                          STORM-ZOOKEEPER-PORT zk-port
                          STORM-LOCAL-DIR nimbus-dir
                          NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
 -          (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
            (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
            (bind notifier (InMemoryTopologyActionNotifier.))
-           (sleep-secs 1)
+           (Time/sleepSecs 1)
            (bind topology (thrift/mk-topology
                             {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
                             {}))

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 1a7bd2c,0925237..1c45266
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@@ -19,10 -20,8 +19,11 @@@
    (:import [org.apache.storm.generated
              HBExecutionException HBNodes HBRecords
              HBServerMessageType HBMessage HBMessageData HBPulse]
-            [org.apache.storm.cluster ClusterStateContext  PaceMakerStateStorageFactory]
 -           [org.apache.storm.cluster ClusterStateContext]
 -           [org.mockito Mockito Matchers]))
++           [org.apache.storm.cluster ClusterStateContext  PaceMakerStateStorageFactory PaceMakerStateStorage]
 +           [org.mockito Mockito Matchers])
 +(:import [org.mockito.exceptions.base MockitoAssertionError])
++(:import [org.apache.storm.pacemaker PacemakerClient])
 +(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
  
  (defn- string-to-bytes [string]
    (byte-array (map int string)))
@@@ -30,26 -29,24 +31,23 @@@
  (defn- bytes-to-string [bytez]
    (apply str (map char bytez)))
  
--(defprotocol send-capture
--  (send [this something])
--  (check-captured [this]))
--
  (defn- make-send-capture [response]
    (let [captured (atom nil)]
--    (reify send-capture
--      (send [this something] (reset! captured something) response)
--      (check-captured [this] @captured))))
 -
 -(defmacro with-mock-pacemaker-client-and-state [client state response & body]
 -  `(let [~client (make-send-capture ~response)]
 -     (stubbing [psf/makeZKState nil
 -                psf/makeClient ~client]
 -               (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
++    (proxy [PacemakerClient] []
++      (send [m] (reset! captured m) response)
++      (checkCaptured [] @captured))))
 +
 +(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
 +  `(let [~client (make-send-capture ~response)
 +         ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
 +
 +     (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
 +       (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
 +       (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
-                (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
++               (let [~state (PaceMakerStateStorage. (PaceMakerStateStorageFactory/initMakeClient nil)
++                   (PaceMakerStateStorageFactory/initZKstate nil  nil nil nil))]
                   ~@body))))
  
 -
  (deftest pacemaker_state_set_worker_hb
    (testing "set_worker_hb"
      (with-mock-pacemaker-client-and-state
@@@ -57,7 -54,7 +55,7 @@@
        (HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
  
        (.set_worker_hb state "/foo" (string-to-bytes "data") nil)
--      (let [sent (.check-captured client)
++      (let [sent (.checkCaptured client)
              pulse (.get_pulse (.get_data sent))]
          (is (= (.get_type sent) HBServerMessageType/SEND_PULSE))
          (is (= (.get_id pulse) "/foo"))
@@@ -65,13 -62,13 +63,12 @@@
  
    (testing "set_worker_hb"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/SEND_PULSE nil)
  
-       (is (thrown? RuntimeException      
 -      (is (thrown? HBExecutionException      
--                   (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
++      (is (thrown? RuntimeException
++            (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
  
--      
  
  (deftest pacemaker_state_delete_worker_hb
    (testing "delete_worker_hb"
@@@ -80,74 -77,74 +77,75 @@@
        (HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
  
        (.delete_worker_hb state "/foo/bar")
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/DELETE_PATH))
          (is (= (.get_path (.get_data sent)) "/foo/bar")))))
  
--    (testing "delete_worker_hb"
--      (with-mock-pacemaker-client-and-state
-         client state pacefactory mock
 -        client state
--        (HBMessage. HBServerMessageType/DELETE_PATH nil)
--        
-         (is (thrown? RuntimeException
 -        (is (thrown? HBExecutionException
--                     (.delete_worker_hb state "/foo/bar"))))))
++  (testing "delete_worker_hb"
++    (with-mock-pacemaker-client-and-state
++      client state pacefactory mock
++      (HBMessage. HBServerMessageType/DELETE_PATH nil)
++
++      (is (thrown? RuntimeException
++            (.delete_worker_hb state "/foo/bar"))))))
  
  (deftest pacemaker_state_get_worker_hb
    (testing "get_worker_hb"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
--                (HBMessageData/pulse
--                 (doto (HBPulse.)
--                   (.set_id "/foo")
--                   (.set_details (string-to-bytes "some data")))))
++        (HBMessageData/pulse
++          (doto (HBPulse.)
++            (.set_id "/foo")
++            (.set_details (string-to-bytes "some data")))))
  
        (.get_worker_hb state "/foo" false)
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/GET_PULSE))
          (is (= (.get_path (.get_data sent)) "/foo")))))
  
    (testing "get_worker_hb - fail (bad response)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE nil)
--      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb state "/foo" false)))))
 -  
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb state "/foo" false)))))
-   
++            (.get_worker_hb state "/foo" false)))))
++
    (testing "get_worker_hb - fail (bad data)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
--      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb state "/foo" false))))))
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb state "/foo" false))))))
++            (.get_worker_hb state "/foo" false))))))
  
  (deftest pacemaker_state_get_worker_hb_children
    (testing "get_worker_hb_children"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
--                (HBMessageData/nodes
--                 (HBNodes. [])))
++        (HBMessageData/nodes
++          (HBNodes. [])))
  
        (.get_worker_hb_children state "/foo" false)
--      (let [sent (.check-captured client)]
++      (let [sent (.checkCaptured client)]
          (is (= (.get_type sent) HBServerMessageType/GET_ALL_NODES_FOR_PATH))
          (is (= (.get_path (.get_data sent)) "/foo")))))
  
    (testing "get_worker_hb_children - fail (bad response)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/DELETE_PATH nil)
  
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb_children state "/foo" false)))))
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb_children state "/foo" false)))))
++            (.get_worker_hb_children state "/foo" false)))))
  
--    (testing "get_worker_hb_children - fail (bad data)"
++  (testing "get_worker_hb_children - fail (bad data)"
      (with-mock-pacemaker-client-and-state
 -      client state
 +      client state pacefactory mock
        (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
-       ;need been update due to HBExecutionException
 -      
 -      (is (thrown? HBExecutionException
 -                   (.get_worker_hb_children state "/foo" false))))))
++
 +      (is (thrown? RuntimeException
-                    (.get_worker_hb_children state "/foo" false))))))
++            (.get_worker_hb_children state "/foo" false))))))
++

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index b89b7bb,9c31ddf..3ebdbcd
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -21,20 -21,20 +21,23 @@@
    (:require [clojure [string :as string] [set :as set]])
    (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
    (:import [org.apache.storm.scheduler ISupervisor])
-   (:import [org.apache.storm.utils ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
    (:import [org.apache.storm.generated RebalanceOptions])
-   (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
 -  (:import [org.mockito Matchers Mockito])
++  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [java.util UUID])
-   (:import [org.mockito Mockito])
++  (:import [org.mockito Mockito Matchers])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [java.io File])
    (:import [java.nio.file Files])
-   (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
 -  (:import [org.apache.storm.utils Utils IPredicate]
++  (:import [org.apache.storm.utils Utils IPredicate])
++  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
+            [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
+                                                  UtilsInstaller])
    (:import [java.nio.file.attribute FileAttribute])
 -  (:use [org.apache.storm config testing util timer log])
 +  (:use [org.apache.storm config testing util timer log converter])
    (:use [org.apache.storm.daemon common])
    (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
 -            [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
 +            [org.apache.storm [thrift :as thrift]])
    (:use [conjure core])
    (:require [clojure.java.io :as io]))
  
@@@ -43,9 -43,10 +46,10 @@@
    [cluster supervisor-id port]
    (let [state (:storm-cluster-state cluster)
          slot-assigns (for [storm-id (.assignments state nil)]
 -                        (let [executors (-> (.assignment-info state storm-id nil)
 +                        (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
                                          :executor->node+port
-                                         reverse-map
+                                         (Utils/reverseMap)
+                                         clojurify-structure
                                          (get [supervisor-id port] ))]
                            (when executors [storm-id executors])
                            ))
@@@ -565,198 -632,203 +635,201 @@@
            fake-isupervisor (reify ISupervisor
                               (getSupervisorId [this] nil)
                               (getAssignmentId [this] nil))
+           fake-cu (proxy [ConfigUtils] []
+                     (supervisorStateImpl [conf] nil)
+                     (supervisorLocalDirImpl [conf] nil))
+           fake-utils (proxy [Utils] []
+                        (localHostnameImpl [] nil)
+                        (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
 -                                                (upTime [] 0))))]
++                                                (upTime [] 0))))
 +          cluster-utils (Mockito/mock ClusterUtils)]
-       (with-open [_ (proxy [MockedConfigUtils] []
-                       (supervisorStateImpl [conf] nil)
-                       (supervisorLocalDirImpl [conf] nil))
+       (with-open [_ (ConfigUtilsInstaller. fake-cu)
 -                  _ (UtilsInstaller. fake-utils)]
 -        (stubbing [cluster/mk-storm-cluster-state nil
 -                   mk-timer nil]
++                  _ (UtilsInstaller. fake-utils)
 +                  mocked-cluster (MockedCluster. cluster-utils)]
-         (stubbing [uptime-computer nil
-               ;   cluster/mk-storm-cluster-state nil
-                  local-hostname nil
-                  mk-timer nil]
            (supervisor/supervisor-data auth-conf nil fake-isupervisor)
-           (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
-         ;  (verify-call-times-for cluster/mk-storm-cluster-state 1)
-         ;  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-         ;                                     expected-acls)
-          )))))
 -          (verify-call-times-for cluster/mk-storm-cluster-state 1)
 -          (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
 -              expected-acls)))))
++          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))))))
  
- (deftest test-write-log-metadata
-   (testing "supervisor writes correct data to logs metadata file"
-     (let [exp-owner "alice"
-           exp-worker-id "42"
-           exp-storm-id "0123456789"
-           exp-port 4242
-           exp-logs-users ["bob" "charlie" "daryl"]
-           exp-logs-groups ["read-only-group" "special-group"]
-           storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
-                       TOPOLOGY-USERS ["charlie" "bob"]
-                       TOPOLOGY-GROUPS ["special-group"]
-                       LOGS-GROUPS ["read-only-group"]
-                       LOGS-USERS ["daryl"]}
-           exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
-                     "worker-id" exp-worker-id
-                     LOGS-USERS exp-logs-users
-                     LOGS-GROUPS exp-logs-groups}
-           conf {}]
-       (mocking [supervisor/write-log-metadata-to-yaml-file!]
-         (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
-                                         exp-storm-id exp-port conf)
-         (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
-                                       exp-storm-id exp-port exp-data conf)))))
+   (deftest test-write-log-metadata
+     (testing "supervisor writes correct data to logs metadata file"
+       (let [exp-owner "alice"
+             exp-worker-id "42"
+             exp-storm-id "0123456789"
+             exp-port 4242
+             exp-logs-users ["bob" "charlie" "daryl"]
+             exp-logs-groups ["read-only-group" "special-group"]
+             storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
+                         TOPOLOGY-USERS ["charlie" "bob"]
+                         TOPOLOGY-GROUPS ["special-group"]
+                         LOGS-GROUPS ["read-only-group"]
+                         LOGS-USERS ["daryl"]}
+             exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
+                       "worker-id" exp-worker-id
+                       LOGS-USERS exp-logs-users
+                       LOGS-GROUPS exp-logs-groups}
+             conf {}]
+         (mocking [supervisor/write-log-metadata-to-yaml-file!]
+           (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
+             exp-storm-id exp-port conf)
+           (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
+             exp-storm-id exp-port exp-data conf)))))
  
- (deftest test-worker-launcher-requires-user
-   (testing "worker-launcher throws on blank user"
-     (mocking [launch-process]
-       (is (thrown-cause-with-msg? java.lang.IllegalArgumentException
-                                   #"(?i).*user cannot be blank.*"
-                                   (supervisor/worker-launcher {} nil ""))))))
+   (deftest test-worker-launcher-requires-user
+     (testing "worker-launcher throws on blank user"
+       (let [utils-proxy (proxy [Utils] []
+                           (launchProcessImpl [& _] nil))]
+         (with-open [_ (UtilsInstaller. utils-proxy)]
+           (is (try
+                 (supervisor/worker-launcher {} nil "")
+                 false
+                 (catch Throwable t
+                   (and (re-matches #"(?i).*user cannot be blank.*" (.getMessage t))
+                        (Utils/exceptionCauseIsInstanceOf java.lang.IllegalArgumentException t)))))))))
  
- (defn found? [sub-str input-str]
-   (if (string? input-str)
-     (contrib-str/substring? sub-str (str input-str))
-     (boolean (some #(contrib-str/substring? sub-str %) input-str))))
+   (defn found? [sub-str input-str]
+     (if (string? input-str)
+       (contrib-str/substring? sub-str (str input-str))
+       (boolean (some #(contrib-str/substring? sub-str %) input-str))))
  
- (defn not-found? [sub-str input-str]
+   (defn not-found? [sub-str input-str]
      (complement (found? sub-str input-str)))
  
- (deftest test-substitute-childopts-happy-path-string
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-string
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-happy-path-list
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-list
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-happy-path-list-arraylist
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-list-arraylist
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-topology-id-alone
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-topology-id-alone
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-no-keys
-   (testing "worker-launcher has no ids to replace in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-no-keys
+     (testing "worker-launcher has no ids to replace in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-nil-childopts
-   (testing "worker-launcher has nil childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts nil
-           expected-childopts nil
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-nil-childopts
+     (testing "worker-launcher has nil childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts nil
+             expected-childopts nil
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-nil-ids
-   (testing "worker-launcher has nil ids"
-     (let [worker-id nil
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-nil-ids
+     (testing "worker-launcher has nil ids"
+       (let [worker-id nil
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-retry-read-assignments
-   (with-simulated-time-local-cluster [cluster
-                                       :supervisors 0
-                                       :ports-per-supervisor 2
-                                       :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
-                                                     NIMBUS-MONITOR-FREQ-SECS 10
-                                                     TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
-                                                     TOPOLOGY-ACKER-EXECUTORS 0}]
-     (letlocals
-      (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
-      (bind topology1 (thrift/mk-topology
-                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
-                       {}))
-      (bind topology2 (thrift/mk-topology
-                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
-                       {}))
-      (bind state (:storm-cluster-state cluster))
-      (bind changed (capture-changed-workers
-                     (submit-mocked-assignment
-                      (:nimbus cluster)
-                      (:storm-cluster-state cluster)
-                      "topology1"
-                      {TOPOLOGY-WORKERS 2}
-                      topology1
-                      {1 "1"
-                       2 "1"}
-                      {[1 1] ["sup1" 1]
-                       [2 2] ["sup1" 2]}
-                      {["sup1" 1] [0.0 0.0 0.0]
-                       ["sup1" 2] [0.0 0.0 0.0]
-                       })
-                     (submit-mocked-assignment
-                      (:nimbus cluster)
-                      (:storm-cluster-state cluster)
-                      "topology2"
-                      {TOPOLOGY-WORKERS 2}
-                      topology2
-                      {1 "1"
-                       2 "1"}
-                      {[1 1] ["sup1" 1]
-                       [2 2] ["sup1" 2]}
-                      {["sup1" 1] [0.0 0.0 0.0]
-                       ["sup1" 2] [0.0 0.0 0.0]
-                       })
-                     ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
-                     (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
-                     ))
-      (is (empty? (:launched changed)))
-      (bind options (RebalanceOptions.))
-      (.set_wait_secs options 0)
-      (bind changed (capture-changed-workers
-                     (.rebalance (:nimbus cluster) "topology2" options)
-                     (advance-cluster-time cluster 10)
-                     (heartbeat-workers cluster "sup1" [1 2 3 4])
-                     (advance-cluster-time cluster 10)
-                     ))
-      (validate-launched-once (:launched changed)
-                              {"sup1" [1 2]}
-                              (get-storm-id (:storm-cluster-state cluster) "topology1"))
-      (validate-launched-once (:launched changed)
-                              {"sup1" [3 4]}
-                              (get-storm-id (:storm-cluster-state cluster) "topology2"))
-      )))
+   (deftest test-retry-read-assignments
+     (with-simulated-time-local-cluster [cluster
+                                         :supervisors 0
+                                         :ports-per-supervisor 2
+                                         :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
+                                                       NIMBUS-MONITOR-FREQ-SECS 10
+                                                       TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+                                                       TOPOLOGY-ACKER-EXECUTORS 0}]
+       (letlocals
+         (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+         (bind topology1 (thrift/mk-topology
+                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+                           {}))
+         (bind topology2 (thrift/mk-topology
+                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+                           {}))
+         (bind state (:storm-cluster-state cluster))
+         (bind changed (capture-changed-workers
+                         (submit-mocked-assignment
+                           (:nimbus cluster)
+                           (:storm-cluster-state cluster)
+                           "topology1"
+                           {TOPOLOGY-WORKERS 2}
+                           topology1
+                           {1 "1"
+                            2 "1"}
+                           {[1 1] ["sup1" 1]
+                            [2 2] ["sup1" 2]}
+                           {["sup1" 1] [0.0 0.0 0.0]
+                            ["sup1" 2] [0.0 0.0 0.0]
+                            })
+                         (submit-mocked-assignment
+                           (:nimbus cluster)
+                           (:storm-cluster-state cluster)
+                           "topology2"
+                           {TOPOLOGY-WORKERS 2}
+                           topology2
+                           {1 "1"
+                            2 "1"}
+                           {[1 1] ["sup1" 1]
+                            [2 2] ["sup1" 2]}
+                           {["sup1" 1] [0.0 0.0 0.0]
+                            ["sup1" 2] [0.0 0.0 0.0]
+                            })
+                         ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                         (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+                         ))
+         (is (empty? (:launched changed)))
+         (bind options (RebalanceOptions.))
+         (.set_wait_secs options 0)
+         (bind changed (capture-changed-workers
+                         (.rebalance (:nimbus cluster) "topology2" options)
+                         (advance-cluster-time cluster 10)
+                         (heartbeat-workers cluster "sup1" [1 2 3 4])
+                         (advance-cluster-time cluster 10)
+                         ))
+         (validate-launched-once (:launched changed)
+           {"sup1" [1 2]}
+           (get-storm-id (:storm-cluster-state cluster) "topology1"))
+         (validate-launched-once (:launched changed)
+           {"sup1" [3 4]}
+           (get-storm-id (:storm-cluster-state cluster) "topology2"))
 -        ))))
++        )))


[19/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/cc1f6d77
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/cc1f6d77
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/cc1f6d77

Branch: refs/heads/master
Commit: cc1f6d77962ec96c69cf5af85e4016a9e08ccdcc
Parents: afcd0c6 d0cd52b
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 19 16:22:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 19 16:47:42 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   7 +
 README.markdown                                 |   1 +
 bin/storm.cmd                                   |   2 +-
 bin/storm.py                                    |   4 +-
 conf/cgconfig.conf.example                      |  41 +++
 conf/defaults.yaml                              |  16 +-
 examples/storm-starter/pom.xml                  |  10 +
 .../org/apache/storm/starter/clj/word_count.clj |   3 +-
 .../starter/ResourceAwareExampleTopology.java   |   2 +-
 pom.xml                                         |  10 +
 storm-clojure/pom.xml                           |  74 ++++
 .../src/clj/org/apache/storm/clojure.clj        | 207 +++++++++++
 .../src/clj/org/apache/storm/thrift.clj         | 286 +++++++++++++++
 storm-clojure/src/test/clj/clojure_test.clj     | 158 +++++++++
 storm-core/src/clj/org/apache/storm/clojure.clj | 207 -----------
 .../org/apache/storm/command/dev_zookeeper.clj  |  28 --
 .../clj/org/apache/storm/command/get_errors.clj |   3 +-
 .../org/apache/storm/command/healthcheck.clj    |  90 -----
 .../clj/org/apache/storm/command/monitor.clj    |   2 +-
 .../clj/org/apache/storm/command/rebalance.clj  |   3 +-
 .../org/apache/storm/command/set_log_level.clj  |   3 +-
 .../apache/storm/command/shell_submission.clj   |   2 +-
 .../src/clj/org/apache/storm/daemon/common.clj  | 119 ++++---
 .../clj/org/apache/storm/daemon/executor.clj    |  51 +--
 .../clj/org/apache/storm/daemon/logviewer.clj   |  19 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 122 +++----
 .../clj/org/apache/storm/daemon/supervisor.clj  | 217 ++++++++----
 .../src/clj/org/apache/storm/daemon/task.clj    |   4 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  | 116 +++---
 .../clj/org/apache/storm/internal/clojure.clj   | 201 +++++++++++
 .../clj/org/apache/storm/internal/thrift.clj    |  96 +++++
 .../src/clj/org/apache/storm/local_state.clj    | 134 -------
 .../org/apache/storm/local_state_converter.clj  |  24 ++
 storm-core/src/clj/org/apache/storm/testing.clj |  37 +-
 storm-core/src/clj/org/apache/storm/thrift.clj  | 286 ---------------
 storm-core/src/clj/org/apache/storm/timer.clj   | 128 -------
 storm-core/src/clj/org/apache/storm/ui/core.clj |   2 +-
 storm-core/src/jvm/org/apache/storm/Config.java |  88 +++++
 .../src/jvm/org/apache/storm/StormTimer.java    | 241 +++++++++++++
 storm-core/src/jvm/org/apache/storm/Thrift.java | 351 +++++++++++++++++++
 .../org/apache/storm/command/DevZookeeper.java  |  35 ++
 .../org/apache/storm/command/HealthCheck.java   | 125 +++++++
 .../container/ResourceIsolationInterface.java   |  51 +++
 .../storm/container/cgroup/CgroupCenter.java    | 216 ++++++++++++
 .../storm/container/cgroup/CgroupCommon.java    | 270 ++++++++++++++
 .../container/cgroup/CgroupCommonOperation.java |  81 +++++
 .../container/cgroup/CgroupCoreFactory.java     |  74 ++++
 .../storm/container/cgroup/CgroupManager.java   | 210 +++++++++++
 .../storm/container/cgroup/CgroupOperation.java |  79 +++++
 .../storm/container/cgroup/CgroupUtils.java     | 118 +++++++
 .../apache/storm/container/cgroup/Device.java   |  75 ++++
 .../storm/container/cgroup/Hierarchy.java       | 130 +++++++
 .../storm/container/cgroup/SubSystem.java       |  81 +++++
 .../storm/container/cgroup/SubSystemType.java   |  36 ++
 .../storm/container/cgroup/SystemOperation.java |  75 ++++
 .../storm/container/cgroup/core/BlkioCore.java  | 213 +++++++++++
 .../storm/container/cgroup/core/CgroupCore.java |  26 ++
 .../storm/container/cgroup/core/CpuCore.java    | 135 +++++++
 .../container/cgroup/core/CpuacctCore.java      |  71 ++++
 .../storm/container/cgroup/core/CpusetCore.java | 209 +++++++++++
 .../container/cgroup/core/DevicesCore.java      | 189 ++++++++++
 .../container/cgroup/core/FreezerCore.java      |  66 ++++
 .../storm/container/cgroup/core/MemoryCore.java | 188 ++++++++++
 .../storm/container/cgroup/core/NetClsCore.java |  69 ++++
 .../container/cgroup/core/NetPrioCore.java      |  65 ++++
 .../jvm/org/apache/storm/testing/NGrouping.java |   4 +-
 .../storm/testing/PythonShellMetricsBolt.java   |  14 +-
 .../storm/testing/PythonShellMetricsSpout.java  |   8 +-
 .../jvm/org/apache/storm/utils/LocalState.java  | 112 +++++-
 .../src/jvm/org/apache/storm/utils/Utils.java   |  17 +-
 .../org/apache/storm/integration_test.clj       | 259 +++++++-------
 .../org/apache/storm/testing4j_test.clj         |  72 ++--
 .../test/clj/org/apache/storm/clojure_test.clj  |  64 ++--
 .../test/clj/org/apache/storm/cluster_test.clj  |   3 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |  23 +-
 .../test/clj/org/apache/storm/grouping_test.clj |  56 +--
 .../storm/messaging/netty_integration_test.clj  |  19 +-
 .../clj/org/apache/storm/messaging_test.clj     |  14 +-
 .../test/clj/org/apache/storm/metrics_test.clj  |  85 +++--
 .../test/clj/org/apache/storm/nimbus_test.clj   | 259 +++++++++-----
 .../scheduler/resource_aware_scheduler_test.clj |   3 +-
 .../clj/org/apache/storm/supervisor_test.clj    |  55 +--
 .../clj/org/apache/storm/tick_tuple_test.clj    |  15 +-
 .../clj/org/apache/storm/transactional_test.clj |   3 +-
 .../test/jvm/org/apache/storm/TestCgroups.java  | 130 +++++++
 .../resource/TestResourceAwareScheduler.java    |   3 +
 86 files changed, 5939 insertions(+), 1561 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-clojure/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-clojure/src/clj/org/apache/storm/thrift.clj
index 0000000,bf13d23..45aa740
mode 000000,100644..100644
--- a/storm-clojure/src/clj/org/apache/storm/thrift.clj
+++ b/storm-clojure/src/clj/org/apache/storm/thrift.clj
@@@ -1,0 -1,286 +1,286 @@@
+ ;; 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.
+ 
+ (ns org.apache.storm.thrift
+   (:import [java.util HashMap]
+            [java.io Serializable]
+            [org.apache.storm.generated NodeInfo Assignment])
+   (:import [org.apache.storm.generated JavaObject Grouping Nimbus StormTopology
+             StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
+             ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
+             GlobalStreamId ComponentObject ComponentObject$_Fields
+             ShellComponent SupervisorInfo])
+   (:import [org.apache.storm.utils Utils NimbusClient ConfigUtils])
+   (:import [org.apache.storm Constants])
+   (:import [org.apache.storm.security.auth ReqContext])
+   (:import [org.apache.storm.grouping CustomStreamGrouping])
+   (:import [org.apache.storm.topology TopologyBuilder])
+   (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
+   (:import [org.apache.storm.thrift.transport TTransport]
+            (org.json.simple JSONValue))
 -  (:use [org.apache.storm util config log zookeeper]))
++  (:use [org.apache.storm util config log]))
+ 
+ (defn instantiate-java-object
+   [^JavaObject obj]
+   (let [name (symbol (.get_full_class_name obj))
+         args (map (memfn getFieldValue) (.get_args_list obj))]
+     (eval `(new ~name ~@args))))
+ 
+ (def grouping-constants
+   {Grouping$_Fields/FIELDS :fields
+    Grouping$_Fields/SHUFFLE :shuffle
+    Grouping$_Fields/ALL :all
+    Grouping$_Fields/NONE :none
+    Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized
+    Grouping$_Fields/CUSTOM_OBJECT :custom-object
+    Grouping$_Fields/DIRECT :direct
+    Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle})
+ 
+ (defn grouping-type
+   [^Grouping grouping]
+   (grouping-constants (.getSetField grouping)))
+ 
+ (defn field-grouping
+   [^Grouping grouping]
+   (when-not (= (grouping-type grouping) :fields)
+     (throw (IllegalArgumentException. "Tried to get grouping fields from non fields grouping")))
+   (.get_fields grouping))
+ 
+ (defn global-grouping?
+   [^Grouping grouping]
+   (and (= :fields (grouping-type grouping))
+        (empty? (field-grouping grouping))))
+ 
+ (defn parallelism-hint
+   [^ComponentCommon component-common]
+   (let [phint (.get_parallelism_hint component-common)]
+     (if-not (.is_set_parallelism_hint component-common) 1 phint)))
+ 
+ (defn nimbus-client-and-conn
+   ([host port]
+     (nimbus-client-and-conn host port nil))
+   ([host port as-user]
+   (log-message "Connecting to Nimbus at " host ":" port " as user: " as-user)
+   (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
+         nimbusClient (NimbusClient. conf host port nil as-user)
+         client (.getClient nimbusClient)
+         transport (.transport nimbusClient)]
+         [client transport] )))
+ 
+ (defmacro with-nimbus-connection
+   [[client-sym host port] & body]
+   `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+     (try
+       ~@body
+     (finally (.close conn#)))))
+ 
+ (defmacro with-configured-nimbus-connection
+   [client-sym & body]
+   `(let [conf# (clojurify-structure (ConfigUtils/readStormConfig))
+          context# (ReqContext/context)
+          user# (if (.principal context#) (.getName (.principal context#)))
+          nimbusClient# (NimbusClient/getConfiguredClientAs conf# user#)
+          ~client-sym (.getClient nimbusClient#)
+          conn# (.transport nimbusClient#)
+          ]
+      (try
+        ~@body
+      (finally (.close conn#)))))
+ 
+ (defn direct-output-fields
+   [fields]
+   (StreamInfo. fields true))
+ 
+ (defn output-fields
+   [fields]
+   (StreamInfo. fields false))
+ 
+ ;TODO: when translating this function, you should replace the map-val with a proper for loop HERE
+ (defn mk-output-spec
+   [output-spec]
+   (let [output-spec (if (map? output-spec)
+                       output-spec
+                       {Utils/DEFAULT_STREAM_ID output-spec})]
+     (map-val
+       (fn [out]
+         (if (instance? StreamInfo out)
+           out
+           (StreamInfo. out false)))
+       output-spec)))
+ 
+ (defnk mk-plain-component-common
+   [inputs output-spec parallelism-hint :conf nil]
+   (let [ret (ComponentCommon. (HashMap. inputs) (HashMap. (mk-output-spec output-spec)))]
+     (when parallelism-hint
+       (.set_parallelism_hint ret parallelism-hint))
+     (when conf
+       (.set_json_conf ret (JSONValue/toJSONString conf)))
+     ret))
+ 
+ (defnk mk-spout-spec*
+   [spout outputs :p nil :conf nil]
+   (SpoutSpec. (ComponentObject/serialized_java (Utils/javaSerialize spout))
+               (mk-plain-component-common {} outputs p :conf conf)))
+ 
+ (defn mk-shuffle-grouping
+   []
+   (Grouping/shuffle (NullStruct.)))
+ 
+ (defn mk-local-or-shuffle-grouping
+   []
+   (Grouping/local_or_shuffle (NullStruct.)))
+ 
+ (defn mk-fields-grouping
+   [fields]
+   (Grouping/fields fields))
+ 
+ (defn mk-global-grouping
+   []
+   (mk-fields-grouping []))
+ 
+ (defn mk-direct-grouping
+   []
+   (Grouping/direct (NullStruct.)))
+ 
+ (defn mk-all-grouping
+   []
+   (Grouping/all (NullStruct.)))
+ 
+ (defn mk-none-grouping
+   []
+   (Grouping/none (NullStruct.)))
+ 
+ (defn deserialized-component-object
+   [^ComponentObject obj]
+   (when (not= (.getSetField obj) ComponentObject$_Fields/SERIALIZED_JAVA)
+     (throw (RuntimeException. "Cannot deserialize non-java-serialized object")))
+   (Utils/javaDeserialize (.get_serialized_java obj) Serializable))
+ 
+ (defn serialize-component-object
+   [obj]
+   (ComponentObject/serialized_java (Utils/javaSerialize obj)))
+ 
+ (defn- mk-grouping
+   [grouping-spec]
+   (cond (nil? grouping-spec)
+         (mk-none-grouping)
+ 
+         (instance? Grouping grouping-spec)
+         grouping-spec
+ 
+         (instance? CustomStreamGrouping grouping-spec)
+         (Grouping/custom_serialized (Utils/javaSerialize grouping-spec))
+ 
+         (instance? JavaObject grouping-spec)
+         (Grouping/custom_object grouping-spec)
+ 
+         (sequential? grouping-spec)
+         (mk-fields-grouping grouping-spec)
+ 
+         (= grouping-spec :shuffle)
+         (mk-shuffle-grouping)
+ 
+         (= grouping-spec :local-or-shuffle)
+         (mk-local-or-shuffle-grouping)
+         (= grouping-spec :none)
+         (mk-none-grouping)
+ 
+         (= grouping-spec :all)
+         (mk-all-grouping)
+ 
+         (= grouping-spec :global)
+         (mk-global-grouping)
+ 
+         (= grouping-spec :direct)
+         (mk-direct-grouping)
+ 
+         true
+         (throw (IllegalArgumentException.
+                  (str grouping-spec " is not a valid grouping")))))
+ 
+ (defn- mk-inputs
+   [inputs]
+   (into {} (for [[stream-id grouping-spec] inputs]
+              [(if (sequential? stream-id)
+                 (GlobalStreamId. (first stream-id) (second stream-id))
+                 (GlobalStreamId. stream-id Utils/DEFAULT_STREAM_ID))
+               (mk-grouping grouping-spec)])))
+ 
+ (defnk mk-bolt-spec*
+   [inputs bolt outputs :p nil :conf nil]
+   (let [common (mk-plain-component-common (mk-inputs inputs) outputs p :conf conf)]
+     (Bolt. (ComponentObject/serialized_java (Utils/javaSerialize bolt))
+            common)))
+ 
+ (defnk mk-spout-spec
+   [spout :parallelism-hint nil :p nil :conf nil]
+   (let [parallelism-hint (if p p parallelism-hint)]
+     {:obj spout :p parallelism-hint :conf conf}))
+ 
+ (defn- shell-component-params
+   [command script-or-output-spec kwargs]
+   (if (string? script-or-output-spec)
+     [(into-array String [command script-or-output-spec])
+      (first kwargs)
+      (rest kwargs)]
+     [(into-array String command)
+      script-or-output-spec
+      kwargs]))
+ 
+ (defnk mk-bolt-spec
+   [inputs bolt :parallelism-hint nil :p nil :conf nil]
+   (let [parallelism-hint (if p p parallelism-hint)]
+     {:obj bolt :inputs inputs :p parallelism-hint :conf conf}))
+ 
+ (defn mk-shell-bolt-spec
+   [inputs command script-or-output-spec & kwargs]
+   (let [[command output-spec kwargs]
+         (shell-component-params command script-or-output-spec kwargs)]
+     (apply mk-bolt-spec inputs
+            (RichShellBolt. command (mk-output-spec output-spec)) kwargs)))
+ 
+ (defn mk-shell-spout-spec
+   [command script-or-output-spec & kwargs]
+   (let [[command output-spec kwargs]
+         (shell-component-params command script-or-output-spec kwargs)]
+     (apply mk-spout-spec
+            (RichShellSpout. command (mk-output-spec output-spec)) kwargs)))
+ 
+ (defn- add-inputs
+   [declarer inputs]
+   (doseq [[id grouping] (mk-inputs inputs)]
+     (.grouping declarer id grouping)))
+ 
+ (defn mk-topology
+   ([spout-map bolt-map]
+    (let [builder (TopologyBuilder.)]
+      (doseq [[name {spout :obj p :p conf :conf}] spout-map]
+        (-> builder (.setSpout name spout (if-not (nil? p) (int p) p)) (.addConfigurations conf)))
+      (doseq [[name {bolt :obj p :p conf :conf inputs :inputs}] bolt-map]
+        (-> builder (.setBolt name bolt (if-not (nil? p) (int p) p)) (.addConfigurations conf) (add-inputs inputs)))
+      (.createTopology builder)))
+   ([spout-map bolt-map state-spout-map]
+    (mk-topology spout-map bolt-map)))
+ 
+ ;; clojurify-structure is needed or else every element becomes the same after successive calls
+ ;; don't know why this happens
+ (def STORM-TOPOLOGY-FIELDS
+   (-> StormTopology/metaDataMap clojurify-structure keys))
+ 
+ (def SPOUT-FIELDS
+   [StormTopology$_Fields/SPOUTS
+    StormTopology$_Fields/STATE_SPOUTS])
+ 

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 870e7f6,0d29376..8d193e6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -17,7 -17,7 +17,7 @@@
    (:import [org.apache.storm StormSubmitter]
             [org.apache.storm.utils Utils]
             [org.apache.storm.zookeeper Zookeeper])
-   (:use [org.apache.storm thrift util config log])
 -  (:use [org.apache.storm util config log zookeeper])
++  (:use [org.apache.storm util config log])
    (:require [clojure.string :as str])
    (:import [org.apache.storm.utils ConfigUtils])
    (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index db342d2,db7fd40..99b34dc
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,8 -15,8 +15,8 @@@
  ;; limitations under the License.
  (ns org.apache.storm.daemon.common
    (:use [org.apache.storm log config util])
 -  (:import [org.apache.storm.generated StormTopology
 +  (:import [org.apache.storm.generated StormTopology NodeInfo
-             InvalidTopologyException GlobalStreamId]
+             InvalidTopologyException GlobalStreamId Grouping Grouping$_Fields]
             [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
             [org.apache.storm.daemon.metrics.reporters PreparableReporter]
             [com.codahale.metrics MetricRegistry])

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 902650c,92cc003..9ff93f8
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -39,8 -39,9 +39,9 @@@
    (:import [java.lang Thread Thread$UncaughtExceptionHandler]
             [java.util.concurrent ConcurrentLinkedQueue]
             [org.json.simple JSONValue]
-            [com.lmax.disruptor.dsl ProducerType])
-   (:require [org.apache.storm [thrift :as thrift] [stats :as stats]])
+            [com.lmax.disruptor.dsl ProducerType]
+            [org.apache.storm StormTimer])
 -  (:require [org.apache.storm [cluster :as cluster] [stats :as stats]])
++  (:require [org.apache.storm [stats :as stats]])
    (:require [org.apache.storm.daemon [task :as task]])
    (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
    (:require [clojure.set :as set]))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index beb6639,28a6fb8..7d28075
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -46,13 -46,14 +46,13 @@@
              KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
              ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
              BeginDownloadResult ListBlobsResult ComponentPageInfo TopologyPageInfo LogConfig LogLevel LogLevelAction
-             ProfileRequest ProfileAction NodeInfo])
+             ProfileRequest ProfileAction NodeInfo LSTopoHistory])
    (:import [org.apache.storm.daemon Shutdownable])
    (:import [org.apache.storm.validation ConfigValidation])
 -  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
 -  (:use [org.apache.storm util config log zookeeper])
 -  (:require [org.apache.storm [cluster :as cluster]
 -                            [converter :as converter]
 -                            [stats :as stats]])
 +  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
-   (:use [org.apache.storm util config log timer local-state converter])
++  (:use [org.apache.storm util config log converter])
 +  (:require [org.apache.storm [converter :as converter]
-                             [stats :as stats]])
++                              [stats :as stats]])
    (:require [clojure.set :as set])
    (:import [org.apache.storm.daemon.common StormBase Assignment])
    (:import [org.apache.storm.zookeeper Zookeeper])
@@@ -270,17 -279,17 +274,17 @@@
        {:status {:type :rebalancing}
         :prev-status status
         :topology-action-options (-> {:delay-secs delay :action :rebalance}
--                                  (assoc-non-nil :num-workers num-workers)
--                                  (assoc-non-nil :component->executors executor-overrides))
++                                  (converter/assoc-non-nil :num-workers num-workers)
++                                  (converter/assoc-non-nil :component->executors executor-overrides))
         })))
  
  (defn do-rebalance [nimbus storm-id status storm-base]
    (let [rebalance-options (:topology-action-options storm-base)]
 -    (.update-storm! (:storm-cluster-state nimbus)
 +    (.updateStorm (:storm-cluster-state nimbus)
        storm-id
-       (thriftify-storm-base (-> {:topology-action-options nil}
 -        (-> {:topology-action-options nil}
--          (assoc-non-nil :component->executors (:component->executors rebalance-options))
-           (assoc-non-nil :num-workers (:num-workers rebalance-options))))))
 -          (assoc-non-nil :num-workers (:num-workers rebalance-options)))))
++      (converter/thriftify-storm-base (-> {:topology-action-options nil}
++          (converter/assoc-non-nil :component->executors (:component->executors rebalance-options))
++          (converter/assoc-non-nil :num-workers (:num-workers rebalance-options))))))
    (mk-assignments nimbus :scratch-topology-id storm-id))
  
  (defn state-transitions [nimbus storm-id status storm-base]
@@@ -363,7 -372,7 +367,7 @@@
                                        storm-base-updates)]
  
               (when storm-base-updates
-                (.updateStorm (:storm-cluster-state nimbus) storm-id (thriftify-storm-base storm-base-updates))))))
 -               (.update-storm! (:storm-cluster-state nimbus) storm-id storm-base-updates)))))
++               (.updateStorm (:storm-cluster-state nimbus) storm-id (converter/thriftify-storm-base storm-base-updates))))))
         )))
  
  (defn transition-name! [nimbus storm-name event & args]
@@@ -1002,9 -1005,9 +1005,9 @@@
          topology (system-topology! storm-conf (read-storm-topology storm-id blob-store))
          num-executors (->> (all-components topology) (map-val num-start-executors))]
      (log-message "Activating " storm-name ": " storm-id)
 -    (.activate-storm! storm-cluster-state
 +    (.activateStorm storm-cluster-state
                        storm-id
-       (thriftify-storm-base (StormBase. storm-name
 -                      (StormBase. storm-name
++      (converter/thriftify-storm-base (StormBase. storm-name
                                    (Time/currentTimeSecs)
                                    {:type topology-initial-status}
                                    (storm-conf TOPOLOGY-WORKERS)
@@@ -1448,41 -1446,39 +1454,39 @@@
        (setup-blobstore nimbus))
  
      (when (is-leader nimbus :throw-exception false)
 -      (doseq [storm-id (.active-storms (:storm-cluster-state nimbus))]
 +      (doseq [storm-id (.activeStorms (:storm-cluster-state nimbus))]
          (transition! nimbus storm-id :startup)))
-     (schedule-recurring (:timer nimbus)
-                         0
-                         (conf NIMBUS-MONITOR-FREQ-SECS)
-                         (fn []
-                           (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN)
-                             (locking (:submit-lock nimbus)
-                               (mk-assignments nimbus)))
-                           (do-cleanup nimbus)))
+ 
+     (.scheduleRecurring (:timer nimbus)
+       0
+       (conf NIMBUS-MONITOR-FREQ-SECS)
+       (fn []
+         (when-not (conf ConfigUtils/NIMBUS_DO_NOT_REASSIGN)
+           (locking (:submit-lock nimbus)
+             (mk-assignments nimbus)))
+         (do-cleanup nimbus)))
      ;; Schedule Nimbus inbox cleaner
-     (schedule-recurring (:timer nimbus)
-                         0
-                         (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS)
-                         (fn []
-                           (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))))
+     (.scheduleRecurring (:timer nimbus)
+       0
+       (conf NIMBUS-CLEANUP-INBOX-FREQ-SECS)
+       (fn [] (clean-inbox (inbox nimbus) (conf NIMBUS-INBOX-JAR-EXPIRATION-SECS))))
      ;; Schedule nimbus code sync thread to sync code from other nimbuses.
      (if (instance? LocalFsBlobStore blob-store)
-       (schedule-recurring (:timer nimbus)
-                           0
-                           (conf NIMBUS-CODE-SYNC-FREQ-SECS)
-                           (fn []
-                             (blob-sync conf nimbus))))
+       (.scheduleRecurring (:timer nimbus)
+         0
+         (conf NIMBUS-CODE-SYNC-FREQ-SECS)
+         (fn [] (blob-sync conf nimbus))))
      ;; Schedule topology history cleaner
      (when-let [interval (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)]
-       (schedule-recurring (:timer nimbus)
+       (.scheduleRecurring (:timer nimbus)
          0
          (conf LOGVIEWER-CLEANUP-INTERVAL-SECS)
-         (fn []
-           (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus))))
-     (schedule-recurring (:timer nimbus)
-                         0
-                         (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
-                         (fn []
-                           (renew-credentials nimbus)))
+         (fn [] (clean-topology-history (conf LOGVIEWER-CLEANUP-AGE-MINS) nimbus))))
+     (.scheduleRecurring (:timer nimbus)
+       0
+       (conf NIMBUS-CREDENTIAL-RENEW-FREQ-SECS)
+       (fn []
+         (renew-credentials nimbus)))
  
      (defgauge nimbus:num-supervisors
        (fn [] (.size (.supervisors (:storm-cluster-state nimbus) nil))))
@@@ -1650,7 -1646,7 +1654,7 @@@
            (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
              (if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
            (locking (:submit-lock nimbus)
-             (.updateStorm storm-cluster-state storm-id  (thriftify-storm-base storm-base-updates)))))
 -            (.update-storm! storm-cluster-state storm-id storm-base-updates))))
++            (.updateStorm storm-cluster-state storm-id  (converter/thriftify-storm-base storm-base-updates)))))
  
        (^void setWorkerProfiler
          [this ^String id ^ProfileRequest profileRequest]

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 58f6291,fad5b1a..f429d09
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -24,17 -24,17 +24,17 @@@
             [java.net JarURLConnection]
             [java.net URI URLDecoder]
             [org.apache.commons.io FileUtils])
-   (:use [org.apache.storm config util log timer local-state converter])
 -  (:use [org.apache.storm config util log local-state-converter])
++  (:use [org.apache.storm config util log converter local-state-converter])
    (:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
    (:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
    (:import [java.nio.file Files StandardCopyOption])
    (:import [org.apache.storm Config])
-   (:import [org.apache.storm.generated WorkerResources ProfileAction])
+   (:import [org.apache.storm.generated WorkerResources ProfileAction LocalAssignment])
    (:import [org.apache.storm.localizer LocalResource])
    (:use [org.apache.storm.daemon common])
-   (:require [org.apache.storm.command [healthcheck :as healthcheck]])
+   (:import [org.apache.storm.command HealthCheck])
    (:require [org.apache.storm.daemon [worker :as worker]]
 -            [org.apache.storm [process-simulator :as psim] [cluster :as cluster] [event :as event]]
 +            [org.apache.storm [process-simulator :as psim] [event :as event]]
              [clojure.set :as set])
    (:import [org.apache.thrift.transport TTransportException])
    (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 395be23,8f9becd..a75dc35
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -15,7 -15,7 +15,7 @@@
  ;; limitations under the License.
  (ns org.apache.storm.daemon.worker
    (:use [org.apache.storm.daemon common])
-   (:use [org.apache.storm config log util timer local-state converter])
 -  (:use [org.apache.storm config log util local-state-converter])
++  (:use [org.apache.storm config log util converter local-state-converter])
    (:require [clj-time.core :as time])
    (:require [clj-time.coerce :as coerce])
    (:require [org.apache.storm.daemon [executor :as executor]])
@@@ -238,13 -243,14 +243,14 @@@
    {})
  
  (defn mk-halting-timer [timer-name]
-   (mk-timer :kill-fn (fn [t]
-                        (log-error t "Error when processing event")
-                        (Utils/exitProcess 20 "Error when processing an event")
-                        )
-             :timer-name timer-name))
+   (StormTimer. timer-name
+     (reify Thread$UncaughtExceptionHandler
+       (^void uncaughtException
+         [this ^Thread t ^Throwable e]
+         (log-error e "Error when processing event")
+         (Utils/exitProcess 20 "Error when processing an event")))))
  
 -(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
 +(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
    (let [assignment-versions (atom {})
          executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
          transfer-queue (DisruptorQueue. "worker-transfer-queue"
@@@ -374,15 -380,16 +380,17 @@@
          conf (:conf worker)
          storm-cluster-state (:storm-cluster-state worker)
          storm-id (:storm-id worker)]
-     (fn this
+     (fn refresh-connections
        ([]
-         (this (fn [& ignored] (schedule (:refresh-connections-timer worker) 0 this))))
+         (refresh-connections (fn [& ignored]
+                 (.schedule
+                   (:refresh-connections-timer worker) 0 refresh-connections))))
        ([callback]
 -         (let [version (.assignment-version storm-cluster-state storm-id callback)
 +         (let [version (.assignmentVersion storm-cluster-state storm-id callback)
                 assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
                              (:data (get @(:assignment-versions worker) storm-id))
 -                            (let [new-assignment (.assignment-info-with-version storm-cluster-state storm-id callback)]
 +                            (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
 +                              new-assignment {:data (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA))) :version version}]
                                (swap! (:assignment-versions worker) assoc storm-id new-assignment)
                                (:data new-assignment)))
                my-assignment (-> assignment
@@@ -428,9 -435,12 +436,12 @@@
  
  (defn refresh-storm-active
    ([worker]
-     (refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
+     (refresh-storm-active
+       worker (fn [& ignored]
+                (.schedule
+                  (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
    ([worker callback]
 -    (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)]
 +    (let [base (clojurify-storm-base (.stormBase (:storm-cluster-state worker) (:storm-id worker) callback))]
        (reset!
          (:storm-active-atom worker)
          (and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
@@@ -757,22 -768,28 +769,28 @@@
      (log-message "Started with log levels: " @original-log-levels)
    
      (defn establish-log-setting-callback []
 -      (.topology-log-config (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
 +      (.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
  
      (establish-log-setting-callback)
 -    (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
 +    (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [] (check-credentials-changed))))
-     (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
-                         (fn [& args]
-                           (check-credentials-changed)
-                           (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
-                             (check-throttle-changed))))
+ 
+     (.scheduleRecurring
+       (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
+         (fn []
+           (check-credentials-changed)
+           (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
+             (check-throttle-changed))))
      ;; The jitter allows the clients to get the data at different times, and avoids thundering herd
      (when-not (.get conf TOPOLOGY-DISABLE-LOADAWARE-MESSAGING)
-       (schedule-recurring-with-jitter (:refresh-load-timer worker) 0 1 500 refresh-load))
-     (schedule-recurring (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
-     (schedule-recurring (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS) (fn [] (reset-log-levels latest-log-config)))
-     (schedule-recurring (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
- 
+       (.scheduleRecurringWithJitter
+         (:refresh-load-timer worker) 0 1 500 refresh-load))
+     (.scheduleRecurring
+       (:refresh-connections-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) refresh-connections)
+     (.scheduleRecurring
+       (:reset-log-levels-timer worker) 0 (conf WORKER-LOG-LEVEL-RESET-POLL-SECS)
+         (fn [] (reset-log-levels latest-log-config)))
+     (.scheduleRecurring
+       (:refresh-active-timer worker) 0 (conf TASK-REFRESH-POLL-SECS) (partial refresh-storm-active worker))
      (log-message "Worker has topology config " (Utils/redactValue (:storm-conf worker) STORM-ZOOKEEPER-TOPOLOGY-AUTH-PAYLOAD))
      (log-message "Worker " worker-id " for storm " storm-id " on " assignment-id ":" port " has finished loading")
      ret

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/internal/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/internal/thrift.clj
index 0000000,4ccf8a7..07e0bdf
mode 000000,100644..100644
--- a/storm-core/src/clj/org/apache/storm/internal/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/internal/thrift.clj
@@@ -1,0 -1,96 +1,96 @@@
+ ;; 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.
+ 
+ (ns org.apache.storm.internal.thrift
+   (:import [java.util HashMap]
+            [java.io Serializable]
+            [org.apache.storm.generated NodeInfo Assignment])
+   (:import [org.apache.storm.generated JavaObject Grouping Nimbus StormTopology
+             StormTopology$_Fields Bolt Nimbus$Client Nimbus$Iface
+             ComponentCommon Grouping$_Fields SpoutSpec NullStruct StreamInfo
+             GlobalStreamId ComponentObject ComponentObject$_Fields
+             ShellComponent SupervisorInfo])
+   (:import [org.apache.storm.utils Utils NimbusClient ConfigUtils])
+   (:import [org.apache.storm Constants])
+   (:import [org.apache.storm.security.auth ReqContext])
+   (:import [org.apache.storm.grouping CustomStreamGrouping])
+   (:import [org.apache.storm.topology TopologyBuilder])
+   (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
+   (:import [org.apache.thrift.transport TTransport])
 -  (:use [org.apache.storm util config log zookeeper]))
++  (:use [org.apache.storm util config log]))
+ 
+ ;; Leaving this definition as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (def grouping-constants
+   {Grouping$_Fields/FIELDS :fields
+    Grouping$_Fields/SHUFFLE :shuffle
+    Grouping$_Fields/ALL :all
+    Grouping$_Fields/NONE :none
+    Grouping$_Fields/CUSTOM_SERIALIZED :custom-serialized
+    Grouping$_Fields/CUSTOM_OBJECT :custom-object
+    Grouping$_Fields/DIRECT :direct
+    Grouping$_Fields/LOCAL_OR_SHUFFLE :local-or-shuffle})
+ 
+ ;; Leaving this method as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (defn grouping-type
+   [^Grouping grouping]
+   (grouping-constants (.getSetField grouping)))
+ 
+ (defn nimbus-client-and-conn
+   ([host port]
+     (nimbus-client-and-conn host port nil))
+   ([host port as-user]
+   (log-message "Connecting to Nimbus at " host ":" port " as user: " as-user)
+   (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
+         nimbusClient (NimbusClient. conf host port nil as-user)
+         client (.getClient nimbusClient)
+         transport (.transport nimbusClient)]
+         [client transport] )))
+ 
+ (defmacro with-nimbus-connection
+   [[client-sym host port] & body]
+   `(let [[^Nimbus$Client ~client-sym ^TTransport conn#] (nimbus-client-and-conn ~host ~port)]
+     (try
+       ~@body
+     (finally (.close conn#)))))
+ 
+ (defmacro with-configured-nimbus-connection
+   [client-sym & body]
+   `(let [conf# (clojurify-structure (ConfigUtils/readStormConfig))
+          context# (ReqContext/context)
+          user# (if (.principal context#) (.getName (.principal context#)))
+          nimbusClient# (NimbusClient/getConfiguredClientAs conf# user#)
+          ~client-sym (.getClient nimbusClient#)
+          conn# (.transport nimbusClient#)
+          ]
+      (try
+        ~@body
+      (finally (.close conn#)))))
+ 
+ ;; Leaving this definition as core.clj is using them as a nested keyword argument
+ ;; Must remove once core.clj is ported to java
+ (defn mk-output-spec
+   [output-spec]
+   (let [output-spec (if (map? output-spec)
+                       output-spec
+                       {Utils/DEFAULT_STREAM_ID output-spec})]
+     (map-val
+       (fn [out]
+         (if (instance? StreamInfo out)
+           out
+           (StreamInfo. out false)))
+       output-spec)))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 3dee54b,7817929..8f313cc
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -48,9 -49,10 +49,10 @@@
    (:import [org.apache.storm.task TopologyContext]
             (org.apache.storm.messaging IContext)
             [org.json.simple JSONValue])
 -  (:require [org.apache.storm [zookeeper :as zk]])
 +  (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
    (:require [org.apache.storm.daemon.acker :as acker])
-   (:use [org.apache.storm util thrift config log local-state converter]))
 -  (:use [org.apache.storm cluster util config log local-state-converter])
++  (:use [org.apache.storm util config log local-state-converter converter])
+   (:use [org.apache.storm.internal thrift]))
  
  (defn feeder-spout
    [fields]

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index 1bb8279,6dce7d6..697bdae
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@@ -21,10 -21,11 +21,12 @@@
    (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
              TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
    (:import [org.apache.storm.tuple Fields])
 -  (:use [org.apache.storm testing config util])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl])
-   (:use [org.apache.storm testing config clojure util converter])
+   (:use [org.apache.storm.internal clojure])
++  (:use [org.apache.storm testing config util])
    (:use [org.apache.storm.daemon common])
-   (:require [org.apache.storm [thrift :as thrift]]))
+   (:import [org.apache.storm Thrift])
+   (:import [org.apache.storm.utils Utils]))
  
  (deftest test-basic-topology
    (doseq [zmq-on? [true false]]

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/cluster_test.clj
index 22c1f80,18e3a80..13198fa
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@@ -30,7 -30,8 +30,8 @@@
    (:require [conjure.core])
    (:use [conjure core])
    (:use [clojure test])
-   (:use [org.apache.storm config util testing thrift log converter]))
 -  (:use [org.apache.storm cluster config util testing log])
++  (:use [org.apache.storm config util testing log converter])
+   (:use [org.apache.storm.internal thrift]))
  
  (defn mk-config [zk-port]
    (merge (clojurify-structure (ConfigUtils/readStormConfig))

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
index f75a8e3,7fffd34..6a3d3ca
--- a/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
+++ b/storm-core/test/clj/org/apache/storm/messaging/netty_integration_test.clj
@@@ -1,3 -1,3 +1,4 @@@
++
  ;; 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

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 09c4371,ce58f42..3670fd1
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@@ -20,13 -20,12 +20,15 @@@
    (:require [org.apache.storm [converter :as converter]])
    (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
              TestAggregatesCounter TestPlannerSpout TestPlannerBolt]
-            [org.apache.storm.nimbus InMemoryTopologyActionNotifier])
+            [org.apache.storm.nimbus InMemoryTopologyActionNotifier]
+            [org.apache.storm.generated GlobalStreamId]
+            [org.apache.storm Thrift])
    (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
    (:import [org.apache.storm.scheduler INimbus])
 +  (:import [org.mockito Mockito])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
 +  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
              TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
              InvalidTopologyException AuthorizationException
@@@ -38,12 -37,10 +40,11 @@@
    (:import [org.apache.storm.zookeeper Zookeeper])
    (:import [org.apache.commons.io FileUtils]
             [org.json.simple JSONValue])
 -  (:use [org.apache.storm testing MockAutoCred util config log zookeeper])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
-   (:use [org.apache.storm testing MockAutoCred util config log timer converter])
++  (:use [org.apache.storm testing MockAutoCred util config log converter])
    (:use [org.apache.storm.daemon common])
    (:require [conjure.core])
-   (:require [org.apache.storm
-              [thrift :as thrift]])
 -  (:require [org.apache.storm [cluster :as cluster]])
++
    (:use [conjure core]))
  
  (defn- from-json
@@@ -1081,10 -1145,11 +1149,11 @@@
                              STORM-CLUSTER-MODE "local"
                              STORM-ZOOKEEPER-PORT zk-port
                              STORM-LOCAL-DIR nimbus-dir}))
 -         (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +         (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-          (bind topology (thrift/mk-topology
-                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+          (bind topology (Thrift/buildTopology
+                          {"1" (Thrift/prepareSpoutDetails
+                                 (TestPlannerSpout. true) (Integer. 3))}
                           {}))
           (submit-local-topology nimbus "t1" {} topology)
           (submit-local-topology nimbus "t2" {} topology)
@@@ -1153,10 -1218,11 +1222,11 @@@
                          STORM-CLUSTER-MODE "local"
                          STORM-ZOOKEEPER-PORT zk-port
                          STORM-LOCAL-DIR nimbus-dir}))
 -          (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
            (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-           (bind topology (thrift/mk-topology
-                            {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+           (bind topology (Thrift/buildTopology
+                            {"1" (Thrift/prepareSpoutDetails
+                                   (TestPlannerSpout. true) (Integer. 3))}
                             {}))
  
            (with-open [_ (MockedZookeeper. (proxy [Zookeeper] []
@@@ -1412,17 -1477,17 +1482,16 @@@
        (with-open [_ (ConfigUtilsInstaller. fake-cu)
                    _ (UtilsInstaller. fake-utils)
                    zk-le (MockedZookeeper. (proxy [Zookeeper] []
 -                          (zkLeaderElectorImpl [conf] nil)))]
 +                          (zkLeaderElectorImpl [conf] nil)))
 +                  mocked-cluster (MockedCluster. cluster-utils)]
          (stubbing [mk-authorization-handler nil
 -                   cluster/mk-storm-cluster-state nil
 -                   nimbus/file-cache-map nil
 -                   nimbus/mk-blob-cache-map nil
 -                   nimbus/mk-bloblist-cache-map nil
 -                   nimbus/mk-scheduler nil]
 -                  (nimbus/nimbus-data auth-conf fake-inimbus)
 -                  (verify-call-times-for cluster/mk-storm-cluster-state 1)
 -                  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
 -                                                      expected-acls))))))
 +                 nimbus/file-cache-map nil
 +                 nimbus/mk-blob-cache-map nil
 +                 nimbus/mk-bloblist-cache-map nil
-                  mk-timer nil
 +                 nimbus/mk-scheduler nil]
 +          (nimbus/nimbus-data auth-conf fake-inimbus)
 +          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
 +          )))))
  
  (deftest test-file-bogus-download
    (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@@ -1453,11 -1518,12 +1522,12 @@@
                        STORM-CLUSTER-MODE "local"
                        STORM-ZOOKEEPER-PORT zk-port
                        STORM-LOCAL-DIR nimbus-dir}))
 -        (bind cluster-state (cluster/mk-storm-cluster-state conf))
 +        (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
          (Time/sleepSecs 1)
-         (bind topology (thrift/mk-topology
-                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
+         (bind topology (Thrift/buildTopology
+                          {"1" (Thrift/prepareSpoutDetails
+                                 (TestPlannerSpout. true) (Integer. 3))}
                           {}))
          (submit-local-topology nimbus "t1" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 30} topology)
          ; make transition for topology t1 to be killed -> nimbus applies this event to cluster state

http://git-wip-us.apache.org/repos/asf/storm/blob/cc1f6d77/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index 3ebdbcd,ef40c4a..cdd66e4
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -22,22 -22,21 +22,21 @@@
    (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
    (:import [org.apache.storm.scheduler ISupervisor])
    (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
-   (:import [org.apache.storm.generated RebalanceOptions])
+   (:import [org.apache.storm.generated RebalanceOptions WorkerResources])
 -  (:import [org.mockito Matchers Mockito])
 +  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [java.util UUID])
++  (:import [org.apache.storm Thrift])
 +  (:import [org.mockito Mockito Matchers])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [java.io File])
    (:import [java.nio.file Files])
 -  (:import [org.apache.storm.utils Utils IPredicate]
 -           [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
 -                                                 UtilsInstaller])
 +  (:import [org.apache.storm.utils Utils IPredicate])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
-            [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
-                                                  UtilsInstaller])
++           [org.apache.storm.utils.staticmocking ConfigUtilsInstaller UtilsInstaller])
    (:import [java.nio.file.attribute FileAttribute])
-   (:use [org.apache.storm config testing util timer log converter])
 -  (:import [org.apache.storm Thrift])
 -  (:import [org.apache.storm.utils Utils])
 -  (:use [org.apache.storm config testing util log])
++  (:use [org.apache.storm config testing util log converter])
    (:use [org.apache.storm.daemon common])
--  (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
-             [org.apache.storm [thrift :as thrift]])
 -            [org.apache.storm [cluster :as cluster]])
++  (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]])
    (:use [conjure core])
    (:require [clojure.java.io :as io]))
  
@@@ -770,66 -772,68 +771,68 @@@
              childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
          (is (= expected-childopts childopts-with-ids)))))
  
 -(deftest test-retry-read-assignments
 -  (with-simulated-time-local-cluster [cluster
 -                                      :supervisors 0
 -                                      :ports-per-supervisor 2
 -                                      :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
 -                                                    NIMBUS-MONITOR-FREQ-SECS 10
 -                                                    TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
 -                                                    TOPOLOGY-ACKER-EXECUTORS 0}]
 -    (letlocals
 -     (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
 -     (bind topology1 (Thrift/buildTopology
 -                      {"1" (Thrift/prepareSpoutDetails
 -                             (TestPlannerSpout. true) (Integer. 2))}
 -                      {}))
 -     (bind topology2 (Thrift/buildTopology
 -                      {"1" (Thrift/prepareSpoutDetails
 -                             (TestPlannerSpout. true) (Integer. 2))}
 -                      {}))
 -     (bind state (:storm-cluster-state cluster))
 -     (bind changed (capture-changed-workers
 -                    (submit-mocked-assignment
 -                     (:nimbus cluster)
 -                     (:storm-cluster-state cluster)
 -                     "topology1"
 -                     {TOPOLOGY-WORKERS 2}
 -                     topology1
 -                     {1 "1"
 -                      2 "1"}
 -                     {[1 1] ["sup1" 1]
 -                      [2 2] ["sup1" 2]}
 -                     {["sup1" 1] [0.0 0.0 0.0]
 -                      ["sup1" 2] [0.0 0.0 0.0]
 -                      })
 -                    (submit-mocked-assignment
 -                     (:nimbus cluster)
 -                     (:storm-cluster-state cluster)
 -                     "topology2"
 -                     {TOPOLOGY-WORKERS 2}
 -                     topology2
 -                     {1 "1"
 -                      2 "1"}
 -                     {[1 1] ["sup1" 1]
 -                      [2 2] ["sup1" 2]}
 -                     {["sup1" 1] [0.0 0.0 0.0]
 -                      ["sup1" 2] [0.0 0.0 0.0]
 -                      })
 -                    ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
 -                    (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
 -                    ))
 -     (is (empty? (:launched changed)))
 -     (bind options (RebalanceOptions.))
 -     (.set_wait_secs options 0)
 -     (bind changed (capture-changed-workers
 -                    (.rebalance (:nimbus cluster) "topology2" options)
 -                    (advance-cluster-time cluster 10)
 -                    (heartbeat-workers cluster "sup1" [1 2 3 4])
 -                    (advance-cluster-time cluster 10)
 -                    ))
 -     (validate-launched-once (:launched changed)
 -                             {"sup1" [1 2]}
 -                             (get-storm-id (:storm-cluster-state cluster) "topology1"))
 -     (validate-launched-once (:launched changed)
 -                             {"sup1" [3 4]}
 -                             (get-storm-id (:storm-cluster-state cluster) "topology2"))
 -     )))
 +  (deftest test-retry-read-assignments
 +    (with-simulated-time-local-cluster [cluster
 +                                        :supervisors 0
 +                                        :ports-per-supervisor 2
 +                                        :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
 +                                                      NIMBUS-MONITOR-FREQ-SECS 10
 +                                                      TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
 +                                                      TOPOLOGY-ACKER-EXECUTORS 0}]
 +      (letlocals
 +        (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
-         (bind topology1 (thrift/mk-topology
-                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
++        (bind topology1 (Thrift/buildTopology
++                          {"1" (Thrift/prepareSpoutDetails
++                                 (TestPlannerSpout. true) (Integer. 2))}
 +                          {}))
-         (bind topology2 (thrift/mk-topology
-                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
++        (bind topology2 (Thrift/buildTopology
++                          {"1" (Thrift/prepareSpoutDetails
++                                 (TestPlannerSpout. true) (Integer. 2))}
 +                          {}))
 +        (bind state (:storm-cluster-state cluster))
 +        (bind changed (capture-changed-workers
 +                        (submit-mocked-assignment
 +                          (:nimbus cluster)
 +                          (:storm-cluster-state cluster)
 +                          "topology1"
 +                          {TOPOLOGY-WORKERS 2}
 +                          topology1
 +                          {1 "1"
 +                           2 "1"}
 +                          {[1 1] ["sup1" 1]
 +                           [2 2] ["sup1" 2]}
 +                          {["sup1" 1] [0.0 0.0 0.0]
 +                           ["sup1" 2] [0.0 0.0 0.0]
 +                           })
 +                        (submit-mocked-assignment
 +                          (:nimbus cluster)
 +                          (:storm-cluster-state cluster)
 +                          "topology2"
 +                          {TOPOLOGY-WORKERS 2}
 +                          topology2
 +                          {1 "1"
 +                           2 "1"}
 +                          {[1 1] ["sup1" 1]
 +                           [2 2] ["sup1" 2]}
 +                          {["sup1" 1] [0.0 0.0 0.0]
 +                           ["sup1" 2] [0.0 0.0 0.0]
 +                           })
 +                        ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
 +                        (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
 +                        ))
 +        (is (empty? (:launched changed)))
 +        (bind options (RebalanceOptions.))
 +        (.set_wait_secs options 0)
 +        (bind changed (capture-changed-workers
 +                        (.rebalance (:nimbus cluster) "topology2" options)
 +                        (advance-cluster-time cluster 10)
 +                        (heartbeat-workers cluster "sup1" [1 2 3 4])
 +                        (advance-cluster-time cluster 10)
 +                        ))
 +        (validate-launched-once (:launched changed)
 +          {"sup1" [1 2]}
 +          (get-storm-id (:storm-cluster-state cluster) "topology1"))
 +        (validate-launched-once (:launched changed)
 +          {"sup1" [3 4]}
 +          (get-storm-id (:storm-cluster-state cluster) "topology2"))
 +        )))


[09/27] storm git commit: update class hierarchy about cluster

Posted by bo...@apache.org.
update class hierarchy about cluster


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/55b86ca4
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/55b86ca4
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/55b86ca4

Branch: refs/heads/master
Commit: 55b86ca4f0ea02b25701f25f454e537cbf6239d4
Parents: 2a17c45
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 5 13:47:12 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 5 14:17:58 2016 +0800

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +-
 .../clj/org/apache/storm/command/heartbeats.clj |   4 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |   5 +-
 .../clj/org/apache/storm/daemon/executor.clj    |   6 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  30 +-
 .../clj/org/apache/storm/daemon/supervisor.clj  |  12 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  20 +-
 .../storm/pacemaker/pacemaker_state_factory.clj |  11 +-
 storm-core/src/clj/org/apache/storm/stats.clj   |   2 +-
 storm-core/src/clj/org/apache/storm/testing.clj |   6 +-
 .../jvm/org/apache/storm/callback/Callback.java |  26 -
 .../storm/callback/ZKStateChangedCallback.java  |  25 +
 .../jvm/org/apache/storm/cluster/Cluster.java   | 236 -------
 .../org/apache/storm/cluster/ClusterState.java  | 220 ------
 .../storm/cluster/ClusterStateContext.java      |   2 +-
 .../storm/cluster/ClusterStateFactory.java      |  28 -
 .../org/apache/storm/cluster/ClusterUtils.java  | 249 +++++++
 .../storm/cluster/DistributedClusterState.java  | 268 --------
 .../org/apache/storm/cluster/StateStorage.java  | 220 ++++++
 .../storm/cluster/StateStorageFactory.java      |  28 +
 .../apache/storm/cluster/StormClusterState.java |   2 +-
 .../storm/cluster/StormClusterStateImpl.java    | 664 ++++++++++++++++++
 .../storm/cluster/StormZkClusterState.java      | 683 -------------------
 .../apache/storm/cluster/ZKStateStorage.java    | 271 ++++++++
 .../storm/cluster/ZKStateStorageFactory.java    |  36 +
 .../testing/staticmocking/MockedCluster.java    |   8 +-
 .../org/apache/storm/zookeeper/Zookeeper.java   |  22 +-
 .../org/apache/storm/integration_test.clj       |   4 +-
 .../test/clj/org/apache/storm/cluster_test.clj  | 124 ++--
 .../test/clj/org/apache/storm/nimbus_test.clj   |  26 +-
 .../clj/org/apache/storm/supervisor_test.clj    |   8 +-
 .../test/jvm/org/apache/storm/ClusterTest.java  |  22 -
 32 files changed, 1648 insertions(+), 1622 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/conf/defaults.yaml
----------------------------------------------------------------------
diff --git a/conf/defaults.yaml b/conf/defaults.yaml
index 74605bb..b517b90 100644
--- a/conf/defaults.yaml
+++ b/conf/defaults.yaml
@@ -51,7 +51,7 @@ storm.auth.simple-white-list.users: []
 storm.auth.simple-acl.users: []
 storm.auth.simple-acl.users.commands: []
 storm.auth.simple-acl.admins: []
-storm.cluster.state.store: "org.apache.storm.cluster.StormZkClusterState"
+storm.cluster.state.store: "org.apache.storm.cluster.ZKStateStorageFactory"
 storm.meta.serialization.delegate: "org.apache.storm.serialization.GzipThriftSerializationDelegate"
 storm.codedistributor.class: "org.apache.storm.codedistributor.LocalFileSystemCodeDistributor"
 storm.workers.artifacts.dir: "workers-artifacts"

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index 954042f..af86b69 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -22,12 +22,12 @@
             [clojure.string :as string])
   (:import [org.apache.storm.generated ClusterWorkerHeartbeat]
            [org.apache.storm.utils Utils ConfigUtils]
-           [org.apache.storm.cluster DistributedClusterState ClusterStateContext])
+           [org.apache.storm.cluster ZKStateStorage ClusterStateContext ClusterUtils])
   (:gen-class))
 
 (defn -main [command path & args]
   (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
-        cluster (DistributedClusterState. conf conf nil (ClusterStateContext.))]
+        cluster (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))]
     (println "Command: [" command "]")
     (condp = command
       "list"

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/common.clj b/storm-core/src/clj/org/apache/storm/daemon/common.clj
index c9534f4..b144f40 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@ -13,7 +13,6 @@
 ;; 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.
-;TopologyActionOptions TopologyStatus StormBase RebalanceOptions KillOptions
 (ns org.apache.storm.daemon.common
   (:use [org.apache.storm log config util])
   (:import [org.apache.storm.generated StormTopology NodeInfo
@@ -22,7 +21,7 @@
   (:import [org.apache.storm.utils Utils ConfigUtils])
   (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [org.apache.storm Constants])
-  (:import [org.apache.storm.cluster StormZkClusterState])
+  (:import [org.apache.storm.cluster StormClusterStateImpl])
   (:import [org.apache.storm.metric SystemBolt])
   (:import [org.apache.storm.metric EventLoggerBolt])
   (:import [org.apache.storm.security.auth IAuthorizer]) 
@@ -84,7 +83,7 @@
 
 (defn topology-bases [storm-cluster-state]
   (let [active-topologies (.activeStorms storm-cluster-state)]
-    (into {} 
+    (into {}
           (dofor [id active-topologies]
                  [id  (.stormBase storm-cluster-state id nil)]
                  ))

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/executor.clj b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 7c34c8f..49ae6cf 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@ -34,7 +34,7 @@
   (:import [org.apache.storm.daemon Shutdownable])
   (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
   (:import [org.apache.storm Config Constants])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
   (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
   (:import [java.util.concurrent ConcurrentLinkedQueue])
   (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
@@ -208,7 +208,7 @@
       (when (<= @interval-errors max-per-interval)
         (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
                               (hostname storm-conf)
-                              (.getThisWorkerPort (:worker-context executor)) error)
+          (long (.getThisWorkerPort (:worker-context executor))) error)
         ))))
 
 ;; in its own function so that it can be mocked out by tracked topologies
@@ -251,7 +251,7 @@
      :batch-transfer-queue batch-transfer->worker
      :transfer-fn (mk-executor-transfer-fn batch-transfer->worker storm-conf)
      :suicide-fn (:suicide-fn worker)
-     :storm-cluster-state (StormZkClusterState. (:cluster-state worker) (Utils/getWorkerACL storm-conf)
+     :storm-cluster-state (ClusterUtils/mkStormClusterState (:state-store worker) (Utils/getWorkerACL storm-conf)
                             (ClusterStateContext. DaemonType/WORKER))
      :type executor-type
      ;; TODO: should refactor this to be part of the executor specific map (spout or bolt with :common field)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 9b00df3..daf5e45 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -48,7 +48,7 @@
             ProfileRequest ProfileAction NodeInfo])
   (:import [org.apache.storm.daemon Shutdownable])
   (:import [org.apache.storm.validation ConfigValidation])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
   (:use [org.apache.storm util config log timer local-state converter])
   (:require [org.apache.storm [converter :as converter]
                             [stats :as stats]])
@@ -173,7 +173,7 @@
      :authorization-handler (mk-authorization-handler (conf NIMBUS-AUTHORIZER) conf)
      :impersonation-authorization-handler (mk-authorization-handler (conf NIMBUS-IMPERSONATION-AUTHORIZER) conf)
      :submitted-count (atom 0)
-     :storm-cluster-state (StormZkClusterState. conf  (when
+     :storm-cluster-state (ClusterUtils/mkStormClusterState conf  (when
                                                                        (Utils/isZkAuthenticationConfiguredStormServer
                                                                          conf)
                                                                        NIMBUS-ZK-ACLS)
@@ -586,11 +586,11 @@
 (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
   (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
   (let [storm-cluster-state (:storm-cluster-state nimbus)
-        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))]
-                         (->> (clojurify-structure executor-stats-java-map)
-                           (map (fn [^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat]
-                                  {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)}))
-                         (into {})))
+        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
+                             executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
+                         (->> (dofor [[^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat] executor-stats-clojurify]
+                             {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)})
+                           (apply merge)))
 
         cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
                                       executor-beats
@@ -1332,6 +1332,14 @@
        (InvalidTopologyException.
         (str "Failed to submit topology. Topology requests more than " workers-allowed " workers."))))))
 
+(defn nimbus-topology-bases [storm-cluster-state]
+  (let [active-topologies (.activeStorms storm-cluster-state)]
+    (into {}
+      (dofor [id active-topologies]
+        [id  (clojurify-storm-base (.stormBase storm-cluster-state id nil))]
+        ))
+    ))
+
 (defn- set-logger-timeouts [log-config]
   (let [timeout-secs (.get_reset_log_level_timeout_secs log-config)
        timeout (time/plus (time/now) (time/secs timeout-secs))]
@@ -1617,7 +1625,7 @@
           (log-message "Nimbus setting debug to " enable? " for storm-name '" storm-name "' storm-id '" storm-id "' sampling pct '" spct "'"
             (if (not (clojure.string/blank? component-id)) (str " component-id '" component-id "'")))
           (locking (:submit-lock nimbus)
-            (.updateStorm storm-cluster-state (thriftify-storm-base storm-id storm-base-updates)))))
+            (.updateStorm storm-cluster-state storm-id  (thriftify-storm-base storm-base-updates)))))
 
       (^void setWorkerProfiler
         [this ^String id ^ProfileRequest profileRequest]
@@ -1804,8 +1812,7 @@
                                        (when-let [version (:version info)] (.set_version sup-sum version))
                                        sup-sum))
               nimbus-uptime ((:uptime nimbus))
-              javabases (topology-bases storm-cluster-state)
-              bases (into {} (dofor [[id base] javabases][id (clojurify-storm-base base)]))
+              bases (nimbus-topology-bases storm-cluster-state)
               nimbuses (.nimbuses storm-cluster-state)
 
               ;;update the isLeader field for each nimbus summary
@@ -2162,8 +2169,7 @@
 
       (^TopologyHistoryInfo getTopologyHistory [this ^String user]
         (let [storm-cluster-state (:storm-cluster-state nimbus)
-              javabases (topology-bases storm-cluster-state)
-              bases (into {} (dofor [[id  base] javabases][id (clojurify-storm-base base)]))
+              bases (topology-bases storm-cluster-state)
               assigned-topology-ids (.assignments storm-cluster-state nil)
               user-group-match-fn (fn [topo-id user conf]
                                     (let [topology-conf (try-read-storm-conf conf topo-id (:blob-store nimbus))

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 079b221..3a83d03 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -19,7 +19,7 @@
            [org.apache.storm.utils LocalState Time Utils ConfigUtils]
            [org.apache.storm.daemon Shutdownable]
            [org.apache.storm Constants]
-           [org.apache.storm.cluster ClusterStateContext DaemonType StormZkClusterState Cluster]
+           [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
            [java.net JarURLConnection]
            [java.net URI]
            [org.apache.commons.io FileUtils])
@@ -66,7 +66,9 @@
                     (if-let [assignment-version (.assignmentVersion storm-cluster-state sid callback)]
                       (if (= assignment-version recorded-version)
                         {sid (get assignment-versions sid)}
-                        {sid (.assignmentInfoWithVersion storm-cluster-state sid callback)})
+                        (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state sid callback)
+                              assignment (clojurify-assignment (:data thriftify-assignment-version))]
+                        {sid {:data assignment :version (:version thriftify-assignment-version)}}))
                       {sid nil})))
            (apply merge)
            (filter-val not-nil?))
@@ -77,8 +79,7 @@
                    (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
                       {sid topo-profile-actions}))
            (apply merge))]
-
-      {:assignments (into {} (for [[k v] new-assignments] [k (clojurify-assignment (:data v))]))
+      {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
        :profiler-actions new-profiler-actions
        :versions new-assignments})))
 
@@ -317,7 +318,7 @@
    :uptime (uptime-computer)
    :version STORM-VERSION
    :worker-thread-pids-atom (atom {})
-   :storm-cluster-state (Cluster/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
+   :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
                                                      SUPERVISOR-ZK-ACLS)
                                                         (ClusterStateContext. DaemonType/SUPERVISOR))
    :local-state (ConfigUtils/supervisorState conf)
@@ -536,6 +537,7 @@
            storm-id->profiler-actions :profiler-actions
            versions :versions}
           (assignments-snapshot storm-cluster-state sync-callback assignment-versions)
+
           storm-code-map (read-storm-code-locations assignments-snapshot)
           all-downloaded-storm-ids (set (read-downloaded-storm-ids conf))
           existing-assignment (ls-local-assignments local-state)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 85ed37d..a793009 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -36,7 +36,7 @@
   (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [org.apache.storm Constants])
   (:import [org.apache.storm.security.auth AuthUtils])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType DistributedClusterState StormZkClusterState])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils])
   (:import [javax.security.auth Subject])
   (:import [java.security PrivilegedExceptionAction])
   (:import [org.apache.logging.log4j LogManager])
@@ -73,7 +73,7 @@
                }]
     ;; do the zookeeper heartbeat
     (try
-      (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) (thriftify-zk-worker-hb zk-hb))
+      (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (long (:port worker)) (thriftify-zk-worker-hb zk-hb))
       (catch Exception exc
         (log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry")))))
 
@@ -241,7 +241,7 @@
                        )
             :timer-name timer-name))
 
-(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
+(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
   (let [assignment-versions (atom {})
         executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
@@ -267,7 +267,7 @@
       :assignment-id assignment-id
       :port port
       :worker-id worker-id
-      :cluster-state cluster-state
+      :state-store state-store
       :storm-cluster-state storm-cluster-state
       ;; when worker bootup, worker will start to setup initial connections to
       ;; other workers. When all connection is ready, we will enable this flag
@@ -596,14 +596,14 @@
   (let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
         storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
         acls (Utils/getWorkerACL storm-conf)
-        cluster-state (DistributedClusterState. conf storm-conf  acls  (ClusterStateContext. DaemonType/WORKER))
-        storm-cluster-state (StormZkClusterState. cluster-state acls (ClusterStateContext.))
+        state-store (ClusterUtils/mkDistributedClusterState conf storm-conf  acls  (ClusterStateContext. DaemonType/WORKER))
+        storm-cluster-state (ClusterUtils/mkStormClusterState state-store acls (ClusterStateContext.))
         initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
         auto-creds (AuthUtils/GetAutoCredentials storm-conf)
         subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
       (Subject/doAs subject (reify PrivilegedExceptionAction
         (run [this]
-          (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state)
+          (let [worker (worker-data conf shared-mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state)
         heartbeat-fn #(do-heartbeat worker)
 
         ;; do this here so that the worker process dies if this fails
@@ -686,10 +686,10 @@
                     (log-message "Trigger any worker shutdown hooks")
                     (run-worker-shutdown-hooks worker)
 
-                    (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id port)
+                    (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id (long port))
                     (log-message "Disconnecting from storm cluster state context")
                     (.disconnect (:storm-cluster-state worker))
-                    (.close (:cluster-state worker))
+                    (.close (:state-store worker))
                     (log-message "Shut down worker " storm-id " " assignment-id " " port))
         ret (reify
              Shutdownable
@@ -732,7 +732,7 @@
       (.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
 
     (establish-log-setting-callback)
-    (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed))))
+    (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [] (check-credentials-changed))))
     (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
                         (fn [& args]
                           (check-credentials-changed)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index b367b4b..28f792d 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -23,16 +23,17 @@
   (:import [org.apache.storm.generated
             HBExecutionException HBServerMessageType HBMessage
             HBMessageData HBPulse]
-           [org.apache.storm.cluster ClusterState DistributedClusterState]
+           [org.apache.storm.cluster ZKStateStorage StateStorage ClusterUtils]
            [org.apache.storm.pacemaker PacemakerClient])
-  (:gen-class))
+  (:gen-class
+    :implements [org.apache.storm.cluster.StateStorageFactory]))
 
 ;; So we can mock the client for testing
 (defn makeClient [conf]
   (PacemakerClient. conf))
 
 (defn makeZKState [conf auth-conf acls context]
-  (DistributedClusterState. conf auth-conf acls context))
+  (ClusterUtils/mkDistributedClusterState conf auth-conf acls context))
 
 (def max-retries 10)
 
@@ -41,9 +42,9 @@
         pacemaker-client (makeClient conf)]
 
     (reify
-      ClusterState
+      StateStorage
       ;; Let these pass through to the zk-state. We only want to handle heartbeats.
-      (register [this callback] (.register zk-state callback))  ; need update callback, have questions?? callback is IFn here
+      (register [this callback] (.register zk-state callback))
       (unregister [this callback] (.unregister zk-state callback))
       (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
       (create_sequential [this path data acls] (.create_sequential zk-state path data acls))

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index d6bcdc3..0bf1757 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -24,7 +24,7 @@
             ExecutorAggregateStats SpecificAggregateStats
             SpoutAggregateStats TopologyPageInfo TopologyStats])
   (:import [org.apache.storm.utils Utils])
-  (:import [org.apache.storm.cluster StormZkClusterState])
+  (:import [org.apache.storm.cluster StormClusterStateImpl])
   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
   (:use [org.apache.storm log util])
   (:use [clojure.math.numeric-tower :only [ceil]]))

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index eb34d36..470a14f 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -45,7 +45,7 @@
   (:import [org.apache.storm.tuple Tuple])
   (:import [org.apache.storm.generated StormTopology])
   (:import [org.apache.storm.task TopologyContext])
-  (:import [org.apache.storm.cluster DistributedClusterState ClusterStateContext StormZkClusterState])
+  (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
   (:require [org.apache.storm.messaging.loader :as msg-loader])
   (:require [org.apache.storm.daemon.acker :as acker])
   (:use [org.apache.storm util thrift config log local-state converter]))
@@ -158,8 +158,8 @@
                      :port-counter port-counter
                      :daemon-conf daemon-conf
                      :supervisors (atom [])
-                     :state (DistributedClusterState. daemon-conf nil nil (ClusterStateContext.))
-                     :storm-cluster-state (StormZkClusterState. daemon-conf nil (ClusterStateContext.))
+                     :state (ClusterUtils/mkDistributedClusterState daemon-conf nil nil (ClusterStateContext.))
+                     :storm-cluster-state (ClusterUtils/mkStormClusterState daemon-conf nil (ClusterStateContext.))
                      :tmp-dirs (atom [nimbus-tmp zk-tmp])
                      :zookeeper (if (not-nil? zk-handle) zk-handle)
                      :shared-context context

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/callback/Callback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/Callback.java b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
deleted file mode 100644
index a37612d..0000000
--- a/storm-core/src/jvm/org/apache/storm/callback/Callback.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/**
- * 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.callback;
-
-import clojure.lang.IFn;
-
-// To remove IFn after porting all callbacks to java
-public interface Callback {
-    public <T> Object execute(T... args);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
new file mode 100644
index 0000000..75b0e99
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/callback/ZKStateChangedCallback.java
@@ -0,0 +1,25 @@
+/**
+ * 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.callback;
+
+import org.apache.zookeeper.Watcher;
+
+public interface ZKStateChangedCallback {
+    public void changed(Watcher.Event.EventType type, String path);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
deleted file mode 100644
index 851858f..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/**
- * 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 org.apache.storm.Config;
-import org.apache.storm.generated.ClusterWorkerHeartbeat;
-import org.apache.storm.generated.ExecutorInfo;
-import org.apache.storm.generated.ExecutorStats;
-import org.apache.storm.generated.ProfileAction;
-import org.apache.storm.utils.Utils;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
-import org.apache.zookeeper.data.Id;
-import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
-
-
-import java.io.UnsupportedEncodingException;
-import java.net.URLEncoder;
-import java.security.NoSuchAlgorithmException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class Cluster {
-
-    public static final String ZK_SEPERATOR = "/";
-
-    public static final String ASSIGNMENTS_ROOT = "assignments";
-    public static final String CODE_ROOT = "code";
-    public static final String STORMS_ROOT = "storms";
-    public static final String SUPERVISORS_ROOT = "supervisors";
-    public static final String WORKERBEATS_ROOT = "workerbeats";
-    public static final String BACKPRESSURE_ROOT = "backpressure";
-    public static final String ERRORS_ROOT = "errors";
-    public static final String BLOBSTORE_ROOT = "blobstore";
-    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
-    public static final String NIMBUSES_ROOT = "nimbuses";
-    public static final String CREDENTIALS_ROOT = "credentials";
-    public static final String LOGCONFIG_ROOT = "logconfigs";
-    public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
-
-    public static final String ASSIGNMENTS_SUBTREE;
-    public static final String STORMS_SUBTREE;
-    public static final String SUPERVISORS_SUBTREE;
-    public static final String WORKERBEATS_SUBTREE;
-    public static final String BACKPRESSURE_SUBTREE;
-    public static final String ERRORS_SUBTREE;
-    public static final String BLOBSTORE_SUBTREE;
-    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE;
-    public static final String NIMBUSES_SUBTREE;
-    public static final String CREDENTIALS_SUBTREE;
-    public static final String LOGCONFIG_SUBTREE;
-    public static final String PROFILERCONFIG_SUBTREE;
-
-    static {
-        ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
-        STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
-        SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
-        WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
-        BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
-        ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
-        BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
-        BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
-        NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
-        CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
-        LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
-        PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
-    }
-
-    // A singleton instance allows us to mock delegated static methods in our
-    // tests by subclassing.
-    private static final Cluster INSTANCE = new Cluster();
-    private static Cluster _instance = INSTANCE;
-
-    /**
-     * Provide an instance of this class for delegates to use.  To mock out
-     * delegated methods, provide an instance of a subclass that overrides the
-     * implementation of the delegated method.
-     *
-     * @param u a Zookeeper instance
-     */
-    public static void setInstance(Cluster u) {
-        _instance = u;
-    }
-
-    /**
-     * Resets the singleton instance to the default. This is helpful to reset
-     * the class to its original functionality when mocking is no longer
-     * desired.
-     */
-    public static void resetInstance() {
-        _instance = INSTANCE;
-    }
-
-    public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
-        List<ACL> aclList = null;
-        String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
-        if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
-            aclList = new ArrayList<>();
-            ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
-            aclList.add(acl1);
-            ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
-            aclList.add(acl2);
-        }
-        return aclList;
-    }
-
-    public static String supervisorPath(String id) {
-        return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
-    }
-
-    public static String assignmentPath(String id) {
-        return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
-    }
-
-    public static String blobstorePath(String key) {
-        return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
-    }
-
-    public static String blobstoreMaxKeySequenceNumberPath(String key) {
-        return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
-    }
-
-    public static String nimbusPath(String id) {
-        return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
-    }
-
-    public static String stormPath(String id) {
-        return STORMS_SUBTREE + ZK_SEPERATOR + id;
-    }
-
-    public static String workerbeatStormRoot(String stormId) {
-        return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String workerbeatPath(String stormId, String node, Long port) {
-        return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
-    }
-
-    public static String backpressureStormRoot(String stormId) {
-        return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String backpressurePath(String stormId, String node, Long port) {
-        return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
-    }
-
-    public static String errorStormRoot(String stormId) {
-        return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String errorPath(String stormId, String componentId) throws UnsupportedEncodingException {
-        return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
-    }
-
-    public static String lastErrorPath(String stormId, String componentId) throws UnsupportedEncodingException {
-        return errorPath(stormId, componentId) + "-last-error";
-    }
-
-    public static String credentialsPath(String stormId) {
-        return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String logConfigPath(String stormId) {
-        return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String profilerConfigPath(String stormId) {
-        return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
-    }
-
-    public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
-        return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
-    }
-
-    public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz){
-        if (serialized != null){
-            return Utils.deserialize(serialized, clazz);
-        }
-        return null;
-    }
-
-    //Ensures that we only return heartbeats for executors assigned to this worker
-    public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat){
-        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
-        Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
-        for (ExecutorInfo executor : executors){
-            if(executorStatsMap.containsKey(executor)){
-                executorWhb.put(executor, workerHeartbeat);
-            }
-        }
-        return executorWhb;
-    }
-
-    public  StormClusterState mkStormClusterStateImpl(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
-        return new StormZkClusterState(clusterState, acls, context);
-    }
-    public static StormClusterState mkStormClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
-        return _instance.mkStormClusterStateImpl(clusterState, acls, context);
-    }
-    
-    // TO be remove
-    public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
-        HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
-        if (map == null) {
-            return rtn;
-        }
-        for (Map.Entry<K, V> entry : map.entrySet()) {
-            K key = entry.getKey();
-            V val = entry.getValue();
-            List<K> list = rtn.get(val);
-            if (list == null) {
-                list = new ArrayList<K>();
-                rtn.put(entry.getValue(), list);
-            }
-            list.add(key);
-        }
-        return rtn;
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
deleted file mode 100644
index e76721b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/**
- * 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.APersistentMap;
-import clojure.lang.IFn;
-import java.util.List;
-
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.storm.callback.Callback;
-import org.apache.zookeeper.data.ACL;
-
-/**
- * ClusterState provides the API for the pluggable state store used by the
- * Storm daemons. Data is stored in path/value format, and the store supports
- * listing sub-paths at a given path.
- * All data should be available across all nodes with eventual consistency.
- *
- * IMPORTANT NOTE:
- * Heartbeats have different api calls used to interact with them. The root
- * path (/) may or may not be the same as the root path for the other api calls.
- *
- * For example, performing these two calls:
- *     set_data("/path", data, acls);
- *     void set_worker_hb("/path", heartbeat, acls);
- * may or may not cause a collision in "/path".
- * Never use the same paths with the *_hb* methods as you do with the others.
- */
-public interface ClusterState {
-
-    /**
-     * Registers a callback function that gets called when CuratorEvents happen.
-     * @param callback is a clojure IFn that accepts the type - translated to
-     * clojure keyword as in zookeeper - and the path: (callback type path)
-     * @return is an id that can be passed to unregister(...) to unregister the
-     * callback.
-     */
-    String register(Callback callback);
-
-    /**
-     * Unregisters a callback function that was registered with register(...).
-     * @param id is the String id that was returned from register(...).
-     */
-    void unregister(String id);
-
-    /**
-     * Path will be appended with a monotonically increasing integer, a new node
-     * will be created there, and data will be put at that node.
-     * @param path The path that the monotonically increasing integer suffix will
-     * be added to.
-     * @param data The data that will be written at the suffixed path's node.
-     * @param acls The acls to apply to the path. May be null.
-     * @return The path with the integer suffix appended.
-     */
-    String create_sequential(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Creates nodes for path and all its parents. Path elements are separated by
-     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
-     * @param path The path to create, along with all its parents.
-     * @param acls The acls to apply to the path. May be null.
-     * @return path
-     */
-    void mkdirs(String path, List<ACL> acls);
-
-    /**
-     * Deletes the node at a given path, and any child nodes that may exist.
-     * @param path The path to delete
-     */
-    void delete_node(String path);
-
-    /**
-     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
-     * by the store when the client disconnects.
-     * @param path The path where a node will be created.
-     * @param data The data to be written at the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Gets the 'version' of the node at a path. Optionally sets a watch
-     * on that node. The version should increase whenever a write happens.
-     * @param path The path to get the version of.
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The integer version of this node.
-     */
-    Integer get_version(String path, boolean watch) throws Exception;
-
-    /**
-     * Check if a node exists and optionally set a watch on the path.
-     * @param path The path to check for the existence of a node.
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return Whether or not a node exists at path.
-     */
-    boolean node_exists(String path, boolean watch);
-
-    /**
-     * Get a list of paths of all the child nodes which exist immediately
-     * under path.
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return list of string paths under path.
-     */
-    List<String> get_children(String path, boolean watch);
-
-    /**
-     * Close the connection to the data store.
-     */
-    void close();
-
-    /**
-     * Set the value of the node at path to data.
-     * @param path The path whose node we want to set.
-     * @param data The data to put in the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_data(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Get the data from the node at path
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The data at the node.
-     */
-    byte[] get_data(String path, boolean watch);
-
-    /**
-     * Get the data at the node along with its version. Data is returned
-     * in an APersistentMap with clojure keyword keys :data and :version.
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return An APersistentMap in the form {:data data :version version}
-     */
-    APersistentMap get_data_with_version(String path, boolean watch);
-
-    /**
-     * Write a worker heartbeat at the path.
-     * @param path The path whose node we want to set.
-     * @param data The data to put in the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_worker_hb(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Get the heartbeat from the node at path
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The heartbeat at the node.
-     */
-    byte[] get_worker_hb(String path, boolean watch);
-
-    /**
-     * Get a list of paths of all the child nodes which exist immediately
-     * under path. This is similar to get_children, but must be used for
-     * any nodes
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return list of string paths under path.
-     */
-    List<String> get_worker_hb_children(String path, boolean watch);
-
-    /**
-     * Deletes the heartbeat at a given path, and any child nodes that may exist.
-     * @param path The path to delete.
-     */
-    void delete_worker_hb(String path);
-
-    /**
-     * Add a ClusterStateListener to the connection.
-     * @param listener A ClusterStateListener to handle changing cluster state
-     * events.
-     */
-    void add_listener(final ConnectionStateListener listener);
-
-    /**
-     * Force consistency on a path. Any writes committed on the path before
-     * this call will be completely propagated when it returns.
-     * @param path The path to synchronize.
-     */
-    void sync_path(String path);
-
-    /**
-     * Allows us to delete the znodes within /storm/blobstore/key_name
-     * whose znodes start with the corresponding nimbusHostPortInfo
-     * @param path /storm/blobstore/key_name
-     * @param nimbusHostPortInfo Contains the host port information of
-     * a nimbus node.
-     */
-    void delete_node_blobstore(String path, String nimbusHostPortInfo);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
index 997bdc3..9ad6a92 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateContext.java
@@ -19,7 +19,7 @@
 package org.apache.storm.cluster;
 
 /**
- * This class is intended to provide runtime-context to ClusterStateFactory
+ * This class is intended to provide runtime-context to StateStorageFactory
  * implementors, giving information such as what daemon is creating it.
  */
 public class ClusterStateContext {

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
deleted file mode 100644
index 6474d82..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterStateFactory.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/**
- * 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.APersistentMap;
-import java.util.List;
-import org.apache.zookeeper.data.ACL;
-
-public interface ClusterStateFactory {
-    
-    ClusterState mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
new file mode 100644
index 0000000..9fd36ca
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -0,0 +1,249 @@
+/**
+ * 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.APersistentMap;
+import org.apache.storm.Config;
+import org.apache.storm.generated.ClusterWorkerHeartbeat;
+import org.apache.storm.generated.ExecutorInfo;
+import org.apache.storm.generated.ExecutorStats;
+import org.apache.storm.generated.ProfileAction;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+import org.apache.zookeeper.data.Id;
+import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URLEncoder;
+import java.security.NoSuchAlgorithmException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class ClusterUtils {
+
+    public static final String ZK_SEPERATOR = "/";
+
+    public static final String ASSIGNMENTS_ROOT = "assignments";
+    public static final String CODE_ROOT = "code";
+    public static final String STORMS_ROOT = "storms";
+    public static final String SUPERVISORS_ROOT = "supervisors";
+    public static final String WORKERBEATS_ROOT = "workerbeats";
+    public static final String BACKPRESSURE_ROOT = "backpressure";
+    public static final String ERRORS_ROOT = "errors";
+    public static final String BLOBSTORE_ROOT = "blobstore";
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
+    public static final String NIMBUSES_ROOT = "nimbuses";
+    public static final String CREDENTIALS_ROOT = "credentials";
+    public static final String LOGCONFIG_ROOT = "logconfigs";
+    public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
+
+    public static final String ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
+    public static final String STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
+    public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
+    public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
+    public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
+    public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
+    public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
+    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
+    public static final String NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
+    public static final String CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
+    public static final String LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
+    public static final String PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
+
+    // A singleton instance allows us to mock delegated static methods in our
+    // tests by subclassing.
+    private static final ClusterUtils INSTANCE = new ClusterUtils();
+    private static ClusterUtils _instance = INSTANCE;
+
+    /**
+     * Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that overrides the
+     * implementation of the delegated method.
+     *
+     * @param u a Cluster instance
+     */
+    public static void setInstance(ClusterUtils u) {
+        _instance = u;
+    }
+
+    /**
+     * Resets the singleton instance to the default. This is helpful to reset the class to its original functionality when mocking is no longer desired.
+     */
+    public static void resetInstance() {
+        _instance = INSTANCE;
+    }
+
+    public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = null;
+        String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
+        if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
+            aclList = new ArrayList<>();
+            ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
+            aclList.add(acl1);
+            ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
+            aclList.add(acl2);
+        }
+        return aclList;
+    }
+
+    public static String supervisorPath(String id) {
+        return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String assignmentPath(String id) {
+        return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String blobstorePath(String key) {
+        return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String blobstoreMaxKeySequenceNumberPath(String key) {
+        return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
+    }
+
+    public static String nimbusPath(String id) {
+        return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String stormPath(String id) {
+        return STORMS_SUBTREE + ZK_SEPERATOR + id;
+    }
+
+    public static String workerbeatStormRoot(String stormId) {
+        return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String workerbeatPath(String stormId, String node, Long port) {
+        return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String backpressureStormRoot(String stormId) {
+        return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String backpressurePath(String stormId, String node, Long port) {
+        return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
+    }
+
+    public static String errorStormRoot(String stormId) {
+        return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String errorPath(String stormId, String componentId) {
+        try {
+            return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
+        } catch (UnsupportedEncodingException e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public static String lastErrorPath(String stormId, String componentId) {
+        return errorPath(stormId, componentId) + "-last-error";
+    }
+
+    public static String credentialsPath(String stormId) {
+        return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String logConfigPath(String stormId) {
+        return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId) {
+        return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
+    }
+
+    public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
+        return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
+    }
+
+    public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz) {
+        if (serialized != null) {
+            return Utils.deserialize(serialized, clazz);
+        }
+        return null;
+    }
+
+    // Ensures that we only return heartbeats for executors assigned to this worker
+    public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
+        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
+        Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
+        for (ExecutorInfo executor : executors) {
+            if (executorStatsMap.containsKey(executor)) {
+                executorWhb.put(executor, workerHeartbeat);
+            }
+        }
+        return executorWhb;
+    }
+
+    public StormClusterState mkStormClusterStateImpl(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+        if (StateStorage instanceof StateStorage) {
+            return new StormClusterStateImpl((StateStorage) StateStorage, acls, context, false);
+        } else {
+            StateStorage Storage = _instance.mkDistributedClusterStateImpl((APersistentMap) StateStorage, (APersistentMap) StateStorage, acls, context);
+            return new StormClusterStateImpl(Storage, acls, context, true);
+        }
+
+    }
+
+    public StateStorage mkDistributedClusterStateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+            throws Exception {
+        String className = null;
+        StateStorage stateStorage = null;
+        if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
+            className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
+        } else {
+            className = "org.apache.storm.cluster.ZKStateStorageFactory";
+        }
+        Class clazz = Class.forName(className);
+        StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
+        stateStorage = storageFactory.mkState(config, auth_conf, acls, context);
+        return stateStorage;
+    }
+
+    public static StateStorage mkDistributedClusterState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+            throws Exception {
+        return _instance.mkDistributedClusterStateImpl(config, auth_conf, acls, context);
+    }
+
+    public static StormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+        return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
+    }
+
+    // TO be remove
+    public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
+        HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
+        if (map == null) {
+            return rtn;
+        }
+        for (Map.Entry<K, V> entry : map.entrySet()) {
+            K key = entry.getKey();
+            V val = entry.getValue();
+            List<K> list = rtn.get(val);
+            if (list == null) {
+                list = new ArrayList<K>();
+                rtn.put(entry.getValue(), list);
+            }
+            list.add(key);
+        }
+        return rtn;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
deleted file mode 100644
index 1bd534e..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/**
- * 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.APersistentMap;
-import org.apache.curator.framework.CuratorFramework;
-import org.apache.curator.framework.state.*;
-import org.apache.curator.framework.state.ConnectionState;
-import org.apache.storm.Config;
-import org.apache.storm.callback.Callback;
-import org.apache.storm.callback.WatcherCallBack;
-import org.apache.storm.utils.Utils;
-import org.apache.storm.zookeeper.Zookeeper;
-import org.apache.zookeeper.CreateMode;
-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.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-public class DistributedClusterState implements ClusterState {
-
-    private static Logger LOG = LoggerFactory.getLogger(DistributedClusterState.class);
-
-    private ConcurrentHashMap<String, Callback> callbacks = new ConcurrentHashMap<String, Callback>();
-    private CuratorFramework zkWriter;
-    private CuratorFramework zkReader;
-    private AtomicBoolean active;
-
-    private boolean isNimbus;
-    private Map authConf;
-    private Map<Object, Object> conf;
-
-    public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
-        this.conf = conf;
-        this.authConf = authConf;
-        if (context.getDaemonType().equals(DaemonType.NIMBUS)) this.isNimbus = true;
-
-        // just mkdir STORM_ZOOKEEPER_ROOT dir
-        CuratorFramework zkTemp = mkZk();
-        String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
-        Zookeeper.mkdirs(zkTemp, rootPath, acls);
-        zkTemp.close();
-
-        active = new AtomicBoolean(true);
-        zkWriter = mkZk(new WatcherCallBack() {
-            @Override
-            public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
-                if (active.get()) {
-                    if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
-                        LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
-                    } else {
-                        LOG.info("Received event {} : {} : {}", state, type, path);
-                    }
-
-                    if (!type.equals(Watcher.Event.EventType.None)) {
-                        for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
-                            Callback fn = e.getValue();
-                            fn.execute(type, path);
-                        }
-                    }
-                }
-            }
-        });
-        if (isNimbus) {
-            zkReader = mkZk(new WatcherCallBack() {
-                @Override
-                public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
-                    if (active.get()) {
-                        if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
-                            LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
-                        } else {
-                            LOG.info("Received event {} : {} : {}", state, type, path);
-                        }
-
-                        if (!type.equals(Watcher.Event.EventType.None)) {
-                            for (Map.Entry<String, Callback> e : callbacks.entrySet()) {
-                                Callback fn = e.getValue();
-                                fn.execute(type, path);
-                            }
-                        }
-                    }
-                }
-            });
-        } else {
-            zkReader = zkWriter;
-        }
-
-    }
-
-    @SuppressWarnings("unchecked")
-    private CuratorFramework mkZk() throws IOException {
-        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "", authConf);
-    }
-
-    @SuppressWarnings("unchecked")
-    private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
-        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
-                String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
-    }
-
-    @Override
-    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
-
-    }
-
-    @Override
-    public String register( Callback callback) {
-        String id = UUID.randomUUID().toString();
-        this.callbacks.put(id,callback);
-        return id;
-    }
-
-    @Override
-    public void unregister(String id) {
-        this.callbacks.remove(id);
-    }
-
-    @Override
-    public String create_sequential(String path, byte[] data, List<ACL> acls) {
-        return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
-    }
-
-    @Override
-    public void mkdirs(String path, List<ACL> acls) {
-        Zookeeper.mkdirs(zkWriter, path, acls);
-    }
-
-    @Override
-    public void delete_node(String path) {
-        Zookeeper.deleteNode(zkWriter, path);
-    }
-
-    @Override
-    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
-        Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
-        if (Zookeeper.exists(zkWriter, path, false)) {
-            try {
-                Zookeeper.setData(zkWriter, path, data);
-            } catch (RuntimeException e) {
-                if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
-                    Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
-                } else {
-                    throw e;
-                }
-            }
-
-        } else {
-            Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
-        }
-    }
-
-    @Override
-    public Integer get_version(String path, boolean watch) throws Exception {
-        Integer ret = Zookeeper.getVersion(zkReader, path, watch);
-        return ret;
-    }
-
-    @Override
-    public boolean node_exists(String path, boolean watch) {
-        return Zookeeper.existsNode(zkWriter, path, watch);
-    }
-
-    @Override
-    public List<String> get_children(String path, boolean watch) {
-        return Zookeeper.getChildren(zkReader, path, watch);
-    }
-
-    @Override
-    public void close() {
-        this.active.set(false);
-        zkWriter.close();
-        if (isNimbus) {
-            zkReader.close();
-        }
-    }
-
-    @Override
-    public void set_data(String path, byte[] data, List<ACL> acls) {
-        if (Zookeeper.exists(zkWriter, path, false)) {
-            Zookeeper.setData(zkWriter, path, data);
-        } else {
-            Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
-            Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
-        }
-    }
-
-    @Override
-    public byte[] get_data(String path, boolean watch) {
-        byte[] ret = null;
-
-        ret = Zookeeper.getData(zkReader, path, watch);
-
-        return ret;
-    }
-
-    @Override
-    public APersistentMap get_data_with_version(String path, boolean watch) {
-        return Zookeeper.getDataWithVersion(zkReader, path, watch);
-    }
-
-    @Override
-    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
-        set_data(path, data, acls);
-    }
-
-    @Override
-    public byte[] get_worker_hb(String path, boolean watch) {
-        return Zookeeper.getData(zkReader, path, watch);
-    }
-
-    @Override
-    public List<String> get_worker_hb_children(String path, boolean watch) {
-        return get_children(path, watch);
-    }
-
-    @Override
-    public void delete_worker_hb(String path) {
-        delete_node(path);
-    }
-
-    @Override
-    public void add_listener(final ConnectionStateListener listener) {
-        Zookeeper.addListener(zkReader, new ConnectionStateListener() {
-            @Override
-            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
-                listener.stateChanged(curatorFramework, connectionState);
-            }
-        });
-    }
-
-    @Override
-    public void sync_path(String path) {
-        Zookeeper.syncPath(zkWriter, path);
-    }
-
-    // To be remove when finished port Util.clj
-    public static String parentPath(String path) {
-        List<String> toks = Zookeeper.tokenizePath(path);
-        int size = toks.size();
-        if (size > 0) {
-            toks.remove(size - 1);
-        }
-        return Zookeeper.toksToPath(toks);
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
new file mode 100644
index 0000000..8895cd1
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
@@ -0,0 +1,220 @@
+/**
+ * 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.APersistentMap;
+import clojure.lang.IFn;
+import java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * StateStorage provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ *     set_data("/path", data, acls);
+ *     void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface StateStorage {
+
+    /**
+     * Registers a callback function that gets called when CuratorEvents happen.
+     * @param callback is a clojure IFn that accepts the type - translated to
+     * clojure keyword as in zookeeper - and the path: (callback type path)
+     * @return is an id that can be passed to unregister(...) to unregister the
+     * callback.
+     */
+    String register(ZKStateChangedCallback callback);
+
+    /**
+     * Unregisters a callback function that was registered with register(...).
+     * @param id is the String id that was returned from register(...).
+     */
+    void unregister(String id);
+
+    /**
+     * Path will be appended with a monotonically increasing integer, a new node
+     * will be created there, and data will be put at that node.
+     * @param path The path that the monotonically increasing integer suffix will
+     * be added to.
+     * @param data The data that will be written at the suffixed path's node.
+     * @param acls The acls to apply to the path. May be null.
+     * @return The path with the integer suffix appended.
+     */
+    String create_sequential(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Creates nodes for path and all its parents. Path elements are separated by
+     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+     * @param path The path to create, along with all its parents.
+     * @param acls The acls to apply to the path. May be null.
+     * @return path
+     */
+    void mkdirs(String path, List<ACL> acls);
+
+    /**
+     * Deletes the node at a given path, and any child nodes that may exist.
+     * @param path The path to delete
+     */
+    void delete_node(String path);
+
+    /**
+     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+     * by the store when the client disconnects.
+     * @param path The path where a node will be created.
+     * @param data The data to be written at the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Gets the 'version' of the node at a path. Optionally sets a watch
+     * on that node. The version should increase whenever a write happens.
+     * @param path The path to get the version of.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The integer version of this node.
+     */
+    Integer get_version(String path, boolean watch) throws Exception;
+
+    /**
+     * Check if a node exists and optionally set a watch on the path.
+     * @param path The path to check for the existence of a node.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return Whether or not a node exists at path.
+     */
+    boolean node_exists(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_children(String path, boolean watch);
+
+    /**
+     * Close the connection to the data store.
+     */
+    void close();
+
+    /**
+     * Set the value of the node at path to data.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_data(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the data from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The data at the node.
+     */
+    byte[] get_data(String path, boolean watch);
+
+    /**
+     * Get the data at the node along with its version. Data is returned
+     * in an APersistentMap with clojure keyword keys :data and :version.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return An APersistentMap in the form {:data data :version version}
+     */
+    APersistentMap get_data_with_version(String path, boolean watch);
+
+    /**
+     * Write a worker heartbeat at the path.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the heartbeat from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The heartbeat at the node.
+     */
+    byte[] get_worker_hb(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path. This is similar to get_children, but must be used for
+     * any nodes
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_worker_hb_children(String path, boolean watch);
+
+    /**
+     * Deletes the heartbeat at a given path, and any child nodes that may exist.
+     * @param path The path to delete.
+     */
+    void delete_worker_hb(String path);
+
+    /**
+     * Add a StateStorageListener to the connection.
+     * @param listener A StateStorageListener to handle changing cluster state
+     * events.
+     */
+    void add_listener(final ConnectionStateListener listener);
+
+    /**
+     * Force consistency on a path. Any writes committed on the path before
+     * this call will be completely propagated when it returns.
+     * @param path The path to synchronize.
+     */
+    void sync_path(String path);
+
+    /**
+     * Allows us to delete the znodes within /storm/blobstore/key_name
+     * whose znodes start with the corresponding nimbusHostPortInfo
+     * @param path /storm/blobstore/key_name
+     * @param nimbusHostPortInfo Contains the host port information of
+     * a nimbus node.
+     */
+    void delete_node_blobstore(String path, String nimbusHostPortInfo);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
new file mode 100644
index 0000000..9803dff
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -0,0 +1,28 @@
+/**
+ * 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.APersistentMap;
+import java.util.List;
+import org.apache.zookeeper.data.ACL;
+
+public interface StateStorageFactory {
+    
+    StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
index ede2ba3..58b125b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -114,7 +114,7 @@ public interface StormClusterState {
 
     public void removeKeyVersion(String blobKey);
 
-    public void reportError(String stormId, String componentId, String node, Integer port, String error);
+    public void reportError(String stormId, String componentId, String node, Long port, String error);
 
     public List<ErrorInfo> errors(String stormId, String componentId);
 


[03/27] storm git commit: delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 48934f6..85ed37d 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -15,11 +15,11 @@
 ;; limitations under the License.
 (ns org.apache.storm.daemon.worker
   (:use [org.apache.storm.daemon common])
-  (:use [org.apache.storm config log util timer local-state])
+  (:use [org.apache.storm config log util timer local-state converter])
   (:require [clj-time.core :as time])
   (:require [clj-time.coerce :as coerce])
   (:require [org.apache.storm.daemon [executor :as executor]])
-  (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
+  (:require [org.apache.storm [disruptor :as disruptor]])
   (:require [clojure.set :as set])
   (:require [org.apache.storm.messaging.loader :as msg-loader])
   (:import [java.util.concurrent Executors]
@@ -36,7 +36,7 @@
   (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [org.apache.storm Constants])
   (:import [org.apache.storm.security.auth AuthUtils])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType DistributedClusterState StormZkClusterState])
   (:import [javax.security.auth Subject])
   (:import [java.security PrivilegedExceptionAction])
   (:import [org.apache.logging.log4j LogManager])
@@ -49,7 +49,7 @@
 
 (defn read-worker-executors [storm-conf storm-cluster-state storm-id assignment-id port assignment-versions]
   (log-message "Reading Assignments.")
-  (let [assignment (:executor->node+port (.assignment-info storm-cluster-state storm-id nil))]
+  (let [assignment (:executor->node+port (clojurify-assignment (.assignmentInfo storm-cluster-state storm-id nil)))]
     (doall
      (concat
       [Constants/SYSTEM_EXECUTOR_ID]
@@ -73,7 +73,7 @@
                }]
     ;; do the zookeeper heartbeat
     (try
-      (.worker-heartbeat! (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) zk-hb)
+      (.workerHeartbeat (:storm-cluster-state worker) (:storm-id worker) (:assignment-id worker) (:port worker) (thriftify-zk-worker-hb zk-hb))
       (catch Exception exc
         (log-error exc "Worker failed to write heatbeats to ZK or Pacemaker...will retry")))))
 
@@ -146,7 +146,7 @@
         ;; update the worker's backpressure flag to zookeeper only when it has changed
         (log-debug "BP " @(:backpressure worker) " WAS " prev-backpressure-flag)
         (when (not= prev-backpressure-flag @(:backpressure worker))
-          (.worker-backpressure! storm-cluster-state storm-id assignment-id port @(:backpressure worker)))
+          (.workerBackpressure storm-cluster-state storm-id assignment-id port @(:backpressure worker)))
         ))))
 
 (defn- mk-disruptor-backpressure-handler [worker]
@@ -354,10 +354,11 @@
       ([]
         (this (fn [& ignored] (schedule (:refresh-connections-timer worker) 0 this))))
       ([callback]
-         (let [version (.assignment-version storm-cluster-state storm-id callback)
+         (let [version (.assignmentVersion storm-cluster-state storm-id callback)
                assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
                             (:data (get @(:assignment-versions worker) storm-id))
-                            (let [new-assignment (.assignment-info-with-version storm-cluster-state storm-id callback)]
+                            (let [java-assignment (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
+                              new-assignment {:data (clojurify-assignment (:data java-assignment)) :version version}]
                               (swap! (:assignment-versions worker) assoc storm-id new-assignment)
                               (:data new-assignment)))
               my-assignment (-> assignment
@@ -403,7 +404,7 @@
   ([worker]
     (refresh-storm-active worker (fn [& ignored] (schedule (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
   ([worker callback]
-    (let [base (.storm-base (:storm-cluster-state worker) (:storm-id worker) callback)]
+    (let [base (clojurify-storm-base (.stormBase (:storm-cluster-state worker) (:storm-id worker) callback))]
       (reset!
         (:storm-active-atom worker)
         (and (= :active (-> base :status :type)) @(:worker-active-flag worker)))
@@ -595,9 +596,9 @@
   (let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
         storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
         acls (Utils/getWorkerACL storm-conf)
-        cluster-state (cluster/mk-distributed-cluster-state conf :auth-conf storm-conf :acls acls :context (ClusterStateContext. DaemonType/WORKER))
-        storm-cluster-state (cluster/mk-storm-cluster-state cluster-state :acls acls)
-        initial-credentials (.credentials storm-cluster-state storm-id nil)
+        cluster-state (DistributedClusterState. conf storm-conf  acls  (ClusterStateContext. DaemonType/WORKER))
+        storm-cluster-state (StormZkClusterState. cluster-state acls (ClusterStateContext.))
+        initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
         auto-creds (AuthUtils/GetAutoCredentials storm-conf)
         subject (AuthUtils/populateSubject nil auto-creds initial-credentials)]
       (Subject/doAs subject (reify PrivilegedExceptionAction
@@ -644,10 +645,10 @@
         _ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE) 
             (.start backpressure-thread))
         callback (fn cb [& ignored]
-                   (let [throttle-on (.topology-backpressure storm-cluster-state storm-id cb)]
+                   (let [throttle-on (.topologyBackpressure storm-cluster-state storm-id cb)]
                      (reset! (:throttle-on worker) throttle-on)))
         _ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
-            (.topology-backpressure storm-cluster-state storm-id callback))
+            (.topologyBackpressure storm-cluster-state storm-id callback))
 
         shutdown* (fn []
                     (log-message "Shutting down worker " storm-id " " assignment-id " " port)
@@ -685,7 +686,7 @@
                     (log-message "Trigger any worker shutdown hooks")
                     (run-worker-shutdown-hooks worker)
 
-                    (.remove-worker-heartbeat! (:storm-cluster-state worker) storm-id assignment-id port)
+                    (.removeWorkerHeartbeat (:storm-cluster-state worker) storm-id assignment-id port)
                     (log-message "Disconnecting from storm cluster state context")
                     (.disconnect (:storm-cluster-state worker))
                     (.close (:cluster-state worker))
@@ -709,29 +710,29 @@
              )
         credentials (atom initial-credentials)
         check-credentials-changed (fn []
-                                    (let [new-creds (.credentials (:storm-cluster-state worker) storm-id nil)]
+                                    (let [new-creds (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id nil))]
                                       (when-not (= new-creds @credentials) ;;This does not have to be atomic, worst case we update when one is not needed
                                         (AuthUtils/updateSubject subject auto-creds new-creds)
                                         (dofor [e @executors] (.credentials-changed e new-creds))
                                         (reset! credentials new-creds))))
        check-throttle-changed (fn []
                                 (let [callback (fn cb [& ignored]
-                                                 (let [throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id cb)]
+                                                 (let [throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id cb)]
                                                    (reset! (:throttle-on worker) throttle-on)))
-                                      new-throttle-on (.topology-backpressure (:storm-cluster-state worker) storm-id callback)]
+                                      new-throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id callback)]
                                     (reset! (:throttle-on worker) new-throttle-on)))
         check-log-config-changed (fn []
-                                  (let [log-config (.topology-log-config (:storm-cluster-state worker) storm-id nil)]
+                                  (let [log-config (.topologyLogConfig (:storm-cluster-state worker) storm-id nil)]
                                     (process-log-config-change latest-log-config original-log-levels log-config)
                                     (establish-log-setting-callback)))]
     (reset! original-log-levels (get-logger-levels))
     (log-message "Started with log levels: " @original-log-levels)
   
     (defn establish-log-setting-callback []
-      (.topology-log-config (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
+      (.topologyLogConfig (:storm-cluster-state worker) storm-id (fn [args] (check-log-config-changed))))
 
     (establish-log-setting-callback)
-    (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed)))
+    (clojurify-crdentials (.credentials (:storm-cluster-state worker) storm-id (fn [args] (check-credentials-changed))))
     (schedule-recurring (:refresh-credentials-timer worker) 0 (conf TASK-CREDENTIALS-POLL-SECS)
                         (fn [& args]
                           (check-credentials-changed)

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index cede59e..b367b4b 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -16,27 +16,23 @@
 
 (ns org.apache.storm.pacemaker.pacemaker-state-factory
   (:require [org.apache.storm.pacemaker pacemaker]
-            [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
             [org.apache.storm
              [config :refer :all]
-             [cluster :refer :all]
              [log :refer :all]
              [util :as util]])
   (:import [org.apache.storm.generated
             HBExecutionException HBServerMessageType HBMessage
             HBMessageData HBPulse]
-           [org.apache.storm.cluster_state zookeeper_state_factory]
-           [org.apache.storm.cluster ClusterState]
+           [org.apache.storm.cluster ClusterState DistributedClusterState]
            [org.apache.storm.pacemaker PacemakerClient])
-  (:gen-class
-   :implements [org.apache.storm.cluster.ClusterStateFactory]))
+  (:gen-class))
 
 ;; So we can mock the client for testing
 (defn makeClient [conf]
   (PacemakerClient. conf))
 
 (defn makeZKState [conf auth-conf acls context]
-  (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
+  (DistributedClusterState. conf auth-conf acls context))
 
 (def max-retries 10)
 
@@ -47,7 +43,7 @@
     (reify
       ClusterState
       ;; Let these pass through to the zk-state. We only want to handle heartbeats.
-      (register [this callback] (.register zk-state callback))
+      (register [this callback] (.register zk-state callback))  ; need update callback, have questions?? callback is IFn here
       (unregister [this callback] (.unregister zk-state callback))
       (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
       (create_sequential [this path data acls] (.create_sequential zk-state path data acls))

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/stats.clj b/storm-core/src/clj/org/apache/storm/stats.clj
index 68b16fd..d6bcdc3 100644
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@ -24,6 +24,7 @@
             ExecutorAggregateStats SpecificAggregateStats
             SpoutAggregateStats TopologyPageInfo TopologyStats])
   (:import [org.apache.storm.utils Utils])
+  (:import [org.apache.storm.cluster StormZkClusterState])
   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
   (:use [org.apache.storm log util])
   (:use [clojure.math.numeric-tower :only [ceil]]))
@@ -794,7 +795,7 @@
 
 (defn get-last-error
   [storm-cluster-state storm-id component-id]
-  (if-let [e (.last-error storm-cluster-state storm-id component-id)]
+  (if-let [e (clojurify-error (.lastError storm-cluster-state storm-id component-id))]
     (ErrorInfo. (:error e) (:time-secs e))))
 
 (defn component-type

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index cc78659..eb34d36 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -45,10 +45,10 @@
   (:import [org.apache.storm.tuple Tuple])
   (:import [org.apache.storm.generated StormTopology])
   (:import [org.apache.storm.task TopologyContext])
-  (:require [org.apache.storm [zookeeper :as zk]])
+  (:import [org.apache.storm.cluster DistributedClusterState ClusterStateContext StormZkClusterState])
   (:require [org.apache.storm.messaging.loader :as msg-loader])
   (:require [org.apache.storm.daemon.acker :as acker])
-  (:use [org.apache.storm cluster util thrift config log local-state]))
+  (:use [org.apache.storm util thrift config log local-state converter]))
 
 (defn feeder-spout
   [fields]
@@ -158,8 +158,8 @@
                      :port-counter port-counter
                      :daemon-conf daemon-conf
                      :supervisors (atom [])
-                     :state (mk-distributed-cluster-state daemon-conf)
-                     :storm-cluster-state (mk-storm-cluster-state daemon-conf)
+                     :state (DistributedClusterState. daemon-conf nil nil (ClusterStateContext.))
+                     :storm-cluster-state (StormZkClusterState. daemon-conf nil (ClusterStateContext.))
                      :tmp-dirs (atom [nimbus-tmp zk-tmp])
                      :zookeeper (if (not-nil? zk-handle) zk-handle)
                      :shared-context context
@@ -403,8 +403,8 @@
                            (select-keys component->tasks component-ids)
                            component->tasks)
         task-ids (apply concat (vals component->tasks))
-        assignment (.assignment-info state storm-id nil)
-        taskbeats (.taskbeats state storm-id (:task->node+port assignment))
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
+        taskbeats (.taskbeats state storm-id (:task->node+port assignment))  ;hava question?
         heartbeats (dofor [id task-ids] (get taskbeats id))
         stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
     (reduce + stats)))
@@ -551,7 +551,7 @@
                      (simulate-wait cluster-map))
 
       (.killTopologyWithOpts (:nimbus cluster-map) storm-name (doto (KillOptions.) (.set_wait_secs 0)))
-      (while-timeout timeout-ms (.assignment-info state storm-id nil)
+      (while-timeout timeout-ms (clojurify-assignment (.assignmentInfo state storm-id nil))
                      (simulate-wait cluster-map))
       (when cleanup-state
         (doseq [spout (spout-objects spouts)]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/thrift.clj b/storm-core/src/clj/org/apache/storm/thrift.clj
index b5af521..4dc21f9 100644
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@ -30,7 +30,7 @@
   (:import [org.apache.storm.topology TopologyBuilder])
   (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
   (:import [org.apache.thrift.transport TTransport])
-  (:use [org.apache.storm util config log zookeeper]))
+  (:use [org.apache.storm util config log]))
 
 (defn instantiate-java-object
   [^JavaObject obj]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/ui/core.clj b/storm-core/src/clj/org/apache/storm/ui/core.clj
index f26d998..14313be 100644
--- a/storm-core/src/clj/org/apache/storm/ui/core.clj
+++ b/storm-core/src/clj/org/apache/storm/ui/core.clj
@@ -21,7 +21,7 @@
         ring.middleware.multipart-params)
   (:use [ring.middleware.json :only [wrap-json-params]])
   (:use [hiccup core page-helpers])
-  (:use [org.apache.storm config util log stats zookeeper converter])
+  (:use [org.apache.storm config util log stats converter])
   (:use [org.apache.storm.ui helpers])
   (:use [org.apache.storm.daemon [common :only [ACKER-COMPONENT-ID ACKER-INIT-STREAM-ID ACKER-ACK-STREAM-ID
                                               ACKER-FAIL-STREAM-ID mk-authorization-handler

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/util.clj b/storm-core/src/clj/org/apache/storm/util.clj
index 23d39f6..165d8ee 100644
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@ -20,6 +20,7 @@
   (:import [java.io FileReader FileNotFoundException])
   (:import [java.nio.file Paths])
   (:import [org.apache.storm Config])
+  (:import [org.apache.storm.generated ErrorInfo])
   (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
             MutableObject MutableInt])
   (:import [org.apache.storm.security.auth NimbusPrincipal])
@@ -261,6 +262,16 @@
                    (instance? Boolean x) (boolean x)
                    true x))
            s))
+; move this func form convert.clj due to cyclic load dependency
+(defn clojurify-error [^ErrorInfo error]
+  (if error
+    {
+      :error (.get_error error)
+      :time-secs (.get_error_time_secs error)
+      :host (.get_host error)
+      :port (.get_port error)
+      }
+    ))
 
 (defmacro with-file-lock
   [path & body]

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/clj/org/apache/storm/zookeeper.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/zookeeper.clj b/storm-core/src/clj/org/apache/storm/zookeeper.clj
deleted file mode 100644
index 413ffd6..0000000
--- a/storm-core/src/clj/org/apache/storm/zookeeper.clj
+++ /dev/null
@@ -1,75 +0,0 @@
-;; 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.
-
-(ns org.apache.storm.zookeeper
-  (:import [org.apache.curator.retry RetryNTimes]
-           [org.apache.storm Config])
-  (:import [org.apache.curator.framework.api CuratorEvent CuratorEventType CuratorListener UnhandledErrorListener])
-  (:import [org.apache.curator.framework.state ConnectionStateListener])
-  (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory])
-  (:import [org.apache.curator.framework.recipes.leader LeaderLatch LeaderLatch$State Participant LeaderLatchListener])
-  (:import [org.apache.zookeeper ZooKeeper Watcher KeeperException$NoNodeException
-            ZooDefs ZooDefs$Ids CreateMode WatchedEvent Watcher$Event Watcher$Event$KeeperState
-            Watcher$Event$EventType KeeperException$NodeExistsException])
-  (:import [org.apache.zookeeper.data Stat])
-  (:import [org.apache.zookeeper.server ZooKeeperServer NIOServerCnxnFactory])
-  (:import [java.net InetSocketAddress BindException InetAddress])
-  (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
-  (:import [java.io File])
-  (:import [java.util List Map])
-  (:import [org.apache.storm.zookeeper Zookeeper ZkKeeperStates ZkEventTypes])
-  (:import [org.apache.storm.utils Utils ZookeeperAuthInfo])
-  (:use [org.apache.storm util log config]))
-
-
-(defn- default-watcher
-  [state type path]
-  (log-message "Zookeeper state update: " state type path))
-
-(defnk mk-client
-  [conf servers port
-   :root ""
-   :watcher default-watcher
-   :auth-conf nil]
-  (let [fk (Utils/newCurator conf servers port root (when auth-conf (ZookeeperAuthInfo. auth-conf)))]
-    (.. fk
-      (getCuratorListenable)
-      (addListener
-        (reify CuratorListener
-          (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e]
-            (when (= (.getType e) CuratorEventType/WATCHED)
-              (let [^WatchedEvent event (.getWatchedEvent e)]
-                (watcher (.getState event)
-                  (.getType event)
-                  (.getPath event))))))))
-    ;;    (.. fk
-    ;;        (getUnhandledErrorListenable)
-    ;;        (addListener
-    ;;         (reify UnhandledErrorListener
-    ;;           (unhandledError [this msg error]
-    ;;             (if (or (exception-cause? InterruptedException error)
-    ;;                     (exception-cause? java.nio.channels.ClosedByInterruptException error))
-    ;;               (do (log-warn-error error "Zookeeper exception " msg)
-    ;;                   (let [to-throw (InterruptedException.)]
-    ;;                     (.initCause to-throw error)
-    ;;                     (throw to-throw)
-    ;;                     ))
-    ;;               (do (log-error error "Unrecoverable Zookeeper error " msg)
-    ;;                   (halt-process! 1 "Unrecoverable Zookeeper error")))
-    ;;             ))))
-    (.start fk)
-    fk))
-

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/callback/Callback.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/callback/Callback.java b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
index 29b9761..a37612d 100644
--- a/storm-core/src/jvm/org/apache/storm/callback/Callback.java
+++ b/storm-core/src/jvm/org/apache/storm/callback/Callback.java
@@ -18,6 +18,9 @@
 
 package org.apache.storm.callback;
 
+import clojure.lang.IFn;
+
+// To remove IFn after porting all callbacks to java
 public interface Callback {
     public <T> Object execute(T... args);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
index 2d6f306..851858f 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/Cluster.java
@@ -27,8 +27,7 @@ import org.apache.zookeeper.ZooDefs;
 import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Id;
 import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 
 import java.io.UnsupportedEncodingException;
 import java.net.URLEncoder;
@@ -84,10 +83,36 @@ public class Cluster {
         PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
     }
 
+    // A singleton instance allows us to mock delegated static methods in our
+    // tests by subclassing.
+    private static final Cluster INSTANCE = new Cluster();
+    private static Cluster _instance = INSTANCE;
+
+    /**
+     * Provide an instance of this class for delegates to use.  To mock out
+     * delegated methods, provide an instance of a subclass that overrides the
+     * implementation of the delegated method.
+     *
+     * @param u a Zookeeper instance
+     */
+    public static void setInstance(Cluster u) {
+        _instance = u;
+    }
+
+    /**
+     * Resets the singleton instance to the default. This is helpful to reset
+     * the class to its original functionality when mocking is no longer
+     * desired.
+     */
+    public static void resetInstance() {
+        _instance = INSTANCE;
+    }
+
     public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
-        List<ACL> aclList = new ArrayList<>();
+        List<ACL> aclList = null;
         String payload = (String)topoConf.get(Config.STORM_ZOOKEEPER_AUTH_PAYLOAD);
         if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)){
+            aclList = new ArrayList<>();
             ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
             aclList.add(acl1);
             ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
@@ -182,6 +207,13 @@ public class Cluster {
         }
         return executorWhb;
     }
+
+    public  StormClusterState mkStormClusterStateImpl(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
+        return new StormZkClusterState(clusterState, acls, context);
+    }
+    public static StormClusterState mkStormClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception{
+        return _instance.mkStormClusterStateImpl(clusterState, acls, context);
+    }
     
     // TO be remove
     public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
index 51e42ff..e76721b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterState.java
@@ -46,7 +46,7 @@ public interface ClusterState {
     /**
      * Registers a callback function that gets called when CuratorEvents happen.
      * @param callback is a clojure IFn that accepts the type - translated to
-     * clojure keyword as in zookeeper.clj - and the path: (callback type path)
+     * clojure keyword as in zookeeper - and the path: (callback type path)
      * @return is an id that can be passed to unregister(...) to unregister the
      * callback.
      */

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
index 3e0beb1..1bd534e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/DistributedClusterState.java
@@ -56,8 +56,7 @@ public class DistributedClusterState implements ClusterState {
     public DistributedClusterState(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
         this.conf = conf;
         this.authConf = authConf;
-        if (context.getDaemonType().equals(DaemonType.NIMBUS))
-            this.isNimbus = true;
+        if (context.getDaemonType().equals(DaemonType.NIMBUS)) this.isNimbus = true;
 
         // just mkdir STORM_ZOOKEEPER_ROOT dir
         CuratorFramework zkTemp = mkZk();
@@ -128,9 +127,9 @@ public class DistributedClusterState implements ClusterState {
     }
 
     @Override
-    public String register(Callback callback) {
+    public String register( Callback callback) {
         String id = UUID.randomUUID().toString();
-        this.callbacks.put(id, callback);
+        this.callbacks.put(id,callback);
         return id;
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
index b3c0f90..ede2ba3 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
@@ -18,7 +18,7 @@
 package org.apache.storm.cluster;
 
 import clojure.lang.APersistentMap;
-import org.apache.storm.callback.Callback;
+import clojure.lang.IFn;
 import org.apache.storm.generated.*;
 import org.apache.storm.nimbus.NimbusInfo;
 
@@ -27,13 +27,13 @@ import java.util.List;
 import java.util.Map;
 
 public interface StormClusterState {
-    public List<String> assignments(Callback callback);
+    public List<String> assignments(IFn callback);
 
-    public Assignment assignmentInfo(String stormId, Callback callback);
+    public Assignment assignmentInfo(String stormId, IFn callback);
 
-    public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback);
+    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
 
-    public Integer assignmentVersion(String stormId, Callback callback) throws Exception;
+    public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
 
     // returns key information under /storm/blobstore/key
     public List<String> blobstoreInfo(String blobKey);
@@ -46,27 +46,27 @@ public interface StormClusterState {
 
     public List<String> activeStorms();
 
-    public StormBase stormBase(String stormId, Callback callback);
+    public StormBase stormBase(String stormId, IFn callback);
 
     public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
 
-    public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift);
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
 
-    public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift);
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
 
-    public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest);
+    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
 
     public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
 
     public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
 
-    public List<String> supervisors(Callback callback);
+    public List<String> supervisors(IFn callback);
 
     public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
 
     public void setupHeatbeats(String stormId);
 
-    public void teardownHeatbeats(String stormId);
+    public void teardownHeartbeats(String stormId);
 
     public void teardownTopologyErrors(String stormId);
 
@@ -76,7 +76,7 @@ public interface StormClusterState {
 
     public void setTopologyLogConfig(String stormId, LogConfig logConfig);
 
-    public LogConfig topologyLogConfig(String stormId, Callback cb);
+    public LogConfig topologyLogConfig(String stormId, IFn cb);
 
     public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
 
@@ -86,7 +86,7 @@ public interface StormClusterState {
 
     public void workerBackpressure(String stormId, String node, Long port, boolean on);
 
-    public boolean topologyBackpressure(String stormId, Callback callback);
+    public boolean topologyBackpressure(String stormId, IFn callback);
 
     public void setupBackpressure(String stormId);
 
@@ -102,11 +102,11 @@ public interface StormClusterState {
 
     // sets up information related to key consisting of nimbus
     // host:port and version info of the blob
-    public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo);
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
 
     public List<String> activeKeys();
 
-    public List<String> blobstore(Callback callback);
+    public List<String> blobstore(IFn callback);
 
     public void removeStorm(String stormId);
 
@@ -114,7 +114,7 @@ public interface StormClusterState {
 
     public void removeKeyVersion(String blobKey);
 
-    public void reportError(String stormId, String componentId, String node, Long port, String error);
+    public void reportError(String stormId, String componentId, String node, Integer port, String error);
 
     public List<ErrorInfo> errors(String stormId, String componentId);
 
@@ -122,7 +122,7 @@ public interface StormClusterState {
 
     public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
 
-    public Credentials credentials(String stormId, Callback callback);
+    public Credentials credentials(String stormId, IFn callback);
 
     public void disconnect();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
index 93d29b2..3f32fe1 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -18,6 +18,7 @@
 package org.apache.storm.cluster;
 
 import clojure.lang.APersistentMap;
+import clojure.lang.IFn;
 import clojure.lang.PersistentArrayMap;
 import clojure.lang.RT;
 import org.apache.commons.lang.StringUtils;
@@ -47,17 +48,17 @@ public class StormZkClusterState implements StormClusterState {
 
     private ClusterState clusterState;
 
-    private ConcurrentHashMap<String, Callback> assignmentInfoCallback;
-    private ConcurrentHashMap<String, Callback> assignmentInfoWithVersionCallback;
-    private ConcurrentHashMap<String, Callback> assignmentVersionCallback;
-    private AtomicReference<Callback> supervisorsCallback;
+    private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
+    private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
+    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
+    private AtomicReference<IFn> supervisorsCallback;
     // we want to reigister a topo directory getChildren callback for all workers of this dir
-    private ConcurrentHashMap<String, Callback> backPressureCallback;
-    private AtomicReference<Callback> assignmentsCallback;
-    private ConcurrentHashMap<String, Callback> stormBaseCallback;
-    private AtomicReference<Callback> blobstoreCallback;
-    private ConcurrentHashMap<String, Callback> credentialsCallback;
-    private ConcurrentHashMap<String, Callback> logConfigCallback;
+    private ConcurrentHashMap<String, IFn> backPressureCallback;
+    private AtomicReference<IFn> assignmentsCallback;
+    private ConcurrentHashMap<String, IFn> stormBaseCallback;
+    private AtomicReference<IFn> blobstoreCallback;
+    private ConcurrentHashMap<String, IFn> credentialsCallback;
+    private ConcurrentHashMap<String, IFn> logConfigCallback;
 
     private List<ACL> acls;
     private String stateId;
@@ -102,7 +103,7 @@ public class StormZkClusterState implements StormClusterState {
                 if (size >= 1) {
                     String params = null;
                     String root = toks.get(0);
-                    Callback fn = null;
+                    IFn fn = null;
                     if (root.equals(Cluster.ASSIGNMENTS_ROOT)) {
                         if (size == 1) {
                             // set null and get the old value
@@ -145,18 +146,18 @@ public class StormZkClusterState implements StormClusterState {
 
     }
 
-    protected void issueCallback(AtomicReference<Callback> cb) {
-        Callback callback = cb.getAndSet(null);
-        callback.execute();
+    protected void issueCallback(AtomicReference<IFn> cb) {
+        IFn callback = cb.getAndSet(null);
+        callback.invoke();
     }
 
-    protected void issueMapCallback(ConcurrentHashMap<String, Callback> callbackConcurrentHashMap, String key) {
-        Callback callback = callbackConcurrentHashMap.remove(key);
-        callback.execute();
+    protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
+        IFn callback = callbackConcurrentHashMap.remove(key);
+        callback.invoke();
     }
 
     @Override
-    public List<String> assignments(Callback callback) {
+    public List<String> assignments(IFn callback) {
         if (callback != null) {
             assignmentsCallback.set(callback);
         }
@@ -164,7 +165,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public Assignment assignmentInfo(String stormId, Callback callback) {
+    public Assignment assignmentInfo(String stormId, IFn callback) {
         if (callback != null) {
             assignmentInfoCallback.put(stormId, callback);
         }
@@ -173,7 +174,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public APersistentMap assignmentInfoWithVersion(String stormId, Callback callback) {
+    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
         if (callback != null) {
             assignmentInfoWithVersionCallback.put(stormId, callback);
         }
@@ -185,7 +186,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public Integer assignmentVersion(String stormId, Callback callback) throws Exception {
+    public Integer assignmentVersion(String stormId, IFn callback) throws Exception {
         if (callback != null) {
             assignmentVersionCallback.put(stormId, callback);
         }
@@ -237,7 +238,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public StormBase stormBase(String stormId, Callback callback) {
+    public StormBase stormBase(String stormId, IFn callback) {
         if (callback != null) {
             stormBaseCallback.put(stormId, callback);
         }
@@ -254,9 +255,9 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public List<ProfileRequest> getWorkerProfileRequets(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
         List<ProfileRequest> requests = new ArrayList<>();
-        List<ProfileRequest> profileRequests = getTopologyProfileRequets(stormId, isThrift);
+        List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
         for (ProfileRequest profileRequest : profileRequests) {
             NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
             if (nodeInfo1.equals(nodeInfo))
@@ -266,7 +267,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public List<ProfileRequest> getTopologyProfileRequets(String stormId, boolean isThrift) {
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
         List<ProfileRequest> profileRequests = new ArrayList<>();
         String path = Cluster.profilerConfigPath(stormId);
         if (clusterState.node_exists(path, false)) {
@@ -283,7 +284,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public void setWorkerProfileRequests(String stormId, ProfileRequest profileRequest) {
+    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();
@@ -300,11 +301,18 @@ public class StormZkClusterState implements StormClusterState {
         clusterState.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
     @Override
     public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
         Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
 
+        LOG.info(executorNodePort.toString());
         Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.reverseMap(executorNodePort);
+        LOG.info(nodePortExecutors.toString());
+
         for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
 
             String node = entry.getKey().get_node();
@@ -320,7 +328,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public List<String> supervisors(Callback callback) {
+    public List<String> supervisors(IFn callback) {
         if (callback != null) {
             supervisorsCallback.set(callback);
         }
@@ -339,7 +347,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public void teardownHeatbeats(String stormId) {
+    public void teardownHeartbeats(String stormId) {
         try {
             clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
         } catch (Exception e) {
@@ -382,7 +390,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public LogConfig topologyLogConfig(String stormId, Callback cb) {
+    public LogConfig topologyLogConfig(String stormId, IFn cb) {
         String path = Cluster.logConfigPath(stormId);
         return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
     }
@@ -426,7 +434,7 @@ public class StormZkClusterState implements StormClusterState {
 
     // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
     @Override
-    public boolean topologyBackpressure(String stormId, Callback callback) {
+    public boolean topologyBackpressure(String stormId, IFn callback) {
         if (callback != null) {
             backPressureCallback.put(stormId, callback);
         }
@@ -458,26 +466,27 @@ public class StormZkClusterState implements StormClusterState {
 
         StormBase stormBase = stormBase(stormId, null);
         if (stormBase.get_component_executors() != null) {
+
+            Map<String, Integer> newComponentExecutors = new HashMap<>();
             Map<String, Integer> componentExecutors = newElems.get_component_executors();
-            if (componentExecutors == null) {
-                componentExecutors = new HashMap<>();
+            //componentExecutors maybe be APersistentMap, which don't support put
+            for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+                    newComponentExecutors.put(entry.getKey(), entry.getValue());
             }
             for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
                 if (!componentExecutors.containsKey(entry.getKey())) {
-                    componentExecutors.put(entry.getKey(), entry.getValue());
+                    newComponentExecutors.put(entry.getKey(), entry.getValue());
                 }
             }
-            if (componentExecutors.size() > 0)
-                newElems.set_component_executors(componentExecutors);
+            if (newComponentExecutors.size() > 0)
+                newElems.set_component_executors(newComponentExecutors);
         }
 
         Map<String, DebugOptions> ComponentDebug = new HashMap<>();
         Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
-        if (oldComponentDebug == null)
-            oldComponentDebug = new HashMap<>();
+
         Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
-        if (newComponentDebug == null)
-            newComponentDebug = new HashMap<>();
+
         Set<String> debugOptionsKeys = oldComponentDebug.keySet();
         debugOptionsKeys.addAll(newComponentDebug.keySet());
         for (String key : debugOptionsKeys) {
@@ -499,7 +508,17 @@ public class StormZkClusterState implements StormClusterState {
         if (ComponentDebug.size() > 0) {
             newElems.set_component_debug(ComponentDebug);
         }
-        // only merge some parameters which are optional
+
+
+        if (StringUtils.isBlank(newElems.get_name())) {
+            newElems.set_name(stormBase.get_name());
+        }
+        if (newElems.get_status() == null){
+            newElems.set_status(stormBase.get_status());
+        }
+        if (newElems.get_num_workers() == 0){
+            newElems.set_num_workers(stormBase.get_num_workers());
+        }
         if (newElems.get_launch_time_secs() == 0) {
             newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
         }
@@ -526,8 +545,8 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public void setupBlobstore(String key, NimbusInfo nimbusInfo, String versionInfo) {
-        String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "_" + versionInfo;
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+        String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
         LOG.info("set-path: {}", path);
         clusterState.mkdirs(Cluster.blobstorePath(key), acls);
         clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
@@ -541,7 +560,7 @@ public class StormZkClusterState implements StormClusterState {
 
     // blobstore state
     @Override
-    public List<String> blobstore(Callback callback) {
+    public List<String> blobstore(IFn callback) {
         if (callback != null) {
             blobstoreCallback.set(callback);
         }
@@ -571,7 +590,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public void reportError(String stormId, String componentId, String node, Long port, String error) {
+    public void reportError(String stormId, String componentId, String node, Integer port, String error) {
 
         try {
             String path = Cluster.errorPath(stormId, componentId);
@@ -644,7 +663,7 @@ public class StormZkClusterState implements StormClusterState {
     }
 
     @Override
-    public Credentials credentials(String stormId, Callback callback) {
+    public Credentials credentials(String stormId, IFn callback) {
         if (callback != null) {
             credentialsCallback.put(stormId, callback);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
new file mode 100644
index 0000000..5d67a54
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
@@ -0,0 +1,31 @@
+/**
+ * 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.testing.staticmocking;
+
+import org.apache.storm.cluster.Cluster;
+
+public class MockedCluster implements AutoCloseable  {
+
+    public MockedCluster(Cluster inst) {
+        Cluster.setInstance(inst);
+    }
+
+    @Override
+    public void close() throws Exception {
+        Cluster.resetInstance();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index cd2bc26..d374511 100644
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@ -21,7 +21,8 @@
   (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
   (:import [org.apache.storm.tuple Fields])
-  (:use [org.apache.storm testing config clojure util])
+  (:import [org.apache.storm.cluster StormZkClusterState])
+  (:use [org.apache.storm testing config clojure util converter])
   (:use [org.apache.storm.daemon common])
   (:require [org.apache.storm [thrift :as thrift]]))
 
@@ -575,34 +576,34 @@
                                              (:topology tracked))
             _ (advance-cluster-time cluster 11)
             storm-id (get-storm-id state "test-errors")
-            errors-count (fn [] (count (.errors state storm-id "2")))]
+            errors-count (fn [] (count (clojurify-error (.errors state storm-id "2"))))]
 
-        (is (nil? (.last-error state storm-id "2")))
+        (is (nil? (clojurify-error (.lastError state storm-id "2"))))
 
         ;; so it launches the topology
         (advance-cluster-time cluster 2)
         (.feed feeder [6])
         (tracked-wait tracked 1)
         (is (= 4 (errors-count)))
-        (is (.last-error state storm-id "2"))
+        (is (clojurify-error (.lastError state storm-id "2")))
         
         (advance-time-secs! 5)
         (.feed feeder [2])
         (tracked-wait tracked 1)
         (is (= 4 (errors-count)))
-        (is (.last-error state storm-id "2"))
+        (is (clojurify-error (.lastError state storm-id "2")))
         
         (advance-time-secs! 6)
         (.feed feeder [2])
         (tracked-wait tracked 1)
         (is (= 6 (errors-count)))
-        (is (.last-error state storm-id "2"))
+        (is (clojurify-error (.lastError state storm-id "2")))
         
         (advance-time-secs! 6)
         (.feed feeder [3])
         (tracked-wait tracked 1)
         (is (= 8 (errors-count)))
-        (is (.last-error state storm-id "2"))))))
+        (is  (clojurify-error (.lastError state storm-id "2")))))))
 
 
 (deftest test-acking-branching-complex

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index ffd913e..d0b9882 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,14 +23,13 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
   (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
-  (:import [org.apache.storm.cluster ClusterState])
+  (:import [org.apache.storm.cluster ClusterState DistributedClusterState ClusterStateContext StormZkClusterState])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
-  (:require [org.apache.storm [zookeeper :as zk]])
   (:require [conjure.core])
   (:use [conjure core])
   (:use [clojure test])
-  (:use [org.apache.storm cluster config util testing thrift log]))
+  (:use [org.apache.storm config util testing thrift log converter]))
 
 (defn mk-config [zk-port]
   (merge (clojurify-structure (ConfigUtils/readStormConfig))
@@ -39,13 +38,13 @@
 
 (defn mk-state
   ([zk-port] (let [conf (mk-config zk-port)]
-               (mk-distributed-cluster-state conf :auth-conf conf)))
+               (DistributedClusterState. conf conf nil (ClusterStateContext.))))
   ([zk-port cb]
      (let [ret (mk-state zk-port)]
        (.register ret cb)
        ret )))
 
-(defn mk-storm-state [zk-port] (mk-storm-cluster-state (mk-config zk-port)))
+(defn mk-storm-state [zk-port] (StormZkClusterState. (mk-config zk-port) nil (ClusterStateContext.)))
 
 (deftest test-basics
   (with-inprocess-zookeeper zk-port
@@ -182,48 +181,48 @@
           base1 (StormBase. "/tmp/storm1" 1 {:type :active} 2 {} "" nil nil {})
           base2 (StormBase. "/tmp/storm2" 2 {:type :active} 2 {} "" nil nil {})]
       (is (= [] (.assignments state nil)))
-      (.set-assignment! state "storm1" assignment1)
-      (is (= assignment1 (.assignment-info state "storm1" nil)))
-      (is (= nil (.assignment-info state "storm3" nil)))
-      (.set-assignment! state "storm1" assignment2)
-      (.set-assignment! state "storm3" assignment1)
+      (.setAssignment state "storm1" (thriftify-assignment assignment1))
+      (is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
+      (is (= nil (clojurify-assignment (.assignmentInfo state "storm3" nil))))
+      (.setAssignment state "storm1" (thriftify-assignment assignment2))
+      (.setAssignment state "storm3" (thriftify-assignment assignment1))
       (is (= #{"storm1" "storm3"} (set (.assignments state nil))))
-      (is (= assignment2 (.assignment-info state "storm1" nil)))
-      (is (= assignment1 (.assignment-info state "storm3" nil)))
+      (is (= assignment2 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
+      (is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm3" nil))))
       
       (is (= [] (.active-storms state)))
-      (.activate-storm! state "storm1" base1)
+      (.activateStorm state "storm1" (thriftify-storm-base base1))
       (is (= ["storm1"] (.active-storms state)))
-      (is (= base1 (.storm-base state "storm1" nil)))
-      (is (= nil (.storm-base state "storm2" nil)))
-      (.activate-storm! state "storm2" base2)
-      (is (= base1 (.storm-base state "storm1" nil)))
-      (is (= base2 (.storm-base state "storm2" nil)))
+      (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
+      (is (= nil (clojurify-storm-base (.stormBase state "storm2" nil))))
+      (.activateStorm state "storm2" (thriftify-storm-base base2))
+      (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
+      (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
       (is (= #{"storm1" "storm2"} (set (.active-storms state))))
-      (.remove-storm-base! state "storm1")
-      (is (= base2 (.storm-base state "storm2" nil)))
+      (.removeStormBase state "storm1")
+      (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
       (is (= #{"storm2"} (set (.active-storms state))))
 
-      (is (nil? (.credentials state "storm1" nil)))
-      (.set-credentials! state "storm1" {"a" "a"} {})
-      (is (= {"a" "a"} (.credentials state "storm1" nil)))
-      (.set-credentials! state "storm1" {"b" "b"} {})
-      (is (= {"b" "b"} (.credentials state "storm1" nil)))
+      (is (nil? (clojurify-crdentials (.credentials state "storm1" nil))))
+      (.setCredentials! state "storm1" (thriftify-credentials {"a" "a"}) {})
+      (is (= {"a" "a"} (clojurify-crdentials (.credentials state "storm1" nil))))
+      (.setCredentials state "storm1" (thriftify-credentials {"b" "b"}) {})
+      (is (= {"b" "b"} (clojurify-crdentials (.credentials state "storm1" nil))))
 
-      (is (= [] (.blobstore-info state nil)))
-      (.setup-blobstore! state "key1" nimbusInfo1 "1")
-      (is (= ["key1"] (.blobstore-info state nil)))
-      (is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstore-info state "key1")))
-      (.setup-blobstore! state "key1" nimbusInfo2 "1")
+      (is (= [] (.blobstoreInfo state nil)))
+      (.setupBlobstore state "key1" nimbusInfo1 "1")
+      (is (= ["key1"] (.blobstoreInfo state nil)))
+      (is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstoreInfo state "key1")))
+      (.setupBlobstore state "key1" nimbusInfo2 "1")
       (is (= #{(str (.toHostPortString nimbusInfo1) "-1")
-               (str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstore-info state "key1"))))
-      (.remove-blobstore-key! state "key1")
-      (is (= [] (.blobstore-info state nil)))
+               (str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstoreInfo state "key1"))))
+      (.removeBlobstoreKey state "key1")
+      (is (= [] (.blobstoreInfo state nil)))
 
       (is (= [] (.nimbuses state)))
-      (.add-nimbus-host! state "nimbus1:port" nimbusSummary1)
+      (.addNimbusHost state "nimbus1:port" nimbusSummary1)
       (is (= [nimbusSummary1] (.nimbuses state)))
-      (.add-nimbus-host! state "nimbus2:port" nimbusSummary2)
+      (.addNimbusHost state "nimbus2:port" nimbusSummary2)
       (is (= #{nimbusSummary1 nimbusSummary2} (set (.nimbuses state))))
 
       ;; TODO add tests for task info and task heartbeat setting and getting
@@ -231,7 +230,7 @@
       )))
 
 (defn- validate-errors! [state storm-id component errors-list]
-  (let [errors (.errors state storm-id component)]
+  (let [errors (clojurify-error (.errors state storm-id component))]
     ;;(println errors)
     (is (= (count errors) (count errors-list)))
     (doseq [[error target] (map vector errors errors-list)]
@@ -245,17 +244,17 @@
   (with-inprocess-zookeeper zk-port
     (with-simulated-time
       (let [state (mk-storm-state zk-port)]
-        (.report-error state "a" "1" (local-hostname) 6700 (RuntimeException.))
+        (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (RuntimeException.)))
         (validate-errors! state "a" "1" ["RuntimeException"])
         (advance-time-secs! 1)
-        (.report-error state "a" "1" (local-hostname) 6700 (IllegalArgumentException.))
+        (.reportError state "a" "1" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
         (validate-errors! state "a" "1" ["IllegalArgumentException" "RuntimeException"])
         (doseq [i (range 10)]
-          (.report-error state "a" "2" (local-hostname) 6700 (RuntimeException.))
+          (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (RuntimeException.)))
           (advance-time-secs! 2))
         (validate-errors! state "a" "2" (repeat 10 "RuntimeException"))
         (doseq [i (range 5)]
-          (.report-error state "a" "2" (local-hostname) 6700 (IllegalArgumentException.))
+          (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
           (advance-time-secs! 2))
         (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
                                                 (repeat 5 "RuntimeException")
@@ -271,10 +270,10 @@
           supervisor-info1 (SupervisorInfo. 10 "hostname-1" "id1" [1 2] [] {} 1000 "0.9.2" nil)
           supervisor-info2 (SupervisorInfo. 10 "hostname-2" "id2" [1 2] [] {} 1000 "0.9.2" nil)]
       (is (= [] (.supervisors state1 nil)))
-      (.supervisor-heartbeat! state2 "2" supervisor-info2)
-      (.supervisor-heartbeat! state1 "1" supervisor-info1)
-      (is (= supervisor-info2 (.supervisor-info state1 "2")))
-      (is (= supervisor-info1 (.supervisor-info state1 "1")))
+      (.supervisorHeartbeat state2 "2" (thriftify-supervisor-info supervisor-info2))
+      (.supervisorHeartbeat state1 "1" (thriftify-supervisor-info supervisor-info1))
+      (is (= supervisor-info2 (clojurify-supervisor-info (.supervisorInfo state1 "2"))))
+      (is (= supervisor-info1 (clojurify-supervisor-info (.supervisorInfo state1 "1"))))
       (is (= #{"1" "2"} (set (.supervisors state1 nil))))
       (is (= #{"1" "2"} (set (.supervisors state2 nil))))
       (.disconnect state2)
@@ -313,12 +312,10 @@
     (let [zk-mock (Mockito/mock Zookeeper)]
       ;; No need for when clauses because we just want to return nil
       (with-open [_ (MockedZookeeper. zk-mock)]
-        (stubbing [zk/mk-client (reify CuratorFramework (^void close [this] nil))]
-          (mk-distributed-cluster-state {})
-          (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil)))))
-    (stubbing [mk-distributed-cluster-state (reify ClusterState
-                                              (register [this callback] nil)
-                                              (mkdirs [this path acls] nil))]
-     (mk-storm-cluster-state {})
-     (verify-call-times-for mk-distributed-cluster-state 1)
-     (verify-first-call-args-for-indices mk-distributed-cluster-state [4] nil))))
+          (. (Mockito/when (Mockito/mock Zookeeper)) (thenReturn (reify CuratorFramework (^void close [this] nil))))
+          (. (Mockito/when (Mockito/mock DistributedClusterState)) (thenReturn {}))
+          (. (Mockito/when (Mockito/mock StormZkClusterState)) (thenReturn (reify ClusterState
+                                                                             (register [this callback] nil)
+                                                                             (mkdirs [this path acls] nil))))
+          (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))))
+

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 19c6f59..d4402fb 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -23,8 +23,10 @@
            [org.apache.storm.nimbus InMemoryTopologyActionNotifier])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
   (:import [org.apache.storm.scheduler INimbus])
+  (:import [org.mockito Mockito])
+  (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [org.apache.storm.nimbus ILeaderElector NimbusInfo])
-  (:import [org.apache.storm.testing.staticmocking MockedConfigUtils])
+  (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
   (:import [org.apache.storm.generated Credentials NotAliveException SubmitOptions
             TopologyInitialStatus TopologyStatus AlreadyAliveException KillOptions RebalanceOptions
             InvalidTopologyException AuthorizationException
@@ -34,12 +36,12 @@
   (:import [org.apache.storm.utils Time Utils ConfigUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:import [org.apache.commons.io FileUtils])
-  (:use [org.apache.storm testing MockAutoCred util config log timer zookeeper])
+  (:import [org.apache.storm.cluster StormZkClusterState ClusterStateContext Cluster])
+  (:use [org.apache.storm testing MockAutoCred util config log timer converter])
   (:use [org.apache.storm.daemon common])
   (:require [conjure.core])
   (:require [org.apache.storm
-             [thrift :as thrift]
-             [cluster :as cluster]])
+             [thrift :as thrift]])
   (:use [conjure core]))
 
 (defn storm-component->task-info [cluster storm-name]
@@ -51,7 +53,7 @@
 
 (defn getCredentials [cluster storm-name]
   (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)]
-    (.credentials (:storm-cluster-state cluster) storm-id nil)))
+    (clojurify-crdentials (.credentials (:storm-cluster-state cluster) storm-id nil))))
 
 (defn storm-component->executor-info [cluster storm-name]
   (let [storm-id (get-storm-id (:storm-cluster-state cluster) storm-name)
@@ -61,7 +63,7 @@
         task->component (storm-task-info topology storm-conf)
         state (:storm-cluster-state cluster)
         get-component (comp task->component first)]
-    (->> (.assignment-info state storm-id nil)
+    (->> (clojurify-assignment (.assignmentInfo state storm-id nil))
          :executor->node+port
          keys
          (map (fn [e] {e (get-component e)}))
@@ -70,13 +72,13 @@
 
 (defn storm-num-workers [state storm-name]
   (let [storm-id (get-storm-id state storm-name)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (count (reverse-map (:executor->node+port assignment)))
     ))
 
 (defn topology-nodes [state storm-name]
   (let [storm-id (get-storm-id state storm-name)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (->> assignment
          :executor->node+port
          vals
@@ -86,7 +88,7 @@
 
 (defn topology-slots [state storm-name]
   (let [storm-id (get-storm-id state storm-name)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (->> assignment
          :executor->node+port
          vals
@@ -95,7 +97,7 @@
 
 (defn topology-node-distribution [state storm-name]
   (let [storm-id (get-storm-id state storm-name)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (->> assignment
          :executor->node+port
          vals
@@ -111,28 +113,28 @@
 
 (defn executor-assignment [cluster storm-id executor-id]
   (let [state (:storm-cluster-state cluster)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     ((:executor->node+port assignment) executor-id)
     ))
 
 (defn executor-start-times [cluster storm-id]
   (let [state (:storm-cluster-state cluster)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (:executor->start-time-secs assignment)))
 
 (defn do-executor-heartbeat [cluster storm-id executor]
   (let [state (:storm-cluster-state cluster)
-        executor->node+port (:executor->node+port (.assignment-info state storm-id nil))
+        executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
         [node port] (get executor->node+port executor)
-        curr-beat (.get-worker-heartbeat state storm-id node port)
+        curr-beat (clojurify-zk-worker-hb (.getworkerHeartbeat state storm-id node port))
         stats (:executor-stats curr-beat)]
-    (.worker-heartbeat! state storm-id node port
-      {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})}
+    (.workerHeartbeat state storm-id node port
+      (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
       )))
 
 (defn slot-assignments [cluster storm-id]
   (let [state (:storm-cluster-state cluster)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (reverse-map (:executor->node+port assignment))
     ))
 
@@ -144,7 +146,7 @@
 
 (defn topology-executors [cluster storm-id]
   (let [state (:storm-cluster-state cluster)
-        assignment (.assignment-info state storm-id nil)]
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))]
     (keys (:executor->node+port assignment))
     ))
 
@@ -162,7 +164,7 @@
   (let [state (:storm-cluster-state cluster)
         storm-id (get-storm-id state storm-name)
         task-ids (task-ids cluster storm-id)
-        assignment (.assignment-info state storm-id nil)
+        assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
         executor->node+port (:executor->node+port assignment)
         task->node+port (to-task->node+port executor->node+port)
         assigned-task-ids (mapcat executor-id->tasks (keys executor->node+port))
@@ -419,54 +421,54 @@
         (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20, LOGS-USERS ["alice", (System/getProperty "user.name")]} topology)
         (bind storm-id (get-storm-id state "test"))
         (advance-cluster-time cluster 5)
-        (is (not-nil? (.storm-base state storm-id nil)))
-        (is (not-nil? (.assignment-info state storm-id nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
         (.killTopology (:nimbus cluster) "test")
         ;; check that storm is deactivated but alive
-        (is (= :killed (-> (.storm-base state storm-id nil) :status :type)))
-        (is (not-nil? (.assignment-info state storm-id nil)))
+        (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id nil)) :status :type)))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
         (advance-cluster-time cluster 35)
         ;; kill topology read on group
         (submit-local-topology (:nimbus cluster) "killgrouptest" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20, LOGS-GROUPS ["alice-group"]} topology)
         (bind storm-id-killgroup (get-storm-id state "killgrouptest"))
         (advance-cluster-time cluster 5)
-        (is (not-nil? (.storm-base state storm-id-killgroup nil)))
-        (is (not-nil? (.assignment-info state storm-id-killgroup nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id-killgroup nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killgroup nil))))
         (.killTopology (:nimbus cluster) "killgrouptest")
         ;; check that storm is deactivated but alive
-        (is (= :killed (-> (.storm-base state storm-id-killgroup nil) :status :type)))
-        (is (not-nil? (.assignment-info state storm-id-killgroup nil)))
+        (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id-killgroup nil)) :status :type)))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killgroup nil))))
         (advance-cluster-time cluster 35)
         ;; kill topology can't read
         (submit-local-topology (:nimbus cluster) "killnoreadtest" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
         (bind storm-id-killnoread (get-storm-id state "killnoreadtest"))
         (advance-cluster-time cluster 5)
-        (is (not-nil? (.storm-base state storm-id-killnoread nil)))
-        (is (not-nil? (.assignment-info state storm-id-killnoread nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id-killnoread nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killnoread nil))))
         (.killTopology (:nimbus cluster) "killnoreadtest")
         ;; check that storm is deactivated but alive
-        (is (= :killed (-> (.storm-base state storm-id-killnoread nil) :status :type)))
-        (is (not-nil? (.assignment-info state storm-id-killnoread nil)))
+        (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id-killnoread nil)) :status :type)))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id-killnoread nil))))
         (advance-cluster-time cluster 35)
 
         ;; active topology can read
         (submit-local-topology (:nimbus cluster) "2test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-USERS ["alice", (System/getProperty "user.name")]} topology)
         (advance-cluster-time cluster 11)
         (bind storm-id2 (get-storm-id state "2test"))
-        (is (not-nil? (.storm-base state storm-id2 nil)))
-        (is (not-nil? (.assignment-info state storm-id2 nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id2 nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id2 nil))))
         ;; active topology can not read
         (submit-local-topology (:nimbus cluster) "testnoread" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-USERS ["alice"]} topology)
         (advance-cluster-time cluster 11)
         (bind storm-id3 (get-storm-id state "testnoread"))
-        (is (not-nil? (.storm-base state storm-id3 nil)))
-        (is (not-nil? (.assignment-info state storm-id3 nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id3 nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id3 nil))))
         ;; active topology can read based on group
         (submit-local-topology (:nimbus cluster) "testreadgroup" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 10, LOGS-GROUPS ["alice-group"]} topology)
         (advance-cluster-time cluster 11)
         (bind storm-id4 (get-storm-id state "testreadgroup"))
-        (is (not-nil? (.storm-base state storm-id4 nil)))
-        (is (not-nil? (.assignment-info state storm-id4 nil)))
+        (is (not-nil? (clojurify-storm-base (.stormBase state storm-id4 nil))))
+        (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
         ;; at this point have 1 running, 1 killed topo
         (let [hist-topo-ids (vec (sort (.get_topo_ids (.getTopologyHistory (:nimbus cluster) (System/getProperty "user.name")))))]
           (log-message "Checking user " (System/getProperty "user.name") " " hist-topo-ids)
@@ -515,22 +517,22 @@
       (submit-local-topology (:nimbus cluster) "test" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 20} topology)
       (bind storm-id (get-storm-id state "test"))
       (advance-cluster-time cluster 15)
-      (is (not-nil? (.storm-base state storm-id nil)))
-      (is (not-nil? (.assignment-info state storm-id nil)))
+      (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+      (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
       (.killTopology (:nimbus cluster) "test")
       ;; check that storm is deactivated but alive
-      (is (= :killed (-> (.storm-base state storm-id nil) :status :type)))
-      (is (not-nil? (.assignment-info state storm-id nil)))
+      (is (= :killed (-> (clojurify-storm-base (.stormBase state storm-id nil)) :status :type)))
+      (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
       (advance-cluster-time cluster 18)
       ;; check that storm is deactivated but alive
-      (is (= 1 (count (.heartbeat-storms state))))
+      (is (= 1 (count (.heartbeatStorms state))))
       (advance-cluster-time cluster 3)
-      (is (nil? (.storm-base state storm-id nil)))
-      (is (nil? (.assignment-info state storm-id nil)))
+      (is (nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+      (is (nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
 
       ;; cleanup happens on monitoring thread
       (advance-cluster-time cluster 11)
-      (is (empty? (.heartbeat-storms state)))
+      (is (empty? (.heartbeatStorms state)))
       ;; TODO: check that code on nimbus was cleaned up locally...
 
       (is (thrown? NotAliveException (.killTopology (:nimbus cluster) "lalala")))
@@ -539,27 +541,27 @@
       (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
       (advance-cluster-time cluster 11)
       (bind storm-id (get-storm-id state "2test"))
-      (is (not-nil? (.storm-base state storm-id nil)))
+      (is (not-nil? (clojurify-storm-base (.stormBase state storm-id nil))))
       (.killTopology (:nimbus cluster) "2test")
       (is (thrown? AlreadyAliveException (submit-local-topology (:nimbus cluster) "2test" {} topology)))
       (advance-cluster-time cluster 11)
-      (is (= 1 (count (.heartbeat-storms state))))
+      (is (= 1 (count (.heartbeatStorms state))))
 
       (advance-cluster-time cluster 6)
-      (is (nil? (.storm-base state storm-id nil)))
-      (is (nil? (.assignment-info state storm-id nil)))
+      (is (nil? (clojurify-storm-base (.stormBase state storm-id nil))))
+      (is (nil? (clojurify-assignment (.assignmentInfo state storm-id nil))))
       (advance-cluster-time cluster 11)
-      (is (= 0 (count (.heartbeat-storms state))))
+      (is (= 0 (count (.heartbeatStorms state))))
 
       (submit-local-topology (:nimbus cluster) "test3" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 5} topology)
       (bind storm-id3 (get-storm-id state "test3"))
       (advance-cluster-time cluster 11)
-      (.remove-storm! state storm-id3)
-      (is (nil? (.storm-base state storm-id3 nil)))
-      (is (nil? (.assignment-info state storm-id3 nil)))
+      (.removeStorm state storm-id3)
+      (is (nil? (clojurify-storm-base (.stormBase state storm-id3 nil))))
+      (is (nil? (clojurify-assignment (.assignmentInfo state storm-id3 nil))))
 
       (advance-cluster-time cluster 11)
-      (is (= 0 (count (.heartbeat-storms state))))
+      (is (= 0 (count (.heartbeatStorms state))))
 
       ;; this guarantees that monitor thread won't trigger for 10 more seconds
       (advance-time-secs! 11)
@@ -575,9 +577,9 @@
 
       (.killTopology (:nimbus cluster) "test3")
       (advance-cluster-time cluster 6)
-      (is (= 1 (count (.heartbeat-storms state))))
+      (is (= 1 (count (.heartbeatStorms state))))
       (advance-cluster-time cluster 5)
-      (is (= 0 (count (.heartbeat-storms state))))
+      (is (= 0 (count (.heartbeatStorms state))))
 
       ;; test kill with opts
       (submit-local-topology (:nimbus cluster) "test4" {TOPOLOGY-MESSAGE-TIMEOUT-SECS 100} topology)
@@ -585,9 +587,9 @@
       (.killTopologyWithOpts (:nimbus cluster) "test4" (doto (KillOptions.) (.set_wait_secs 10)))
       (bind storm-id4 (get-storm-id state "test4"))
       (advance-cluster-time cluster 9)
-      (is (not-nil? (.assignment-info state storm-id4 nil)))
+      (is (not-nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
       (advance-cluster-time cluster 2)
-      (is (nil? (.assignment-info state storm-id4 nil)))
+      (is (nil? (clojurify-assignment (.assignmentInfo state storm-id4 nil))))
       )))
 
 (deftest test-reassignment
@@ -906,7 +908,7 @@
  (let [assignments (.assignments state nil)]
    (log-message "Assignemts: " assignments)
    (let [id->node->ports (into {} (for [id assignments
-                                                :let [executor->node+port (:executor->node+port (.assignment-info state id nil))
+                                                :let [executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state id nil)))
                                                       node+ports (set (.values executor->node+port))
                                                       node->ports (apply merge-with (fn [a b] (distinct (concat a b))) (for [[node port] node+ports] {node [port]}))]]
                                                 {id node->ports}))
@@ -1029,7 +1031,7 @@
                             STORM-CLUSTER-MODE "local"
                             STORM-ZOOKEEPER-PORT zk-port
                             STORM-LOCAL-DIR nimbus-dir}))
-         (bind cluster-state (cluster/mk-storm-cluster-state conf))
+         (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
          (bind topology (thrift/mk-topology
                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1043,7 +1045,7 @@
            (nimbus/blob-rm-topology-keys storm-id1 blob-store cluster-state)
            (.shutdown blob-store))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
-         (is ( = #{storm-id2} (set (.active-storms cluster-state))))
+         (is ( = #{storm-id2} (set (.activeStorms cluster-state))))
          (.shutdown nimbus)
          (.disconnect cluster-state)
          )))))
@@ -1101,7 +1103,7 @@
                         STORM-CLUSTER-MODE "local"
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir}))
-          (bind cluster-state (cluster/mk-storm-cluster-state conf))
+          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind topology (thrift/mk-topology
                            {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1111,7 +1113,7 @@
                           (zkLeaderElectorImpl [conf] (mock-leader-elector :is-leader false))))]
 
             (letlocals
-              (bind non-leader-cluster-state (cluster/mk-storm-cluster-state conf))
+              (bind non-leader-cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
               (bind non-leader-nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
 
               ;first we verify that the master nimbus can perform all actions, even with another nimbus present.
@@ -1347,13 +1349,15 @@
                      STORM-PRINCIPAL-TO-LOCAL-PLUGIN "org.apache.storm.security.auth.DefaultPrincipalToLocal"
                      NIMBUS-THRIFT-PORT 6666})
           expected-acls nimbus/NIMBUS-ZK-ACLS
-          fake-inimbus (reify INimbus (getForcedScheduler [this] nil))]
+          fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
+          storm-zk (Mockito/mock Cluster)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (nimbusTopoHistoryStateImpl [conf] nil))
                   zk-le (MockedZookeeper. (proxy [Zookeeper] []
-                          (zkLeaderElectorImpl [conf] nil)))]
+                          (zkLeaderElectorImpl [conf] nil)))
+                  storm-zk-le (MockedCluster. storm-zk)]
         (stubbing [mk-authorization-handler nil
-                 cluster/mk-storm-cluster-state nil
+               ;  cluster/mk-storm-cluster-state nil
                  nimbus/file-cache-map nil
                  nimbus/mk-blob-cache-map nil
                  nimbus/mk-bloblist-cache-map nil
@@ -1362,9 +1366,11 @@
                  mk-timer nil
                  nimbus/mk-scheduler nil]
           (nimbus/nimbus-data auth-conf fake-inimbus)
-          (verify-call-times-for cluster/mk-storm-cluster-state 1)
-          (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-                                              expected-acls))))))
+          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+          ;   (verify-call-times-for cluster/mk-storm-cluster-state 1)
+       ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+       ; expected-acls)
+          )))))
 
 (deftest test-file-bogus-download
   (with-local-cluster [cluster :daemon-conf {SUPERVISOR-ENABLE false TOPOLOGY-ACKER-EXECUTORS 0 TOPOLOGY-EVENTLOGGER-EXECUTORS 0}]
@@ -1395,7 +1401,7 @@
                       STORM-CLUSTER-MODE "local"
                       STORM-ZOOKEEPER-PORT zk-port
                       STORM-LOCAL-DIR nimbus-dir}))
-        (bind cluster-state (cluster/mk-storm-cluster-state conf))
+        (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
         (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
         (sleep-secs 1)
         (bind topology (thrift/mk-topology
@@ -1427,7 +1433,7 @@
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir
                         NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
-          (bind cluster-state (cluster/mk-storm-cluster-state conf))
+          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind notifier (InMemoryTopologyActionNotifier.))
           (sleep-secs 1)

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
index 361c4be..307296a 100644
--- a/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
+++ b/storm-core/test/clj/org/apache/storm/security/auth/nimbus_auth_test.clj
@@ -17,7 +17,6 @@
   (:use [clojure test])
   (:require [org.apache.storm [testing :as testing]])
   (:require [org.apache.storm.daemon [nimbus :as nimbus]])
-  (:require [org.apache.storm [zookeeper :as zk]])
   (:require [org.apache.storm.security.auth [auth-test :refer [nimbus-timeout]]])
   (:import [java.nio ByteBuffer])
   (:import [org.apache.storm Config])
@@ -25,7 +24,7 @@
   (:import [org.apache.storm.generated NotAliveException])
   (:import [org.apache.storm.security.auth AuthUtils ThriftServer ThriftClient 
                                          ReqContext ThriftConnectionType])
-  (:use [org.apache.storm cluster util config log])
+  (:use [org.apache.storm util config log])
   (:use [org.apache.storm.daemon common nimbus])
   (:import [org.apache.storm.generated Nimbus Nimbus$Client Nimbus$Processor 
             AuthorizationException SubmitOptions TopologyInitialStatus KillOptions])


[21/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/3717f393
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/3717f393
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/3717f393

Branch: refs/heads/master
Commit: 3717f3931e51825f16739cbed138e9789b7d1428
Parents: b7bc9bf 4ca7522
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Sun Feb 21 12:59:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Sun Feb 21 12:59:50 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |  3 +
 external/storm-hdfs/pom.xml                     | 23 ++---
 .../clj/org/apache/storm/daemon/supervisor.clj  | 48 +++++++---
 storm-core/src/clj/org/apache/storm/event.clj   | 71 --------------
 storm-core/src/clj/org/apache/storm/ui/core.clj |  6 +-
 .../org/apache/storm/event/EventManager.java    | 24 +++++
 .../org/apache/storm/event/EventManagerImp.java | 97 ++++++++++++++++++++
 7 files changed, 173 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/3717f393/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index f429d09,21e5854..a6f3a62
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -34,7 -35,7 +35,7 @@@
    (:use [org.apache.storm.daemon common])
    (:import [org.apache.storm.command HealthCheck])
    (:require [org.apache.storm.daemon [worker :as worker]]
-             [org.apache.storm [process-simulator :as psim] [event :as event]]
 -            [org.apache.storm [process-simulator :as psim] [cluster :as cluster]]
++            [org.apache.storm [process-simulator :as psim]]
              [clojure.set :as set])
    (:import [org.apache.thrift.transport TTransportException])
    (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])

http://git-wip-us.apache.org/repos/asf/storm/blob/3717f393/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------


[05/27] storm git commit: callback maybe null

Posted by bo...@apache.org.
callback maybe null


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/5916b0b8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/5916b0b8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/5916b0b8

Branch: refs/heads/master
Commit: 5916b0b8089f9dd184fdd1ab2f18eb5e5deabc65
Parents: 9a79fb7
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 21:15:41 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 21:15:41 2016 +0800

----------------------------------------------------------------------
 .../src/jvm/org/apache/storm/cluster/StormZkClusterState.java  | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/5916b0b8/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
index 3f32fe1..3a4205b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
@@ -148,12 +148,14 @@ public class StormZkClusterState implements StormClusterState {
 
     protected void issueCallback(AtomicReference<IFn> cb) {
         IFn callback = cb.getAndSet(null);
-        callback.invoke();
+        if (callback != null)
+            callback.invoke();
     }
 
     protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
         IFn callback = callbackConcurrentHashMap.remove(key);
-        callback.invoke();
+        if (callback != null)
+            callback.invoke();
     }
 
     @Override


[25/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/0af67994
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/0af67994
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/0af67994

Branch: refs/heads/master
Commit: 0af6799411c747a6c900438ded222b7d757c7961
Parents: b09e275 11232b5
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 24 11:20:00 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 24 11:20:00 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   4 +-
 README.markdown                                 |   3 +-
 .../org/apache/storm/kafka/IntSerializer.java   |  10 +-
 .../src/jvm/org/apache/storm/utils/Time.java    |   1 +
 .../src/jvm/org/apache/storm/utils/Utils.java   |  17 +-
 .../test/clj/org/apache/storm/utils_test.clj    | 111 ----------
 .../jvm/org/apache/storm/utils/TimeTest.java    | 112 ++++++++++
 .../jvm/org/apache/storm/utils/UtilsTest.java   | 219 +++++++++++++++++++
 8 files changed, 353 insertions(+), 124 deletions(-)
----------------------------------------------------------------------



[20/27] storm git commit: resolve my little fault

Posted by bo...@apache.org.
resolve my little fault


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b7bc9bf9
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b7bc9bf9
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b7bc9bf9

Branch: refs/heads/master
Commit: b7bc9bf9bd0793f2acbf158651e3200b30f97d99
Parents: cc1f6d7
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Sun Feb 21 10:30:05 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Sun Feb 21 10:30:05 2016 +0800

----------------------------------------------------------------------
 storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b7bc9bf9/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index aae4231..1095fff 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -94,7 +94,7 @@ public class ClusterUtils {
     public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
         List<ACL> aclList = null;
         String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
-        if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
+        if (Utils.isZkAuthenticationConfiguredTopology(topoConf)) {
             aclList = new ArrayList<>();
             ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
             aclList.add(acl1);


[17/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d5463879
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d5463879
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d5463879

Branch: refs/heads/master
Commit: d5463879c0441a5a57eb23653a70bcf95d2efbaf
Parents: e0f3cb5 4699990
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 17 11:28:10 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 17 11:28:10 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |   8 +
 bin/storm-config.cmd                            |   4 +
 bin/storm.cmd                                   |  22 +-
 bin/storm.py                                    |   8 +-
 .../spout/RandomNumberGeneratorSpout.java       |  95 +++++
 .../trident/TridentMinMaxOfDevicesTopology.java | 201 +++++++++++
 .../TridentMinMaxOfVehiclesTopology.java        | 180 ++++++++++
 pom.xml                                         |   6 +
 storm-core/pom.xml                              |   9 +
 .../clj/org/apache/storm/command/activate.clj   |  24 --
 .../clj/org/apache/storm/command/deactivate.clj |  24 --
 .../org/apache/storm/command/kill_topology.clj  |  29 --
 .../src/clj/org/apache/storm/command/list.clj   |  38 --
 .../clj/org/apache/storm/daemon/executor.clj    |  68 ++--
 .../src/clj/org/apache/storm/daemon/worker.clj  |  54 +--
 .../src/clj/org/apache/storm/disruptor.clj      |  89 -----
 .../jvm/org/apache/storm/command/Activate.java  |  40 +++
 .../src/jvm/org/apache/storm/command/CLI.java   | 353 +++++++++++++++++++
 .../org/apache/storm/command/Deactivate.java    |  40 +++
 .../org/apache/storm/command/KillTopology.java  |  51 +++
 .../src/jvm/org/apache/storm/command/List.java  |  50 +++
 .../jvm/org/apache/storm/trident/Stream.java    | 121 ++++++-
 .../operation/builtin/ComparisonAggregator.java |  91 +++++
 .../storm/trident/operation/builtin/Max.java    |  37 ++
 .../operation/builtin/MaxWithComparator.java    |  51 +++
 .../storm/trident/operation/builtin/Min.java    |  36 ++
 .../operation/builtin/MinWithComparator.java    |  51 +++
 .../org/apache/storm/utils/DisruptorQueue.java  |  15 +-
 .../org/apache/storm/utils/NimbusClient.java    |  19 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   |  20 +-
 .../jvm/org/apache/storm/command/TestCLI.java   |  59 ++++
 31 files changed, 1593 insertions(+), 300 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d5463879/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 33b89ed,3af365b..902650c
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -38,8 -38,10 +38,9 @@@
    (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
    (:import [java.lang Thread Thread$UncaughtExceptionHandler]
             [java.util.concurrent ConcurrentLinkedQueue]
-            [org.json.simple JSONValue])
-   (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
+            [org.json.simple JSONValue]
+            [com.lmax.disruptor.dsl ProducerType])
 -  (:require [org.apache.storm [thrift :as thrift]
 -             [cluster :as cluster] [stats :as stats]])
++  (:require [org.apache.storm [thrift :as thrift] [stats :as stats]])
    (:require [org.apache.storm.daemon [task :as task]])
    (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
    (:require [clojure.set :as set]))

http://git-wip-us.apache.org/repos/asf/storm/blob/d5463879/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index 9863427,83ae9be..b80cd9e
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,7 -19,7 +19,7 @@@
    (:require [clj-time.core :as time])
    (:require [clj-time.coerce :as coerce])
    (:require [org.apache.storm.daemon [executor :as executor]])
-   (:require [org.apache.storm [disruptor :as disruptor]])
 -  (:require [org.apache.storm [cluster :as cluster]])
++
    (:require [clojure.set :as set])
    (:import [java.util.concurrent Executors]
             [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
@@@ -244,13 -244,14 +244,14 @@@
                         )
              :timer-name timer-name))
  
 -(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf cluster-state storm-cluster-state]
 +(defn worker-data [conf mq-context storm-id assignment-id port worker-id storm-conf state-store storm-cluster-state]
    (let [assignment-versions (atom {})
          executors (set (read-worker-executors storm-conf storm-cluster-state storm-id assignment-id port assignment-versions))
-         transfer-queue (disruptor/disruptor-queue "worker-transfer-queue" (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
+         transfer-queue (DisruptorQueue. "worker-transfer-queue"
+                                                   (storm-conf TOPOLOGY-TRANSFER-BUFFER-SIZE)
                                                    (storm-conf TOPOLOGY-DISRUPTOR-WAIT-TIMEOUT-MILLIS)
-                                                   :batch-size (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
-                                                   :batch-timeout (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
+                                                   (storm-conf TOPOLOGY-DISRUPTOR-BATCH-SIZE)
+                                                   (storm-conf TOPOLOGY-DISRUPTOR-BATCH-TIMEOUT-MILLIS))
          executor-receive-queue-map (mk-receive-queue-map storm-conf executors)
  
          receive-queue-map (->> executor-receive-queue-map


[11/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a8962de/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --cc storm-core/test/clj/org/apache/storm/supervisor_test.clj
index b89b7bb,9c31ddf..3ebdbcd
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@@ -21,20 -21,20 +21,23 @@@
    (:require [clojure [string :as string] [set :as set]])
    (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount TestAggregatesCounter TestPlannerSpout])
    (:import [org.apache.storm.scheduler ISupervisor])
-   (:import [org.apache.storm.utils ConfigUtils])
+   (:import [org.apache.storm.utils Time Utils$UptimeComputer ConfigUtils])
    (:import [org.apache.storm.generated RebalanceOptions])
-   (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
 -  (:import [org.mockito Matchers Mockito])
++  (:import [org.apache.storm.testing.staticmocking MockedCluster])
    (:import [java.util UUID])
-   (:import [org.mockito Mockito])
++  (:import [org.mockito Mockito Matchers])
 +  (:import [org.mockito.exceptions.base MockitoAssertionError])
    (:import [java.io File])
    (:import [java.nio.file Files])
-   (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
 -  (:import [org.apache.storm.utils Utils IPredicate]
++  (:import [org.apache.storm.utils Utils IPredicate])
++  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils]
+            [org.apache.storm.utils.staticmocking ConfigUtilsInstaller
+                                                  UtilsInstaller])
    (:import [java.nio.file.attribute FileAttribute])
 -  (:use [org.apache.storm config testing util timer log])
 +  (:use [org.apache.storm config testing util timer log converter])
    (:use [org.apache.storm.daemon common])
    (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
 -            [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
 +            [org.apache.storm [thrift :as thrift]])
    (:use [conjure core])
    (:require [clojure.java.io :as io]))
  
@@@ -43,9 -43,10 +46,10 @@@
    [cluster supervisor-id port]
    (let [state (:storm-cluster-state cluster)
          slot-assigns (for [storm-id (.assignments state nil)]
 -                        (let [executors (-> (.assignment-info state storm-id nil)
 +                        (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
                                          :executor->node+port
-                                         reverse-map
+                                         (Utils/reverseMap)
+                                         clojurify-structure
                                          (get [supervisor-id port] ))]
                            (when executors [storm-id executors])
                            ))
@@@ -565,198 -632,203 +635,201 @@@
            fake-isupervisor (reify ISupervisor
                               (getSupervisorId [this] nil)
                               (getAssignmentId [this] nil))
+           fake-cu (proxy [ConfigUtils] []
+                     (supervisorStateImpl [conf] nil)
+                     (supervisorLocalDirImpl [conf] nil))
+           fake-utils (proxy [Utils] []
+                        (localHostnameImpl [] nil)
+                        (makeUptimeComputer [] (proxy [Utils$UptimeComputer] []
 -                                                (upTime [] 0))))]
++                                                (upTime [] 0))))
 +          cluster-utils (Mockito/mock ClusterUtils)]
-       (with-open [_ (proxy [MockedConfigUtils] []
-                       (supervisorStateImpl [conf] nil)
-                       (supervisorLocalDirImpl [conf] nil))
+       (with-open [_ (ConfigUtilsInstaller. fake-cu)
 -                  _ (UtilsInstaller. fake-utils)]
 -        (stubbing [cluster/mk-storm-cluster-state nil
 -                   mk-timer nil]
++                  _ (UtilsInstaller. fake-utils)
 +                  mocked-cluster (MockedCluster. cluster-utils)]
-         (stubbing [uptime-computer nil
-               ;   cluster/mk-storm-cluster-state nil
-                  local-hostname nil
-                  mk-timer nil]
            (supervisor/supervisor-data auth-conf nil fake-isupervisor)
-           (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
-         ;  (verify-call-times-for cluster/mk-storm-cluster-state 1)
-         ;  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-         ;                                     expected-acls)
-          )))))
 -          (verify-call-times-for cluster/mk-storm-cluster-state 1)
 -          (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
 -              expected-acls)))))
++          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))))))
  
- (deftest test-write-log-metadata
-   (testing "supervisor writes correct data to logs metadata file"
-     (let [exp-owner "alice"
-           exp-worker-id "42"
-           exp-storm-id "0123456789"
-           exp-port 4242
-           exp-logs-users ["bob" "charlie" "daryl"]
-           exp-logs-groups ["read-only-group" "special-group"]
-           storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
-                       TOPOLOGY-USERS ["charlie" "bob"]
-                       TOPOLOGY-GROUPS ["special-group"]
-                       LOGS-GROUPS ["read-only-group"]
-                       LOGS-USERS ["daryl"]}
-           exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
-                     "worker-id" exp-worker-id
-                     LOGS-USERS exp-logs-users
-                     LOGS-GROUPS exp-logs-groups}
-           conf {}]
-       (mocking [supervisor/write-log-metadata-to-yaml-file!]
-         (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
-                                         exp-storm-id exp-port conf)
-         (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
-                                       exp-storm-id exp-port exp-data conf)))))
+   (deftest test-write-log-metadata
+     (testing "supervisor writes correct data to logs metadata file"
+       (let [exp-owner "alice"
+             exp-worker-id "42"
+             exp-storm-id "0123456789"
+             exp-port 4242
+             exp-logs-users ["bob" "charlie" "daryl"]
+             exp-logs-groups ["read-only-group" "special-group"]
+             storm-conf {TOPOLOGY-SUBMITTER-USER "alice"
+                         TOPOLOGY-USERS ["charlie" "bob"]
+                         TOPOLOGY-GROUPS ["special-group"]
+                         LOGS-GROUPS ["read-only-group"]
+                         LOGS-USERS ["daryl"]}
+             exp-data {TOPOLOGY-SUBMITTER-USER exp-owner
+                       "worker-id" exp-worker-id
+                       LOGS-USERS exp-logs-users
+                       LOGS-GROUPS exp-logs-groups}
+             conf {}]
+         (mocking [supervisor/write-log-metadata-to-yaml-file!]
+           (supervisor/write-log-metadata! storm-conf exp-owner exp-worker-id
+             exp-storm-id exp-port conf)
+           (verify-called-once-with-args supervisor/write-log-metadata-to-yaml-file!
+             exp-storm-id exp-port exp-data conf)))))
  
- (deftest test-worker-launcher-requires-user
-   (testing "worker-launcher throws on blank user"
-     (mocking [launch-process]
-       (is (thrown-cause-with-msg? java.lang.IllegalArgumentException
-                                   #"(?i).*user cannot be blank.*"
-                                   (supervisor/worker-launcher {} nil ""))))))
+   (deftest test-worker-launcher-requires-user
+     (testing "worker-launcher throws on blank user"
+       (let [utils-proxy (proxy [Utils] []
+                           (launchProcessImpl [& _] nil))]
+         (with-open [_ (UtilsInstaller. utils-proxy)]
+           (is (try
+                 (supervisor/worker-launcher {} nil "")
+                 false
+                 (catch Throwable t
+                   (and (re-matches #"(?i).*user cannot be blank.*" (.getMessage t))
+                        (Utils/exceptionCauseIsInstanceOf java.lang.IllegalArgumentException t)))))))))
  
- (defn found? [sub-str input-str]
-   (if (string? input-str)
-     (contrib-str/substring? sub-str (str input-str))
-     (boolean (some #(contrib-str/substring? sub-str %) input-str))))
+   (defn found? [sub-str input-str]
+     (if (string? input-str)
+       (contrib-str/substring? sub-str (str input-str))
+       (boolean (some #(contrib-str/substring? sub-str %) input-str))))
  
- (defn not-found? [sub-str input-str]
+   (defn not-found? [sub-str input-str]
      (complement (found? sub-str input-str)))
  
- (deftest test-substitute-childopts-happy-path-string
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-string
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log -Xms256m -Xmx%HEAP-MEM%m"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-happy-path-list
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-list
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m")
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-happy-path-list-arraylist
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-happy-path-list-arraylist
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts '["-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log" "-Xms256m" "-Xmx%HEAP-MEM%m"]
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01-w-01-9999.log" "-Xms256m" "-Xmx512m")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-topology-id-alone
-   (testing "worker-launcher replaces ids in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-topology-id-alone
+     (testing "worker-launcher replaces ids in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%TOPOLOGY-ID%.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-s-01.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-no-keys
-   (testing "worker-launcher has no ids to replace in childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-no-keys
+     (testing "worker-launcher has no ids to replace in childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-nil-childopts
-   (testing "worker-launcher has nil childopts"
-     (let [worker-id "w-01"
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts nil
-           expected-childopts nil
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-nil-childopts
+     (testing "worker-launcher has nil childopts"
+       (let [worker-id "w-01"
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts nil
+             expected-childopts nil
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-substitute-childopts-nil-ids
-   (testing "worker-launcher has nil ids"
-     (let [worker-id nil
-           topology-id "s-01"
-           port 9999
-           mem-onheap 512
-           childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
-           expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
-           childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
-       (is (= expected-childopts childopts-with-ids)))))
+   (deftest test-substitute-childopts-nil-ids
+     (testing "worker-launcher has nil ids"
+       (let [worker-id nil
+             topology-id "s-01"
+             port 9999
+             mem-onheap 512
+             childopts "-Xloggc:/home/y/lib/storm/current/logs/gc.worker-%ID%-%TOPOLOGY-ID%-%WORKER-ID%-%WORKER-PORT%.log"
+             expected-childopts '("-Xloggc:/home/y/lib/storm/current/logs/gc.worker-9999-s-01--9999.log")
+             childopts-with-ids (supervisor/substitute-childopts childopts worker-id topology-id port mem-onheap)]
+         (is (= expected-childopts childopts-with-ids)))))
  
- (deftest test-retry-read-assignments
-   (with-simulated-time-local-cluster [cluster
-                                       :supervisors 0
-                                       :ports-per-supervisor 2
-                                       :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
-                                                     NIMBUS-MONITOR-FREQ-SECS 10
-                                                     TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
-                                                     TOPOLOGY-ACKER-EXECUTORS 0}]
-     (letlocals
-      (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
-      (bind topology1 (thrift/mk-topology
-                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
-                       {}))
-      (bind topology2 (thrift/mk-topology
-                       {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
-                       {}))
-      (bind state (:storm-cluster-state cluster))
-      (bind changed (capture-changed-workers
-                     (submit-mocked-assignment
-                      (:nimbus cluster)
-                      (:storm-cluster-state cluster)
-                      "topology1"
-                      {TOPOLOGY-WORKERS 2}
-                      topology1
-                      {1 "1"
-                       2 "1"}
-                      {[1 1] ["sup1" 1]
-                       [2 2] ["sup1" 2]}
-                      {["sup1" 1] [0.0 0.0 0.0]
-                       ["sup1" 2] [0.0 0.0 0.0]
-                       })
-                     (submit-mocked-assignment
-                      (:nimbus cluster)
-                      (:storm-cluster-state cluster)
-                      "topology2"
-                      {TOPOLOGY-WORKERS 2}
-                      topology2
-                      {1 "1"
-                       2 "1"}
-                      {[1 1] ["sup1" 1]
-                       [2 2] ["sup1" 2]}
-                      {["sup1" 1] [0.0 0.0 0.0]
-                       ["sup1" 2] [0.0 0.0 0.0]
-                       })
-                     ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
-                     (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
-                     ))
-      (is (empty? (:launched changed)))
-      (bind options (RebalanceOptions.))
-      (.set_wait_secs options 0)
-      (bind changed (capture-changed-workers
-                     (.rebalance (:nimbus cluster) "topology2" options)
-                     (advance-cluster-time cluster 10)
-                     (heartbeat-workers cluster "sup1" [1 2 3 4])
-                     (advance-cluster-time cluster 10)
-                     ))
-      (validate-launched-once (:launched changed)
-                              {"sup1" [1 2]}
-                              (get-storm-id (:storm-cluster-state cluster) "topology1"))
-      (validate-launched-once (:launched changed)
-                              {"sup1" [3 4]}
-                              (get-storm-id (:storm-cluster-state cluster) "topology2"))
-      )))
+   (deftest test-retry-read-assignments
+     (with-simulated-time-local-cluster [cluster
+                                         :supervisors 0
+                                         :ports-per-supervisor 2
+                                         :daemon-conf {ConfigUtils/NIMBUS_DO_NOT_REASSIGN true
+                                                       NIMBUS-MONITOR-FREQ-SECS 10
+                                                       TOPOLOGY-MESSAGE-TIMEOUT-SECS 30
+                                                       TOPOLOGY-ACKER-EXECUTORS 0}]
+       (letlocals
+         (bind sup1 (add-supervisor cluster :id "sup1" :ports [1 2 3 4]))
+         (bind topology1 (thrift/mk-topology
+                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+                           {}))
+         (bind topology2 (thrift/mk-topology
+                           {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 2)}
+                           {}))
+         (bind state (:storm-cluster-state cluster))
+         (bind changed (capture-changed-workers
+                         (submit-mocked-assignment
+                           (:nimbus cluster)
+                           (:storm-cluster-state cluster)
+                           "topology1"
+                           {TOPOLOGY-WORKERS 2}
+                           topology1
+                           {1 "1"
+                            2 "1"}
+                           {[1 1] ["sup1" 1]
+                            [2 2] ["sup1" 2]}
+                           {["sup1" 1] [0.0 0.0 0.0]
+                            ["sup1" 2] [0.0 0.0 0.0]
+                            })
+                         (submit-mocked-assignment
+                           (:nimbus cluster)
+                           (:storm-cluster-state cluster)
+                           "topology2"
+                           {TOPOLOGY-WORKERS 2}
+                           topology2
+                           {1 "1"
+                            2 "1"}
+                           {[1 1] ["sup1" 1]
+                            [2 2] ["sup1" 2]}
+                           {["sup1" 1] [0.0 0.0 0.0]
+                            ["sup1" 2] [0.0 0.0 0.0]
+                            })
+                         ;; Instead of sleeping until topology is scheduled, rebalance topology so mk-assignments is called.
+                         (.rebalance (:nimbus cluster) "topology1" (doto (RebalanceOptions.) (.set_wait_secs 0)))
+                         ))
+         (is (empty? (:launched changed)))
+         (bind options (RebalanceOptions.))
+         (.set_wait_secs options 0)
+         (bind changed (capture-changed-workers
+                         (.rebalance (:nimbus cluster) "topology2" options)
+                         (advance-cluster-time cluster 10)
+                         (heartbeat-workers cluster "sup1" [1 2 3 4])
+                         (advance-cluster-time cluster 10)
+                         ))
+         (validate-launched-once (:launched changed)
+           {"sup1" [1 2]}
+           (get-storm-id (:storm-cluster-state cluster) "topology1"))
+         (validate-launched-once (:launched changed)
+           {"sup1" [3 4]}
+           (get-storm-id (:storm-cluster-state cluster) "topology2"))
 -        ))))
++        )))


[24/27] storm git commit: update/fix some codes based on @revans2

Posted by bo...@apache.org.
update/fix some codes based on @revans2


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/b09e2755
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/b09e2755
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/b09e2755

Branch: refs/heads/master
Commit: b09e2755c2fedcb9048afda63bea897fe1dbd12b
Parents: d709525
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 24 11:19:06 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 24 11:19:06 2016 +0800

----------------------------------------------------------------------
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  2 +-
 .../clj/org/apache/storm/daemon/supervisor.clj  |  4 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  8 +-
 .../org/apache/storm/cluster/ClusterUtils.java  | 25 +-----
 .../storm/cluster/IStormClusterState.java       |  4 +-
 .../storm/cluster/PaceMakerStateStorage.java    | 12 ++-
 .../storm/cluster/StormClusterStateImpl.java    | 41 +++++-----
 .../apache/storm/cluster/ZKStateStorage.java    | 80 +++++++-------------
 .../org/apache/storm/zookeeper/Zookeeper.java   |  9 ++-
 9 files changed, 74 insertions(+), 111 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 2f6587a..e524ec2 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -1676,7 +1676,7 @@
                                              [(node->host node) port])
                                     executor->node+port)
               nodeinfos (stats/extract-nodeinfos-from-hb-for-comp executor->host+port task->component false component_id)
-              all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id true))
+              all-pending-actions-for-topology (clojurify-profile-request (.getTopologyProfileRequests storm-cluster-state id))
               latest-profile-actions (remove nil? (map (fn [nodeInfo]
                                                          (->> all-pending-actions-for-topology
                                                               (filter #(and (= (:host nodeInfo) (.get_node (.get_nodeInfo %)))

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 0cee414..1446ac9 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -81,7 +81,7 @@
           (->>
             (dofor [sid (distinct storm-ids)]
 
-                   (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid false)] (clojurify-profile-request request)))]
+                   (if-let [topo-profile-actions (into [] (for [request (.getTopologyProfileRequests storm-cluster-state sid)] (clojurify-profile-request request)))]
                       {sid topo-profile-actions}))
            (apply merge))]
       {:assignments (into {} (for [[k v] new-assignments] [k (:data v)]))
@@ -607,7 +607,7 @@
           storm-cluster-state (:storm-cluster-state supervisor)
           ^ISupervisor isupervisor (:isupervisor supervisor)
           ^LocalState local-state (:local-state supervisor)
-          sync-callback (fn [& ignored] (.add event-manager (reify Runnable
+          sync-callback (fn [] (.add event-manager (reify Runnable
                                                                    (^void run [this]
                                                                      (callback-supervisor)))))
           assignment-versions @(:assignment-versions supervisor)

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index af88f6a..110d415 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -383,7 +383,7 @@
         storm-id (:storm-id worker)]
     (fn refresh-connections
       ([]
-        (refresh-connections (fn [& ignored]
+        (refresh-connections (fn []
                 (.schedule
                   (:refresh-connections-timer worker) 0 refresh-connections))))
       ([callback]
@@ -438,7 +438,7 @@
 (defn refresh-storm-active
   ([worker]
     (refresh-storm-active
-      worker (fn [& ignored]
+      worker (fn []
                (.schedule
                  (:refresh-active-timer worker) 0 (partial refresh-storm-active worker)))))
   ([worker callback]
@@ -685,7 +685,7 @@
         backpressure-thread (WorkerBackpressureThread. (:backpressure-trigger worker) worker backpressure-handler)
         _ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE) 
             (.start backpressure-thread))
-        callback (fn cb [& ignored]
+        callback (fn cb []
                    (let [throttle-on (.topologyBackpressure storm-cluster-state storm-id cb)]
                      (reset! (:throttle-on worker) throttle-on)))
         _ (if ((:storm-conf worker) TOPOLOGY-BACKPRESSURE-ENABLE)
@@ -757,7 +757,7 @@
                                         (dofor [e @executors] (.credentials-changed e new-creds))
                                         (reset! credentials new-creds))))
        check-throttle-changed (fn []
-                                (let [callback (fn cb [& ignored]
+                                (let [callback (fn cb []
                                                  (let [throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id cb)]
                                                    (reset! (:throttle-on worker) throttle-on)))
                                       new-throttle-on (.topologyBackpressure (:storm-cluster-state worker) storm-id callback)]

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 1095fff..96c177b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -211,7 +211,6 @@ public class ClusterUtils {
             IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, (Map) stateStorage, acls, context);
             return new StormClusterStateImpl(Storage, acls, context, true);
         }
-
     }
 
     public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
@@ -237,25 +236,9 @@ public class ClusterUtils {
     }
 
     public static String stringifyError(Throwable error) {
-        String errorString = null;
-        StringWriter result = null;
-        PrintWriter printWriter = null;
-        try {
-            result = new StringWriter();
-            printWriter = new PrintWriter(result);
-            error.printStackTrace(printWriter);
-            if (result != null) {
-                errorString = result.toString();
-            }
-        } finally {
-            try {
-                if (result != null)
-                    result.close();
-                if (printWriter != null)
-                    printWriter.close();
-            } catch (Exception e) {
-            }
-        }
-        return errorString;
+        StringWriter result = new StringWriter();
+        PrintWriter printWriter = new PrintWriter(result);
+        error.printStackTrace(printWriter);
+        return result.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index c88935e..e26c598 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -45,9 +45,9 @@ public interface IStormClusterState {
 
     public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
 
-    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo);
 
-    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId);
 
     public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
 

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index c29078e..c42bd38 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -127,7 +127,8 @@ public class PaceMakerStateStorage implements IStateStorage {
                 if (retry <= 0) {
                     throw Utils.wrapInRuntime(e);
                 }
-                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+                retry--;
+                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
             }
         }
     }
@@ -148,7 +149,8 @@ public class PaceMakerStateStorage implements IStateStorage {
                 if (retry <= 0) {
                     throw Utils.wrapInRuntime(e);
                 }
-                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+                retry--;
+                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
             }
         }
     }
@@ -169,7 +171,8 @@ public class PaceMakerStateStorage implements IStateStorage {
                 if (retry <= 0) {
                     throw Utils.wrapInRuntime(e);
                 }
-                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+                retry--;
+                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry);
             }
         }
     }
@@ -190,7 +193,8 @@ public class PaceMakerStateStorage implements IStateStorage {
                 if (retry <= 0) {
                     throw Utils.wrapInRuntime(e);
                 }
-                LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+                retry--;
+                LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 5fa586a..bde7670 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -106,7 +106,7 @@ public class StormClusterStateImpl implements IStormClusterState {
                     } 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));
+                        issueMapCallback(backPressureCallback, toks.get(1));
                     } else {
                         LOG.error("{} Unknown callback for subtree {}", new RuntimeException("Unknown callback for this path"), path);
                         Runtime.getRuntime().exit(30);
@@ -242,9 +242,9 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift) {
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo) {
         List<ProfileRequest> requests = new ArrayList<>();
-        List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId, isThrift);
+        List<ProfileRequest> profileRequests = getTopologyProfileRequests(stormId);
         for (ProfileRequest profileRequest : profileRequests) {
             NodeInfo nodeInfo1 = profileRequest.get_nodeInfo();
             if (nodeInfo1.equals(nodeInfo))
@@ -254,7 +254,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift) {
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId) {
         List<ProfileRequest> profileRequests = new ArrayList<>();
         String path = ClusterUtils.profilerConfigPath(stormId);
         if (stateStorage.node_exists(path, false)) {
@@ -382,6 +382,9 @@ public class StormClusterStateImpl implements IStormClusterState {
 
     @Override
     public LogConfig topologyLogConfig(String stormId, Runnable cb) {
+        if (cb != null){
+            logConfigCallback.put(stormId, cb);
+        }
         String path = ClusterUtils.logConfigPath(stormId);
         return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
     }
@@ -625,25 +628,21 @@ public class StormClusterStateImpl implements IStormClusterState {
     @Override
     public List<ErrorInfo> errors(String stormId, String componentId) {
         List<ErrorInfo> errorInfos = new ArrayList<>();
-        try {
-            String path = ClusterUtils.errorPath(stormId, componentId);
-            if (stateStorage.node_exists(path, false)) {
-                List<String> childrens = stateStorage.get_children(path, false);
-                for (String child : childrens) {
-                    String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
-                    ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
-                    if (errorInfo != null)
-                        errorInfos.add(errorInfo);
-                }
+        String path = ClusterUtils.errorPath(stormId, componentId);
+        if (stateStorage.node_exists(path, false)) {
+            List<String> childrens = stateStorage.get_children(path, false);
+            for (String child : childrens) {
+                String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+                ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+                if (errorInfo != null)
+                    errorInfos.add(errorInfo);
             }
-            Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
-                public int compare(ErrorInfo arg0, ErrorInfo arg1) {
-                    return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
-                }
-            });
-        } catch (Exception e) {
-            throw Utils.wrapInRuntime(e);
         }
+        Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+            public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+                return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
+            }
+        });
 
         return errorInfos;
     }

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index 56115ce..4cf0c05 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -53,6 +53,26 @@ public class ZKStateStorage implements IStateStorage {
     private Map authConf;
     private Map<Object, Object> conf;
 
+    private class ZkWatcherCallBack implements WatcherCallBack{
+        @Override
+        public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+            if (active.get()) {
+                if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                    LOG.debug("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                } else {
+                    LOG.debug("Received event {} : {} : {}", state, type, path);
+                }
+
+                if (!type.equals(Watcher.Event.EventType.None)) {
+                    for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+                        ZKStateChangedCallback fn = e.getValue();
+                        fn.changed(type, path);
+                    }
+                }
+            }
+        }
+    }
+
     public ZKStateStorage(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
         this.conf = conf;
         this.authConf = authConf;
@@ -66,45 +86,9 @@ public class ZKStateStorage implements IStateStorage {
         zkTemp.close();
 
         active = new AtomicBoolean(true);
-        zkWriter = mkZk(new WatcherCallBack() {
-            @Override
-            public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
-                if (active.get()) {
-                    if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
-                        LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
-                    } else {
-                        LOG.info("Received event {} : {} : {}", state, type, path);
-                    }
-
-                    if (!type.equals(Watcher.Event.EventType.None)) {
-                        for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
-                            ZKStateChangedCallback fn = e.getValue();
-                            fn.changed(type, path);
-                        }
-                    }
-                }
-            }
-        });
+        zkWriter = mkZk(new ZkWatcherCallBack());
         if (isNimbus) {
-            zkReader = mkZk(new WatcherCallBack() {
-                @Override
-                public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
-                    if (active.get()) {
-                        if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
-                            LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
-                        } else {
-                            LOG.debug("Received event {} : {} : {}", state, type, path);
-                        }
-
-                        if (!type.equals(Watcher.Event.EventType.None)) {
-                            for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
-                                ZKStateChangedCallback fn = e.getValue();
-                                fn.changed(type, path);
-                            }
-                        }
-                    }
-                }
-            });
+            zkReader = mkZk(new ZkWatcherCallBack());
         } else {
             zkReader = zkWriter;
         }
@@ -157,15 +141,15 @@ public class ZKStateStorage implements IStateStorage {
 
     @Override
     public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
-        Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+        Zookeeper.mkdirs(zkWriter, Zookeeper.parentPath(path), acls);
         if (Zookeeper.exists(zkWriter, path, false)) {
             try {
                 Zookeeper.setData(zkWriter, path, data);
-            } catch (Exception e) {
+            } catch (RuntimeException e) {
                 if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
                     Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
                 } else {
-                    throw Utils.wrapInRuntime(e);
+                    throw e;
                 }
             }
 
@@ -182,7 +166,7 @@ public class ZKStateStorage implements IStateStorage {
 
     @Override
     public boolean node_exists(String path, boolean watch) {
-        return Zookeeper.existsNode(zkWriter, path, watch);
+        return Zookeeper.existsNode(zkReader, path, watch);
     }
 
     @Override
@@ -204,7 +188,7 @@ public class ZKStateStorage implements IStateStorage {
         if (Zookeeper.exists(zkWriter, path, false)) {
             Zookeeper.setData(zkWriter, path, data);
         } else {
-            Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+            Zookeeper.mkdirs(zkWriter, Zookeeper.parentPath(path), acls);
             Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
         }
     }
@@ -257,14 +241,4 @@ public class ZKStateStorage implements IStateStorage {
     public void sync_path(String path) {
         Zookeeper.syncPath(zkWriter, path);
     }
-
-    // To be remove when finished port Util.clj
-    public static String parentPath(String path) {
-        List<String> toks = Zookeeper.tokenizePath(path);
-        int size = toks.size();
-        if (size > 0) {
-            toks.remove(size - 1);
-        }
-        return Zookeeper.toksToPath(toks);
-    }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/b09e2755/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index e5b2666..5e9039a 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -394,9 +394,12 @@ public class Zookeeper {
     }
 
     public static String parentPath(String path) {
-        List<String> tokens = tokenizePath(path);
-        tokens.remove(tokens.size() - 1);
-        return "/" + StringUtils.join(tokens, "/");
+        List<String> toks = Zookeeper.tokenizePath(path);
+        int size = toks.size();
+        if (size > 0) {
+            toks.remove(size - 1);
+        }
+        return Zookeeper.toksToPath(toks);
     }
 
     public static String toksToPath(List<String> toks) {


[14/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/924f9a29
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/924f9a29
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/924f9a29

Branch: refs/heads/master
Commit: 924f9a29dc1713921178e61ce2f5b6a14df97ed7
Parents: 2ee8bec 12ceb09
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 15:13:33 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:14:18 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |  16 +
 README.markdown                                 |   1 +
 bin/storm-config.cmd                            |   6 +-
 bin/storm.cmd                                   |  47 +-
 bin/storm.py                                    |   8 +-
 conf/defaults.yaml                              |   4 +
 dev-tools/travis/travis-script.sh               |   4 +-
 .../starter/trident/TridentMapExample.java      | 123 +++
 external/sql/storm-sql-core/pom.xml             |   9 +
 external/storm-elasticsearch/pom.xml            |   2 +
 .../storm/hbase/security/HBaseSecurityUtil.java |  36 +-
 .../jvm/org/apache/storm/kafka/KafkaSpout.java  |   8 +-
 .../jvm/org/apache/storm/kafka/KafkaUtils.java  |  44 +-
 .../apache/storm/kafka/PartitionManager.java    |  42 +-
 .../kafka/trident/TridentKafkaEmitter.java      |  23 +-
 external/storm-mqtt/core/pom.xml                |   4 +-
 log4j2/cluster.xml                              |   2 +-
 log4j2/worker.xml                               |   2 +-
 pom.xml                                         |   9 +-
 storm-core/pom.xml                              |  11 +-
 .../src/clj/org/apache/storm/LocalCluster.clj   |   4 +-
 storm-core/src/clj/org/apache/storm/clojure.clj |   8 +-
 .../clj/org/apache/storm/command/blobstore.clj  |  11 +-
 .../org/apache/storm/command/config_value.clj   |  25 -
 .../org/apache/storm/command/dev_zookeeper.clj  |   6 +-
 .../clj/org/apache/storm/command/get_errors.clj |  12 +-
 .../apache/storm/command/shell_submission.clj   |   4 +-
 storm-core/src/clj/org/apache/storm/config.clj  |  18 +-
 .../src/clj/org/apache/storm/converter.clj      |  14 +-
 .../src/clj/org/apache/storm/daemon/acker.clj   |  21 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |  46 +-
 .../src/clj/org/apache/storm/daemon/drpc.clj    |  25 +-
 .../clj/org/apache/storm/daemon/executor.clj    | 530 +++++-----
 .../clj/org/apache/storm/daemon/logviewer.clj   |  70 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 172 ++--
 .../clj/org/apache/storm/daemon/supervisor.clj  | 202 ++--
 .../src/clj/org/apache/storm/daemon/task.clj    |   2 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  78 +-
 .../src/clj/org/apache/storm/disruptor.clj      |  10 +-
 storm-core/src/clj/org/apache/storm/event.clj   |   2 +-
 .../src/clj/org/apache/storm/local_state.clj    |   9 +-
 .../clj/org/apache/storm/messaging/loader.clj   |  34 -
 .../clj/org/apache/storm/messaging/local.clj    |  23 -
 .../org/apache/storm/pacemaker/pacemaker.clj    |   7 +-
 .../storm/pacemaker/pacemaker_state_factory.clj | 122 ---
 .../clj/org/apache/storm/process_simulator.clj  |   4 +-
 .../apache/storm/scheduler/DefaultScheduler.clj |   7 +-
 .../apache/storm/scheduler/EvenScheduler.clj    |  23 +-
 .../storm/scheduler/IsolationScheduler.clj      |  29 +-
 storm-core/src/clj/org/apache/storm/stats.clj   |  82 +-
 storm-core/src/clj/org/apache/storm/testing.clj |  89 +-
 storm-core/src/clj/org/apache/storm/thrift.clj  |   6 +-
 storm-core/src/clj/org/apache/storm/timer.clj   |  12 +-
 .../clj/org/apache/storm/trident/testing.clj    |   9 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |  99 +-
 .../src/clj/org/apache/storm/ui/helpers.clj     |  14 +-
 storm-core/src/clj/org/apache/storm/util.clj    | 925 +----------------
 storm-core/src/jvm/org/apache/storm/Config.java |  39 +
 .../org/apache/storm/cluster/ClusterUtils.java  |  49 +-
 .../storm/cluster/IStormClusterState.java       |   9 +-
 .../storm/cluster/PaceMakerStateStorage.java    |   4 +-
 .../storm/cluster/StateStorageFactory.java      |   2 +-
 .../storm/cluster/StormClusterStateImpl.java    |  51 +-
 .../storm/cluster/ZKStateStorageFactory.java    |   4 +-
 .../org/apache/storm/command/ConfigValue.java   |  30 +
 .../storm/daemon/metrics/MetricsUtils.java      | 108 ++
 .../reporters/ConsolePreparableReporter.java    |  76 ++
 .../reporters/CsvPreparableReporter.java        |  80 ++
 .../reporters/JmxPreparableReporter.java        |  70 ++
 .../metrics/reporters/PreparableReporter.java   |  32 +
 .../storm/logging/ThriftAccessLogger.java       |  13 +-
 .../apache/storm/pacemaker/PacemakerClient.java |   5 +
 .../serialization/SerializationFactory.java     |  17 +-
 .../staticmocking/MockedConfigUtils.java        |  31 -
 .../jvm/org/apache/storm/trident/Stream.java    |  87 +-
 .../storm/trident/operation/Consumer.java       |  35 +
 .../trident/operation/FlatMapFunction.java      |  37 +
 .../storm/trident/operation/MapFunction.java    |  36 +
 .../operation/impl/ConsumerExecutor.java        |  38 +
 .../operation/impl/FlatMapFunctionExecutor.java |  43 +
 .../operation/impl/MapFunctionExecutor.java     |  41 +
 .../trident/planner/processor/MapProcessor.java |  87 ++
 .../jvm/org/apache/storm/utils/ConfigUtils.java |  20 +-
 .../jvm/org/apache/storm/utils/Container.java   |  11 +-
 .../jvm/org/apache/storm/utils/IPredicate.java  |  27 +
 .../org/apache/storm/utils/NimbusClient.java    |   2 +-
 .../utils/StormConnectionStateConverter.java    |  44 +
 .../jvm/org/apache/storm/utils/TestUtils.java   |  34 -
 .../src/jvm/org/apache/storm/utils/Time.java    |  26 +-
 .../src/jvm/org/apache/storm/utils/Utils.java   | 989 +++++++++++++++++--
 .../storm/validation/ConfigValidation.java      |   2 +-
 .../org/apache/storm/zookeeper/Zookeeper.java   |   7 +
 .../org/apache/storm/integration_test.clj       |  98 +-
 .../org/apache/storm/testing4j_test.clj         |  37 +-
 .../apache/storm/trident/integration_test.clj   |  15 +-
 .../test/clj/org/apache/storm/cluster_test.clj  |  27 +-
 .../test/clj/org/apache/storm/drpc_test.clj     |  23 +-
 .../clj/org/apache/storm/logviewer_test.clj     | 267 ++---
 .../storm/messaging/netty_integration_test.clj  |   2 +-
 .../test/clj/org/apache/storm/nimbus_test.clj   | 112 ++-
 .../storm/pacemaker_state_factory_test.clj      |  74 +-
 .../scheduler/resource_aware_scheduler_test.clj |  21 +-
 .../apache/storm/security/auth/auth_test.clj    |  11 +-
 .../authorizer/DRPCSimpleACLAuthorizer_test.clj |   2 +-
 .../BlowfishTupleSerializer_test.clj            |   1 -
 .../clj/org/apache/storm/serialization_test.clj |  23 +-
 .../clj/org/apache/storm/supervisor_test.clj    | 649 ++++++------
 .../clj/org/apache/storm/transactional_test.clj |  18 +
 .../clj/org/apache/storm/trident/state_test.clj |   5 +-
 .../clj/org/apache/storm/trident/tuple_test.clj |  15 +-
 .../test/clj/org/apache/storm/utils_test.clj    |  16 +-
 .../test/clj/org/apache/storm/worker_test.clj   |   1 -
 .../staticmocking/ConfigUtilsInstaller.java     |  38 +
 .../utils/staticmocking/UtilsInstaller.java     |  38 +
 .../storm/utils/staticmocking/package-info.java |  95 ++
 115 files changed, 4281 insertions(+), 2648 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/conf/defaults.yaml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
index 7be526d,657e242..ab5cbed
--- a/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
+++ b/storm-core/src/clj/org/apache/storm/command/dev_zookeeper.clj
@@@ -14,7 -14,8 +14,8 @@@
  ;; See the License for the specific language governing permissions and
  ;; limitations under the License.
  (ns org.apache.storm.command.dev-zookeeper
+   (:import [org.apache.storm.utils Utils])
 -  (:use [org.apache.storm zookeeper util config])
 +  (:use [org.apache.storm util config])
    (:import [org.apache.storm.utils ConfigUtils])
    (:import [org.apache.storm.zookeeper Zookeeper])
    (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/command/shell_submission.clj
index 3978d2f,0253338..870e7f6
--- a/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
+++ b/storm-core/src/clj/org/apache/storm/command/shell_submission.clj
@@@ -15,8 -15,9 +15,9 @@@
  ;; limitations under the License.
  (ns org.apache.storm.command.shell-submission
    (:import [org.apache.storm StormSubmitter]
+            [org.apache.storm.utils Utils]
             [org.apache.storm.zookeeper Zookeeper])
 -  (:use [org.apache.storm thrift util config log zookeeper])
 +  (:use [org.apache.storm thrift util config log])
    (:require [clojure.string :as str])
    (:import [org.apache.storm.utils ConfigUtils])
    (:gen-class))

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/common.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/common.clj
index b144f40,eb1ec1e..db342d2
--- a/storm-core/src/clj/org/apache/storm/daemon/common.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/common.clj
@@@ -15,17 -15,19 +15,20 @@@
  ;; limitations under the License.
  (ns org.apache.storm.daemon.common
    (:use [org.apache.storm log config util])
 -  (:import [org.apache.storm.generated StormTopology
 +  (:import [org.apache.storm.generated StormTopology NodeInfo
              InvalidTopologyException GlobalStreamId]
-            [org.apache.storm.utils ThriftTopologyUtils])
-   (:import [org.apache.storm.utils Utils ConfigUtils])
+            [org.apache.storm.utils Utils ConfigUtils IPredicate ThriftTopologyUtils]
+            [org.apache.storm.daemon.metrics.reporters PreparableReporter]
+            [com.codahale.metrics MetricRegistry])
+   (:import [org.apache.storm.daemon.metrics MetricsUtils])
    (:import [org.apache.storm.task WorkerTopologyContext])
    (:import [org.apache.storm Constants])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl])
    (:import [org.apache.storm.metric SystemBolt])
    (:import [org.apache.storm.metric EventLoggerBolt])
-   (:import [org.apache.storm.security.auth IAuthorizer]) 
-   (:import [java.io InterruptedIOException])
+   (:import [org.apache.storm.security.auth IAuthorizer])
+   (:import [java.io InterruptedIOException]
+            [org.json.simple JSONValue])
    (:require [clojure.set :as set])  
    (:require [org.apache.storm.daemon.acker :as acker])
    (:require [org.apache.storm.thrift :as thrift])
@@@ -73,12 -83,10 +84,11 @@@
  (defn new-executor-stats []
    (ExecutorStats. 0 0 0 0 0))
  
 +
  (defn get-storm-id [storm-cluster-state storm-name]
-   (let [active-storms (.activeStorms storm-cluster-state)]
-     (find-first
-       #(= storm-name (.get_name (.stormBase storm-cluster-state % nil)))
-       active-storms)
 -  (let [active-storms (.active-storms storm-cluster-state)
 -        pred  (reify IPredicate (test [this x] (= storm-name (:storm-name (.storm-base storm-cluster-state x nil)))))]
++  (let [active-storms (.activeStorms storm-cluster-state)
++        pred  (reify IPredicate (test [this x] (= storm-name (.get_name (.stormBase storm-cluster-state x nil)))))]
+     (Utils/findOne pred active-storms)
      ))
  
  (defn topology-bases [storm-cluster-state]

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/executor.clj
index 49ae6cf,e2380b7..33b89ed
--- a/storm-core/src/clj/org/apache/storm/daemon/executor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/executor.clj
@@@ -34,10 -34,13 +34,12 @@@
    (:import [org.apache.storm.daemon Shutdownable])
    (:import [org.apache.storm.metric.api IMetric IMetricsConsumer$TaskInfo IMetricsConsumer$DataPoint StateMetric])
    (:import [org.apache.storm Config Constants])
 -  (:import [org.apache.storm.cluster ClusterStateContext DaemonType])
 +  (:import [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils])
    (:import [org.apache.storm.grouping LoadAwareCustomStreamGrouping LoadAwareShuffleGrouping LoadMapping ShuffleGrouping])
-   (:import [java.util.concurrent ConcurrentLinkedQueue])
+   (:import [java.lang Thread Thread$UncaughtExceptionHandler]
+            [java.util.concurrent ConcurrentLinkedQueue]
+            [org.json.simple JSONValue])
 -  (:require [org.apache.storm [thrift :as thrift]
 -             [cluster :as cluster] [disruptor :as disruptor] [stats :as stats]])
 +  (:require [org.apache.storm [thrift :as thrift] [disruptor :as disruptor] [stats :as stats]])
    (:require [org.apache.storm.daemon [task :as task]])
    (:require [org.apache.storm.daemon.builtin-metrics :as builtin-metrics])
    (:require [clojure.set :as set]))
@@@ -206,9 -211,9 +210,9 @@@
        (swap! interval-errors inc)
  
        (when (<= @interval-errors max-per-interval)
 -        (cluster/report-error (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
 +        (.reportError (:storm-cluster-state executor) (:storm-id executor) (:component-id executor)
-                               (hostname storm-conf)
+                               (Utils/hostname storm-conf)
 -                              (.getThisWorkerPort (:worker-context executor)) error)
 +          (long (.getThisWorkerPort (:worker-context executor))) error)
          ))))
  
  ;; in its own function so that it can be mocked out by tracked topologies

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index daf5e45,710cd83..6bdbdc0
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -38,9 -39,9 +39,9 @@@
    (:import [org.apache.storm.scheduler INimbus SupervisorDetails WorkerSlot TopologyDetails
              Cluster Topologies SchedulerAssignment SchedulerAssignmentImpl DefaultScheduler ExecutorDetails])
    (:import [org.apache.storm.nimbus NimbusInfo])
-   (:import [org.apache.storm.utils TimeCacheMap TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
+   (:import [org.apache.storm.utils TimeCacheMap Time TimeCacheMap$ExpiredCallback Utils ConfigUtils TupleUtils ThriftTopologyUtils
              BufferFileInputStream BufferInputStream])
 -  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo
 +  (:import [org.apache.storm.generated NotAliveException AlreadyAliveException StormTopology ErrorInfo ClusterWorkerHeartbeat
              ExecutorInfo InvalidTopologyException Nimbus$Iface Nimbus$Processor SubmitOptions TopologyInitialStatus
              KillOptions RebalanceOptions ClusterSummary SupervisorSummary TopologySummary TopologyInfo TopologyHistoryInfo
              ExecutorSummary AuthorizationException GetInfoOptions NumErrorsChoice SettableBlobMeta ReadableBlobMeta
@@@ -407,10 -414,10 +409,10 @@@
    [storm-cluster-state]
  
    (let [assignments (.assignments storm-cluster-state nil)]
-     (defaulted
+     (or
        (apply merge-with set/union
               (for [a assignments
 -                   [_ [node port]] (-> (.assignment-info storm-cluster-state a nil) :executor->node+port)]
 +                   [_ [node port]] (-> (clojurify-assignment (.assignmentInfo storm-cluster-state a nil)) :executor->node+port)]
                 {node #{port}}
                 ))
        {})
@@@ -586,12 -594,7 +589,12 @@@
  (defn update-heartbeats! [nimbus storm-id all-executors existing-assignment]
    (log-debug "Updating heartbeats for " storm-id " " (pr-str all-executors))
    (let [storm-cluster-state (:storm-cluster-state nimbus)
 -        executor-beats (.executor-beats storm-cluster-state storm-id (:executor->node+port existing-assignment))
 +        executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
 +                             executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
-                          (->> (dofor [[^ExecutorInfo executor-info ^ClusterWorkerHeartbeat cluster-worker-heartbeat] executor-stats-clojurify]
-                              {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-worker-hb cluster-worker-heartbeat)})
++                         (->> (dofor [[^ExecutorInfo executor-info  executor-heartbeat] executor-stats-clojurify]
++                             {[(.get_task_start executor-info) (.get_task_end executor-info)] executor-heartbeat})
 +                           (apply merge)))
 +
          cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
                                        executor-beats
                                        all-executors
@@@ -988,10 -1002,10 +1002,10 @@@
          topology (system-topology! storm-conf (read-storm-topology storm-id blob-store))
          num-executors (->> (all-components topology) (map-val num-start-executors))]
      (log-message "Activating " storm-name ": " storm-id)
 -    (.activate-storm! storm-cluster-state
 +    (.activateStorm storm-cluster-state
                        storm-id
 -                      (StormBase. storm-name
 +      (thriftify-storm-base (StormBase. storm-name
-                                   (current-time-secs)
+                                   (Time/currentTimeSecs)
                                    {:type topology-initial-status}
                                    (storm-conf TOPOLOGY-WORKERS)
                                    num-executors
@@@ -1137,9 -1152,9 +1152,9 @@@
          (when-not (empty? to-cleanup-ids)
            (doseq [id to-cleanup-ids]
              (log-message "Cleaning up " id)
 -            (.teardown-heartbeats! storm-cluster-state id)
 -            (.teardown-topology-errors! storm-cluster-state id)
 +            (.teardownHeartbeats storm-cluster-state id)
 +            (.teardownTopologyErrors storm-cluster-state id)
-             (rmr (ConfigUtils/masterStormDistRoot conf id))
+             (Utils/forceDelete (ConfigUtils/masterStormDistRoot conf id))
              (blob-rm-topology-keys id blob-store storm-cluster-state)
              (swap! (:heartbeats-cache nimbus) dissoc id)))))
      (log-message "not a leader, skipping cleanup")))
@@@ -1811,8 -1830,8 +1838,8 @@@
                                           (.set_used_cpu sup-sum used-cpu))
                                         (when-let [version (:version info)] (.set_version sup-sum version))
                                         sup-sum))
-               nimbus-uptime ((:uptime nimbus))
+               nimbus-uptime (. (:uptime nimbus) upTime)
 -              bases (topology-bases storm-cluster-state)
 +              bases (nimbus-topology-bases storm-cluster-state)
                nimbuses (.nimbuses storm-cluster-state)
  
                ;;update the isLeader field for each nimbus summary

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index 3a83d03,ae9e92f..273a6bd
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -16,14 -16,15 +16,15 @@@
  (ns org.apache.storm.daemon.supervisor
    (:import [java.io File IOException FileOutputStream])
    (:import [org.apache.storm.scheduler ISupervisor]
-            [org.apache.storm.utils LocalState Time Utils ConfigUtils]
+            [org.apache.storm.utils LocalState Time Utils Utils$ExitCodeCallable
+                                    ConfigUtils]
             [org.apache.storm.daemon Shutdownable]
             [org.apache.storm Constants]
 -           [org.apache.storm.cluster ClusterStateContext DaemonType]
 +           [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
             [java.net JarURLConnection]
-            [java.net URI]
+            [java.net URI URLDecoder]
             [org.apache.commons.io FileUtils])
 -  (:use [org.apache.storm config util log timer local-state])
 +  (:use [org.apache.storm config util log timer local-state converter])
    (:import [org.apache.storm.generated AuthorizationException KeyNotFoundException WorkerResources])
    (:import [org.apache.storm.utils NimbusLeaderNotFoundException VersionInfo])
    (:import [java.nio.file Files StandardCopyOption])
@@@ -315,12 -319,14 +321,12 @@@
     :shared-context shared-context
     :isupervisor isupervisor
     :active (atom true)
-    :uptime (uptime-computer)
+    :uptime (Utils/makeUptimeComputer)
     :version STORM-VERSION
     :worker-thread-pids-atom (atom {})
 -   :storm-cluster-state (cluster/mk-storm-cluster-state conf :acls (when
 -                                                                     (Utils/isZkAuthenticationConfiguredStormServer
 -                                                                       conf)
 -                                                                     SUPERVISOR-ZK-ACLS)
 -                                                        :context (ClusterStateContext. DaemonType/SUPERVISOR))
 +   :storm-cluster-state (ClusterUtils/mkStormClusterState conf (when (Utils/isZkAuthenticationConfiguredStormServer conf)
 +                                                     SUPERVISOR-ZK-ACLS)
 +                                                        (ClusterStateContext. DaemonType/SUPERVISOR))
     :local-state (ConfigUtils/supervisorState conf)
     :supervisor-id (.getSupervisorId isupervisor)
     :assignment-id (.getAssignmentId isupervisor)
@@@ -777,19 -791,19 +792,19 @@@
          synchronize-blobs-fn (update-blobs-for-all-topologies-fn supervisor)
          downloaded-storm-ids (set (read-downloaded-storm-ids conf))
          run-profiler-actions-fn (mk-run-profiler-actions-for-all-topologies supervisor)
 -        heartbeat-fn (fn [] (.supervisor-heartbeat!
 +        heartbeat-fn (fn [] (.supervisorHeartbeat
                                 (:storm-cluster-state supervisor)
                                 (:supervisor-id supervisor)
-                               (thriftify-supervisor-info (->SupervisorInfo (current-time-secs)
 -                               (->SupervisorInfo (Time/currentTimeSecs)
++                               (thriftify-supervisor-info (->SupervisorInfo (Time/currentTimeSecs)
                                                   (:my-hostname supervisor)
                                                   (:assignment-id supervisor)
                                                   (keys @(:curr-assignment supervisor))
                                                    ;; used ports
                                                   (.getMetadata isupervisor)
                                                   (conf SUPERVISOR-SCHEDULER-META)
-                                                  ((:uptime supervisor))
+                                                  (. (:uptime supervisor) upTime)
                                                   (:version supervisor)
 -                                                 (mk-supervisor-capacities conf))))]
 +                                                 (mk-supervisor-capacities conf)))))]
      (heartbeat-fn)
  
      ;; should synchronize supervisor so it doesn't launch anything after being down (optimization)

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index ae5be57,fe8cfae..9863427
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,16 -19,18 +19,18 @@@
    (:require [clj-time.core :as time])
    (:require [clj-time.coerce :as coerce])
    (:require [org.apache.storm.daemon [executor :as executor]])
 -  (:require [org.apache.storm [disruptor :as disruptor] [cluster :as cluster]])
 +  (:require [org.apache.storm [disruptor :as disruptor]])
    (:require [clojure.set :as set])
-   (:require [org.apache.storm.messaging.loader :as msg-loader])
    (:import [java.util.concurrent Executors]
-            [org.apache.storm.hooks IWorkerHook BaseWorkerHook])
-   (:import [java.util ArrayList HashMap])
-   (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue])
+            [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
+            [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])
+   (:import [java.util ArrayList HashMap]
+            [java.util.concurrent.locks ReentrantReadWriteLock])
+   (:import [org.apache.commons.io FileUtils])
+   (:import [org.apache.storm.utils Utils ConfigUtils TransferDrainer ThriftTopologyUtils WorkerBackpressureThread DisruptorQueue Time])
    (:import [org.apache.storm.grouping LoadMapping])
    (:import [org.apache.storm.messaging TransportFactory])
-   (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status])
+   (:import [org.apache.storm.messaging TaskMessage IContext IConnection ConnectionWithStatus ConnectionWithStatus$Status DeserializingConnectionCallback])
    (:import [org.apache.storm.daemon Shutdownable])
    (:import [org.apache.storm.serialization KryoTupleSerializer])
    (:import [org.apache.storm.generated StormTopology])

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index a36da3a,be4361a..0000000
deleted file mode 100644,100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ /dev/null
@@@ -1,122 -1,141 +1,0 @@@
--;; 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.
--
--(ns org.apache.storm.pacemaker.pacemaker-state-factory
--  (:require [org.apache.storm.pacemaker pacemaker]
 -            [org.apache.storm.cluster-state [zookeeper-state-factory :as zk-factory]]
--            [org.apache.storm
--             [config :refer :all]
 -             [cluster :refer :all]
--             [log :refer :all]
--             [util :as util]])
--  (:import [org.apache.storm.generated
--            HBExecutionException HBServerMessageType HBMessage
--            HBMessageData HBPulse]
-            [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
 -           [org.apache.storm.cluster_state zookeeper_state_factory]
 -           [org.apache.storm.cluster ClusterState]
--           [org.apache.storm.pacemaker PacemakerClient])
--  (:gen-class
-     :implements [org.apache.storm.cluster.StateStorageFactory]))
 -   :implements [org.apache.storm.cluster.ClusterStateFactory]))
--
--;; So we can mock the client for testing
--(defn makeClient [conf]
--  (PacemakerClient. conf))
--
--(defn makeZKState [conf auth-conf acls context]
-   (ClusterUtils/mkStateStorage conf auth-conf acls context))
 -  (.mkState (zookeeper_state_factory.) conf auth-conf acls context))
--
--(def max-retries 10)
 -
 -(defn retry-on-exception
 -  "Retries specific function on exception based on retries count"
 -  [retries task-description f & args]
 -  (let [res (try {:value (apply f args)}
 -                 (catch Exception e
 -                   (if (<= 0 retries)
 -                     (throw e)
 -                     {:exception e})))]
 -    (if (:exception res)
 -      (do 
 -        (log-error (:exception res) (str "Failed to " task-description ". Will make [" retries "] more attempts."))
 -        (recur (dec retries) task-description f args))
 -      (do 
 -        (log-debug (str "Successful " task-description "."))
 -        (:value res)))))
--
--(defn -mkState [this conf auth-conf acls context]
--  (let [zk-state (makeZKState conf auth-conf acls context)
--        pacemaker-client (makeClient conf)]
--
--    (reify
-       IStateStorage
 -      ClusterState
--      ;; Let these pass through to the zk-state. We only want to handle heartbeats.
--      (register [this callback] (.register zk-state callback))
--      (unregister [this callback] (.unregister zk-state callback))
--      (set_ephemeral_node [this path data acls] (.set_ephemeral_node zk-state path data acls))
--      (create_sequential [this path data acls] (.create_sequential zk-state path data acls))
--      (set_data [this path data acls] (.set_data zk-state path data acls))
--      (delete_node [this path] (.delete_node zk-state path))
--      (delete_node_blobstore [this path nimbus-host-port-info] (.delete_node_blobstore zk-state path nimbus-host-port-info))
--      (get_data [this path watch?] (.get_data zk-state path watch?))
--      (get_data_with_version [this path watch?] (.get_data_with_version zk-state path watch?))
--      (get_version [this path watch?] (.get_version zk-state path watch?))
--      (get_children [this path watch?] (.get_children zk-state path watch?))
--      (mkdirs [this path acls] (.mkdirs zk-state path acls))
--      (node_exists [this path watch?] (.node_exists zk-state path watch?))
--      (add_listener [this listener] (.add_listener zk-state listener))
--      (sync_path [this path] (.sync_path zk-state path))
--      
--      (set_worker_hb [this path data acls]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "set_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/SEND_PULSE
--                                   (HBMessageData/pulse
--                                    (doto (HBPulse.)
--                                      (.set_id path)
--                                      (.set_details data)))))]
--            (if (= (.get_type response) HBServerMessageType/SEND_PULSE_RESPONSE)
--              :ok
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--
--      (delete_worker_hb [this path]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "delete_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/DELETE_PATH
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/DELETE_PATH_RESPONSE)
--              :ok
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (get_worker_hb [this path watch?]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "get_worker_hb"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/GET_PULSE
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/GET_PULSE_RESPONSE)
--              (try 
--                (.get_details (.get_pulse (.get_data response)))
--                (catch Exception e
--                  (throw (HBExecutionException. (.toString e)))))
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (get_worker_hb_children [this path watch?]
-         (util/retry-on-exception
 -        (retry-on-exception
--         max-retries
--         "get_worker_hb_children"
--         #(let [response
--                (.send pacemaker-client
--                       (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH
--                                   (HBMessageData/path path)))]
--            (if (= (.get_type response) HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE)
--              (try
--                (into [] (.get_pulseIds (.get_nodes (.get_data response))))
--                (catch Exception e
--                  (throw (HBExecutionException. (.toString e)))))
--              (throw (HBExecutionException. "Invalid Response Type"))))))
--      
--      (close [this]
--        (.close zk-state)
--        (.close pacemaker-client)))))

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/stats.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/stats.clj
index 0bf1757,8b37fc3..8632ed3
--- a/storm-core/src/clj/org/apache/storm/stats.clj
+++ b/storm-core/src/clj/org/apache/storm/stats.clj
@@@ -24,8 -24,8 +24,9 @@@
              ExecutorAggregateStats SpecificAggregateStats
              SpoutAggregateStats TopologyPageInfo TopologyStats])
    (:import [org.apache.storm.utils Utils])
 +  (:import [org.apache.storm.cluster StormClusterStateImpl])
-   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric])
+   (:import [org.apache.storm.metric.internal MultiCountStatAndMetric MultiLatencyStatAndMetric]
+            [java.util Collection])
    (:use [org.apache.storm log util])
    (:use [clojure.math.numeric-tower :only [ceil]]))
  

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/testing.clj
index 5a0bdf2,c872742..eef7754
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@@ -44,11 -45,12 +45,12 @@@
    (:import [org.apache.storm.transactional.partitioned PartitionedTransactionalSpoutExecutor])
    (:import [org.apache.storm.tuple Tuple])
    (:import [org.apache.storm.generated StormTopology])
-   (:import [org.apache.storm.task TopologyContext])
+   (:import [org.apache.storm.task TopologyContext]
+            (org.apache.storm.messaging IContext)
+            [org.json.simple JSONValue])
 -  (:require [org.apache.storm [zookeeper :as zk]])
 +  (:import [org.apache.storm.cluster ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
-   (:require [org.apache.storm.messaging.loader :as msg-loader])
    (:require [org.apache.storm.daemon.acker :as acker])
 -  (:use [org.apache.storm cluster util thrift config log local-state]))
 +  (:use [org.apache.storm util thrift config log local-state converter]))
  
  (defn feeder-spout
    [fields]

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/thrift.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/thrift.clj
index 4dc21f9,779c1d1..7aab729
--- a/storm-core/src/clj/org/apache/storm/thrift.clj
+++ b/storm-core/src/clj/org/apache/storm/thrift.clj
@@@ -29,8 -29,9 +29,9 @@@
    (:import [org.apache.storm.grouping CustomStreamGrouping])
    (:import [org.apache.storm.topology TopologyBuilder])
    (:import [org.apache.storm.clojure RichShellBolt RichShellSpout])
-   (:import [org.apache.thrift.transport TTransport])
+   (:import [org.apache.thrift.transport TTransport]
+            (org.json.simple JSONValue))
 -  (:use [org.apache.storm util config log zookeeper]))
 +  (:use [org.apache.storm util config log]))
  
  (defn instantiate-java-object
    [^JavaObject obj]

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/clj/org/apache/storm/util.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/util.clj
index 165d8ee,f685d12..72778bb
--- a/storm-core/src/clj/org/apache/storm/util.clj
+++ b/storm-core/src/clj/org/apache/storm/util.clj
@@@ -20,9 -20,8 +20,9 @@@
    (:import [java.io FileReader FileNotFoundException])
    (:import [java.nio.file Paths])
    (:import [org.apache.storm Config])
-   (:import [org.apache.storm.generated ErrorInfo])
-   (:import [org.apache.storm.utils Time Container ClojureTimerTask Utils
-             MutableObject MutableInt])
++ (:import [org.apache.storm.generated ErrorInfo])
+   (:import [org.apache.storm.utils Time ClojureTimerTask Utils
+             MutableObject])
    (:import [org.apache.storm.security.auth NimbusPrincipal])
    (:import [javax.security.auth Subject])
    (:import [java.util UUID Random ArrayList List Collections])
@@@ -262,58 -163,9 +164,19 @@@
                     (instance? Boolean x) (boolean x)
                     true x))
             s))
 +; move this func form convert.clj due to cyclic load dependency
 +(defn clojurify-error [^ErrorInfo error]
 +  (if error
 +    {
 +      :error (.get_error error)
 +      :time-secs (.get_error_time_secs error)
 +      :host (.get_host error)
 +      :port (.get_port error)
 +      }
 +    ))
  
- (defmacro with-file-lock
-   [path & body]
-   `(let [f# (File. ~path)
-          _# (.createNewFile f#)
-          rf# (RandomAccessFile. f# "rw")
-          lock# (.. rf# (getChannel) (lock))]
-      (try
-        ~@body
-        (finally
-          (.release lock#)
-          (.close rf#)))))
- 
- (defn tokenize-path
-   [^String path]
-   (let [toks (.split path "/")]
-     (vec (filter (complement empty?) toks))))
- 
- (defn assoc-conj
-   [m k v]
-   (merge-with concat m {k [v]}))
- 
- ;; returns [ones in first set not in second, ones in second set not in first]
- (defn set-delta
-   [old curr]
-   (let [s1 (set old)
-         s2 (set curr)]
-     [(set/difference s1 s2) (set/difference s2 s1)]))
- 
- (defn parent-path
-   [path]
-   (let [toks (tokenize-path path)]
-     (str "/" (str/join "/" (butlast toks)))))
- 
- (defn toks->path
-   [toks]
-   (str "/" (str/join "/" toks)))
- 
- (defn normalize-path
-   [^String path]
-   (toks->path (tokenize-path path)))
- 
+ ;TODO: We're keeping this function around until all the code using it is properly tranlated to java
+ ;TODO: by properly having the for loop IN THE JAVA FUNCTION that originally used this function.
  (defn map-val
    [afn amap]
    (into {}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index b30d1d2,0000000..0c663f0
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@@ -1,249 -1,0 +1,282 @@@
 +/**
 + * 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.APersistentMap;
++import clojure.lang.PersistentArrayMap;
++import clojure.lang.RT;
 +import org.apache.storm.Config;
 +import org.apache.storm.generated.ClusterWorkerHeartbeat;
 +import org.apache.storm.generated.ExecutorInfo;
 +import org.apache.storm.generated.ExecutorStats;
 +import org.apache.storm.generated.ProfileAction;
 +import org.apache.storm.utils.Utils;
 +import org.apache.zookeeper.ZooDefs;
 +import org.apache.zookeeper.data.ACL;
 +import org.apache.zookeeper.data.Id;
 +import org.apache.zookeeper.server.auth.DigestAuthenticationProvider;
 +
++import java.io.PrintWriter;
++import java.io.StringWriter;
 +import java.io.UnsupportedEncodingException;
 +import java.net.URLEncoder;
 +import java.security.NoSuchAlgorithmException;
 +import java.util.ArrayList;
 +import java.util.HashMap;
 +import java.util.List;
 +import java.util.Map;
 +
 +public class ClusterUtils {
 +
 +    public static final String ZK_SEPERATOR = "/";
 +
 +    public static final String ASSIGNMENTS_ROOT = "assignments";
 +    public static final String CODE_ROOT = "code";
 +    public static final String STORMS_ROOT = "storms";
 +    public static final String SUPERVISORS_ROOT = "supervisors";
 +    public static final String WORKERBEATS_ROOT = "workerbeats";
 +    public static final String BACKPRESSURE_ROOT = "backpressure";
 +    public static final String ERRORS_ROOT = "errors";
 +    public static final String BLOBSTORE_ROOT = "blobstore";
 +    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT = "blobstoremaxkeysequencenumber";
 +    public static final String NIMBUSES_ROOT = "nimbuses";
 +    public static final String CREDENTIALS_ROOT = "credentials";
 +    public static final String LOGCONFIG_ROOT = "logconfigs";
 +    public static final String PROFILERCONFIG_ROOT = "profilerconfigs";
 +
 +    public static final String ASSIGNMENTS_SUBTREE = ZK_SEPERATOR + ASSIGNMENTS_ROOT;
 +    public static final String STORMS_SUBTREE = ZK_SEPERATOR + STORMS_ROOT;
 +    public static final String SUPERVISORS_SUBTREE = ZK_SEPERATOR + SUPERVISORS_ROOT;
 +    public static final String WORKERBEATS_SUBTREE = ZK_SEPERATOR + WORKERBEATS_ROOT;
 +    public static final String BACKPRESSURE_SUBTREE = ZK_SEPERATOR + BACKPRESSURE_ROOT;
 +    public static final String ERRORS_SUBTREE = ZK_SEPERATOR + ERRORS_ROOT;
 +    public static final String BLOBSTORE_SUBTREE = ZK_SEPERATOR + BLOBSTORE_ROOT;
 +    public static final String BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE = ZK_SEPERATOR + BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_ROOT;
 +    public static final String NIMBUSES_SUBTREE = ZK_SEPERATOR + NIMBUSES_ROOT;
 +    public static final String CREDENTIALS_SUBTREE = ZK_SEPERATOR + CREDENTIALS_ROOT;
 +    public static final String LOGCONFIG_SUBTREE = ZK_SEPERATOR + LOGCONFIG_ROOT;
 +    public static final String PROFILERCONFIG_SUBTREE = ZK_SEPERATOR + PROFILERCONFIG_ROOT;
 +
 +    // A singleton instance allows us to mock delegated static methods in our
 +    // tests by subclassing.
 +    private static final ClusterUtils INSTANCE = new ClusterUtils();
 +    private static ClusterUtils _instance = INSTANCE;
 +
 +    /**
 +     * Provide an instance of this class for delegates to use. To mock out delegated methods, provide an instance of a subclass that overrides the
 +     * implementation of the delegated method.
 +     *
 +     * @param u a Cluster instance
 +     */
 +    public static void setInstance(ClusterUtils u) {
 +        _instance = u;
 +    }
 +
 +    /**
 +     * Resets the singleton instance to the default. This is helpful to reset the class to its original functionality when mocking is no longer desired.
 +     */
 +    public static void resetInstance() {
 +        _instance = INSTANCE;
 +    }
 +
 +    public static List<ACL> mkTopoOnlyAcls(Map topoConf) throws NoSuchAlgorithmException {
 +        List<ACL> aclList = null;
 +        String payload = (String) topoConf.get(Config.STORM_ZOOKEEPER_TOPOLOGY_AUTH_PAYLOAD);
 +        if (Utils.isZkAuthenticationConfiguredStormServer(topoConf)) {
 +            aclList = new ArrayList<>();
 +            ACL acl1 = ZooDefs.Ids.CREATOR_ALL_ACL.get(0);
 +            aclList.add(acl1);
 +            ACL acl2 = new ACL(ZooDefs.Perms.READ, new Id("digest", DigestAuthenticationProvider.generateDigest(payload)));
 +            aclList.add(acl2);
 +        }
 +        return aclList;
 +    }
 +
 +    public static String supervisorPath(String id) {
 +        return SUPERVISORS_SUBTREE + ZK_SEPERATOR + id;
 +    }
 +
 +    public static String assignmentPath(String id) {
 +        return ASSIGNMENTS_SUBTREE + ZK_SEPERATOR + id;
 +    }
 +
 +    public static String blobstorePath(String key) {
 +        return BLOBSTORE_SUBTREE + ZK_SEPERATOR + key;
 +    }
 +
 +    public static String blobstoreMaxKeySequenceNumberPath(String key) {
 +        return BLOBSTORE_MAX_KEY_SEQUENCE_NUMBER_SUBTREE + ZK_SEPERATOR + key;
 +    }
 +
 +    public static String nimbusPath(String id) {
 +        return NIMBUSES_SUBTREE + ZK_SEPERATOR + id;
 +    }
 +
 +    public static String stormPath(String id) {
 +        return STORMS_SUBTREE + ZK_SEPERATOR + id;
 +    }
 +
 +    public static String workerbeatStormRoot(String stormId) {
 +        return WORKERBEATS_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String workerbeatPath(String stormId, String node, Long port) {
 +        return workerbeatStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
 +    }
 +
 +    public static String backpressureStormRoot(String stormId) {
 +        return BACKPRESSURE_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String backpressurePath(String stormId, String node, Long port) {
 +        return backpressureStormRoot(stormId) + ZK_SEPERATOR + node + "-" + port;
 +    }
 +
 +    public static String errorStormRoot(String stormId) {
 +        return ERRORS_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String errorPath(String stormId, String componentId) {
 +        try {
 +            return errorStormRoot(stormId) + ZK_SEPERATOR + URLEncoder.encode(componentId, "UTF-8");
 +        } catch (UnsupportedEncodingException e) {
 +            throw Utils.wrapInRuntime(e);
 +        }
 +    }
 +
 +    public static String lastErrorPath(String stormId, String componentId) {
 +        return errorPath(stormId, componentId) + "-last-error";
 +    }
 +
 +    public static String credentialsPath(String stormId) {
 +        return CREDENTIALS_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String logConfigPath(String stormId) {
 +        return LOGCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String profilerConfigPath(String stormId) {
 +        return PROFILERCONFIG_SUBTREE + ZK_SEPERATOR + stormId;
 +    }
 +
 +    public static String profilerConfigPath(String stormId, String host, Long port, ProfileAction requestType) {
 +        return profilerConfigPath(stormId) + ZK_SEPERATOR + host + "_" + port + "_" + requestType;
 +    }
 +
 +    public static <T> T maybeDeserialize(byte[] serialized, Class<T> clazz) {
 +        if (serialized != null) {
 +            return Utils.deserialize(serialized, clazz);
 +        }
 +        return null;
 +    }
 +
-     // Ensures that we only return heartbeats for executors assigned to this worker
-     public static Map<ExecutorInfo, ClusterWorkerHeartbeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
-         Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhb = new HashMap<>();
++    /**
++     * Ensures that we only return heartbeats for executors assigned to this worker
++     * @param executors
++     * @param workerHeartbeat
++     * @return
++     */
++    public static Map<ExecutorInfo, APersistentMap> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
++        Map<ExecutorInfo, APersistentMap> executorWhb = new HashMap<>();
 +        Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
 +        for (ExecutorInfo executor : executors) {
 +            if (executorStatsMap.containsKey(executor)) {
-                 executorWhb.put(executor, workerHeartbeat);
++                APersistentMap executorBeat =
++                        new PersistentArrayMap(new Object[] { RT.keyword(null, "time-secs"), workerHeartbeat.get_time_secs(), RT.keyword(null, "uptime"),
++                                workerHeartbeat.get_uptime_secs(), RT.keyword(null, "stats"), workerHeartbeat.get_executor_stats().get(executor) });
++                executorWhb.put(executor, executorBeat);
 +            }
 +        }
 +        return executorWhb;
 +    }
 +
 +    public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
 +        if (stateStorage instanceof IStateStorage) {
 +            return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
 +        } else {
 +            IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
 +            return new StormClusterStateImpl(Storage, acls, context, true);
 +        }
 +
 +    }
 +
-     public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
-             throws Exception {
++    public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
 +        String className = null;
 +        IStateStorage stateStorage = null;
 +        if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
 +            className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
 +        } else {
 +            className = "org.apache.storm.cluster.ZKStateStorageFactory";
 +        }
 +        Class clazz = Class.forName(className);
 +        StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
 +        stateStorage = storageFactory.mkStore(config, auth_conf, acls, context);
 +        return stateStorage;
 +    }
 +
-     public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
-             throws Exception {
++    public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
 +        return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
 +    }
 +
 +    public static IStormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
 +        return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
 +    }
 +
 +    // TO be remove
 +    public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
 +        HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
 +        if (map == null) {
 +            return rtn;
 +        }
 +        for (Map.Entry<K, V> entry : map.entrySet()) {
 +            K key = entry.getKey();
 +            V val = entry.getValue();
 +            List<K> list = rtn.get(val);
 +            if (list == null) {
 +                list = new ArrayList<K>();
 +                rtn.put(entry.getValue(), list);
 +            }
 +            list.add(key);
 +        }
 +        return rtn;
 +    }
++
++    public static String StringifyError(Throwable error) {
++        String errorString = null;
++        StringWriter result = null;
++        PrintWriter printWriter = null;
++        try {
++            result = new StringWriter();
++            printWriter = new PrintWriter(result);
++            error.printStackTrace(printWriter);
++            if (result != null) {
++                errorString = result.toString();
++            }
++        } finally {
++            try {
++                if (result != null)
++                    result.close();
++                if (printWriter != null)
++                    printWriter.close();
++            } catch (Exception e) {
++            }
++        }
++        return errorString;
++    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index 59d1af7,0000000..01cf56a
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@@ -1,129 -1,0 +1,124 @@@
 +/**
 + * 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.APersistentMap;
 +import clojure.lang.IFn;
 +import org.apache.storm.generated.*;
 +import org.apache.storm.nimbus.NimbusInfo;
 +
 +import java.security.NoSuchAlgorithmException;
 +import java.util.List;
 +import java.util.Map;
 +
 +public interface IStormClusterState {
 +    public List<String> assignments(IFn callback);
 +
 +    public Assignment assignmentInfo(String stormId, IFn callback);
 +
 +    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
 +
 +    public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
 +
-     // returns key information under /storm/blobstore/key
 +    public List<String> blobstoreInfo(String blobKey);
 +
-     // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
 +    public List nimbuses();
 +
-     // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
 +    public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
 +
 +    public List<String> activeStorms();
 +
 +    public StormBase stormBase(String stormId, IFn callback);
 +
 +    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
 +
 +    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
 +
 +    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
 +
 +    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
 +
 +    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
 +
-     public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
++    public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
 +
 +    public List<String> supervisors(IFn callback);
 +
 +    public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
 +
 +    public void setupHeatbeats(String stormId);
 +
 +    public void teardownHeartbeats(String stormId);
 +
 +    public void teardownTopologyErrors(String stormId);
 +
 +    public List<String> heartbeatStorms();
 +
 +    public List<String> errorTopologies();
 +
 +    public void setTopologyLogConfig(String stormId, LogConfig logConfig);
 +
 +    public LogConfig topologyLogConfig(String stormId, IFn cb);
 +
 +    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
 +
 +    public void removeWorkerHeartbeat(String stormId, String node, Long port);
 +
 +    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
 +
 +    public void workerBackpressure(String stormId, String node, Long port, boolean on);
 +
 +    public boolean topologyBackpressure(String stormId, IFn callback);
 +
 +    public void setupBackpressure(String stormId);
 +
 +    public void removeWorkerBackpressure(String stormId, String node, Long port);
 +
 +    public void activateStorm(String stormId, StormBase stormBase);
 +
 +    public void updateStorm(String stormId, StormBase newElems);
 +
 +    public void removeStormBase(String stormId);
 +
 +    public void setAssignment(String stormId, Assignment info);
 +
-     // sets up information related to key consisting of nimbus
-     // host:port and version info of the blob
 +    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
 +
 +    public List<String> activeKeys();
 +
 +    public List<String> blobstore(IFn callback);
 +
 +    public void removeStorm(String stormId);
 +
 +    public void removeBlobstoreKey(String blobKey);
 +
 +    public void removeKeyVersion(String blobKey);
 +
-     public void reportError(String stormId, String componentId, String node, Long port, String error);
++    public void reportError(String stormId, String componentId, String node, Long port, Throwable error);
 +
 +    public List<ErrorInfo> errors(String stormId, String componentId);
 +
 +    public ErrorInfo lastError(String stormId, String componentId);
 +
 +    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
 +
 +    public Credentials credentials(String stormId, IFn callback);
 +
 +    public void disconnect();
 +
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index 1226c55,0000000..a9c4d89
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@@ -1,212 -1,0 +1,212 @@@
 +/**
 + * 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.APersistentMap;
 +import org.apache.curator.framework.state.ConnectionStateListener;
 +import org.apache.storm.callback.ZKStateChangedCallback;
 +import org.apache.storm.generated.*;
 +import org.apache.storm.pacemaker.PacemakerClient;
 +import org.apache.storm.utils.Utils;
 +import org.apache.zookeeper.data.ACL;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import java.util.List;
 +
 +public class PaceMakerStateStorage implements IStateStorage {
 +
 +    private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
 +
 +    private PacemakerClient pacemakerClient;
 +    private IStateStorage stateStorage;
 +    private static final int maxRetries = 10;
 +
 +    public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
 +        this.pacemakerClient = pacemakerClient;
 +        this.stateStorage = stateStorage;
 +    }
 +
 +    @Override
 +    public String register(ZKStateChangedCallback callback) {
 +        return stateStorage.register(callback);
 +    }
 +
 +    @Override
 +    public void unregister(String id) {
 +        stateStorage.unregister(id);
 +    }
 +
 +    @Override
 +    public String create_sequential(String path, byte[] data, List<ACL> acls) {
 +        return stateStorage.create_sequential(path, data, acls);
 +    }
 +
 +    @Override
 +    public void mkdirs(String path, List<ACL> acls) {
 +        stateStorage.mkdirs(path, acls);
 +    }
 +
 +    @Override
 +    public void delete_node(String path) {
 +        stateStorage.delete_node(path);
 +    }
 +
 +    @Override
 +    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
 +        stateStorage.set_ephemeral_node(path, data, acls);
 +    }
 +
 +    @Override
 +    public Integer get_version(String path, boolean watch) throws Exception {
 +        return stateStorage.get_version(path, watch);
 +    }
 +
 +    @Override
 +    public boolean node_exists(String path, boolean watch) {
 +        return stateStorage.node_exists(path, watch);
 +    }
 +
 +    @Override
 +    public List<String> get_children(String path, boolean watch) {
 +        return stateStorage.get_children(path, watch);
 +    }
 +
 +    @Override
 +    public void close() {
 +        stateStorage.close();
 +        pacemakerClient.close();
 +    }
 +
 +    @Override
 +    public void set_data(String path, byte[] data, List<ACL> acls) {
 +        stateStorage.set_data(path, data, acls);
 +    }
 +
 +    @Override
 +    public byte[] get_data(String path, boolean watch) {
 +        return stateStorage.get_data(path, watch);
 +    }
 +
 +    @Override
 +    public APersistentMap get_data_with_version(String path, boolean watch) {
 +        return stateStorage.get_data_with_version(path, watch);
 +    }
 +
 +    @Override
 +    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
 +                HBPulse hbPulse = new HBPulse();
 +                hbPulse.set_id(path);
 +                hbPulse.set_details(data);
 +                HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful set_worker_hb");
 +                break;
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public byte[] get_worker_hb(String path, boolean watch) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
 +                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                return response.get_data().get_pulse().get_details();
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public List<String> get_worker_hb_children(String path, boolean watch) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
-                 HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++                HBMessage message = new HBMessage(HBServerMessageType.GET_ALL_NODES_FOR_PATH, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                return response.get_data().get_nodes().get_pulseIds();
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public void delete_worker_hb(String path) {
 +        int retry = maxRetries;
 +        while (true) {
 +            try {
-                 HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
++                HBMessage message = new HBMessage(HBServerMessageType.DELETE_PATH, HBMessageData.path(path));
 +                HBMessage response = pacemakerClient.send(message);
 +                if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
 +                    throw new HBExecutionException("Invalid Response Type");
 +                }
 +                LOG.debug("Successful get_worker_hb");
 +                break;
 +            } catch (Exception e) {
 +                if (retry <= 0) {
 +                    throw Utils.wrapInRuntime(e);
 +                }
 +                LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
 +            }
 +        }
 +    }
 +
 +    @Override
 +    public void add_listener(ConnectionStateListener listener) {
 +        stateStorage.add_listener(listener);
 +    }
 +
 +    @Override
 +    public void sync_path(String path) {
 +        stateStorage.sync_path(path);
 +    }
 +
 +    @Override
 +    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
 +        stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/storm/blob/924f9a29/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --cc storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index c2477d6,0000000..110da41
mode 100644,000000..100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@@ -1,28 -1,0 +1,28 @@@
 +/**
 + * 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.APersistentMap;
 +import java.util.List;
 +import org.apache.zookeeper.data.ACL;
 +
 +public interface StateStorageFactory {
-     
++
 +    IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
 +
 +}


[15/27] storm git commit: remove some useless code

Posted by bo...@apache.org.
remove some useless code


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/419be35d
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/419be35d
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/419be35d

Branch: refs/heads/master
Commit: 419be35d5ed0cbbc9abdf89058c1b484ecbf8e6d
Parents: 924f9a2 9a8962d
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 16 11:17:22 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:17:22 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/daemon/supervisor.clj | 3 ++-
 storm-core/test/clj/org/apache/storm/nimbus_test.clj      | 2 ++
 2 files changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[22/27] storm git commit: Merge branch 'master' into ClusterUtils

Posted by bo...@apache.org.
Merge branch 'master' into ClusterUtils


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/581cd298
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/581cd298
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/581cd298

Branch: refs/heads/master
Commit: 581cd29876fab988477ac661174cad5ffbeced1e
Parents: 3717f39 58050a5
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 23 19:54:31 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 23 19:56:06 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |  2 +
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  2 +-
 .../clj/org/apache/storm/daemon/supervisor.clj  |  8 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  3 +-
 .../clj/org/apache/storm/process_simulator.clj  | 49 ------------
 storm-core/src/clj/org/apache/storm/testing.clj |  9 ++-
 storm-core/src/clj/org/apache/storm/ui/core.clj |  4 +-
 .../jvm/org/apache/storm/ProcessSimulator.java  | 82 ++++++++++++++++++++
 8 files changed, 98 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 7d28075,28a6fb8..e43bab9
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@@ -1212,10 -1212,10 +1212,10 @@@
        (.deleteBlob blob-store key nimbus-subject))
      (log-debug "Creating list of key entries for blobstore inside zookeeper" all-keys "local" locally-available-active-keys)
      (doseq [key locally-available-active-keys]
 -      (.setup-blobstore! storm-cluster-state key (:nimbus-host-port-info nimbus) (get-version-for-key key nimbus-host-port-info conf)))))
 +      (.setupBlobstore storm-cluster-state key (:nimbus-host-port-info nimbus) (get-version-for-key key nimbus-host-port-info conf)))))
  
  (defn- get-errors [storm-cluster-state storm-id component-id]
-   (->> (apply clojurify-error (.errors storm-cluster-state storm-id component-id))
 -  (->> (.errors storm-cluster-state storm-id component-id)
++  (->> (map clojurify-error (.errors storm-cluster-state storm-id component-id))
         (map #(doto (ErrorInfo. (:error %) (:time-secs %))
                     (.set_host (:host %))
                     (.set_port (:port %))))))

http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index a6f3a62,a34d461..0cee414
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@@ -35,7 -35,7 +35,7 @@@
    (:use [org.apache.storm.daemon common])
    (:import [org.apache.storm.command HealthCheck])
    (:require [org.apache.storm.daemon [worker :as worker]]
-             [org.apache.storm [process-simulator :as psim]]
 -            [org.apache.storm [cluster :as cluster]]
++
              [clojure.set :as set])
    (:import [org.apache.thrift.transport TTransportException])
    (:import [org.apache.zookeeper data.ACL ZooDefs$Ids ZooDefs$Perms])

http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --cc storm-core/src/clj/org/apache/storm/daemon/worker.clj
index a75dc35,db4a61a..af88f6a
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@@ -19,8 -19,9 +19,9 @@@
    (:require [clj-time.core :as time])
    (:require [clj-time.coerce :as coerce])
    (:require [org.apache.storm.daemon [executor :as executor]])
 -  (:require [org.apache.storm [cluster :as cluster]])
 +
    (:require [clojure.set :as set])
+   (:import [java.io File])
    (:import [java.util.concurrent Executors]
             [org.apache.storm.hooks IWorkerHook BaseWorkerHook]
             [uk.org.lidalia.sysoutslf4j.context SysOutOverSLF4J])

http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/storm/blob/581cd298/storm-core/src/clj/org/apache/storm/ui/core.clj
----------------------------------------------------------------------


[08/27] storm git commit: update class hierarchy about cluster

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
new file mode 100644
index 0000000..cd2bc4a
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -0,0 +1,664 @@
+/**
+ * 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 StormClusterState {
+
+    private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
+
+    private StateStorage stateStorage;
+
+    private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
+    private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
+    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
+    private AtomicReference<IFn> supervisorsCallback;
+    // we want to reigister a topo directory getChildren callback for all workers of this dir
+    private ConcurrentHashMap<String, IFn> backPressureCallback;
+    private AtomicReference<IFn> assignmentsCallback;
+    private ConcurrentHashMap<String, IFn> stormBaseCallback;
+    private AtomicReference<IFn> blobstoreCallback;
+    private ConcurrentHashMap<String, IFn> credentialsCallback;
+    private ConcurrentHashMap<String, IFn> logConfigCallback;
+
+    private List<ACL> acls;
+    private String stateId;
+    private boolean solo;
+
+    public StormClusterStateImpl(StateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+
+        this.stateStorage = StateStorage;
+        this.solo = solo;
+
+        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 params = null;
+                    String root = toks.get(0);
+                    IFn fn = null;
+                    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<IFn> cb) {
+        IFn callback = cb.getAndSet(null);
+        if (callback != null)
+            callback.invoke();
+    }
+
+    protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
+        IFn callback = callbackConcurrentHashMap.remove(key);
+        if (callback != null)
+            callback.invoke();
+    }
+
+    @Override
+    public List<String> assignments(IFn callback) {
+        if (callback != null) {
+            assignmentsCallback.set(callback);
+        }
+        return stateStorage.get_children(ClusterUtils.ASSIGNMENTS_SUBTREE, callback != null);
+    }
+
+    @Override
+    public Assignment assignmentInfo(String stormId, IFn 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 APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
+        if (callback != null) {
+            assignmentInfoWithVersionCallback.put(stormId, callback);
+        }
+        Assignment assignment = null;
+        Integer version = 0;
+        APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
+        if (aPersistentMap != null) {
+            assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
+            version = (Integer) aPersistentMap.get(RT.keyword(null, "version"));
+        }
+        APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
+        return map;
+    }
+
+    @Override
+    public Integer assignmentVersion(String stormId, IFn 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, IFn 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
+    @Override
+    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
+
+        Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.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(IFn 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 (Zookeeper.exceptionCause(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 (Zookeeper.exceptionCause(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, IFn cb) {
+        String path = ClusterUtils.logConfigPath(stormId);
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
+    }
+
+    @Override
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
+        if (info != null) {
+            String path = ClusterUtils.workerbeatPath(stormId, node, port);
+            stateStorage.set_worker_hb(path, Utils.serialize(info), acls);
+        }
+    }
+
+    @Override
+    public void removeWorkerHeartbeat(String stormId, String node, Long port) {
+        String path = ClusterUtils.workerbeatPath(stormId, node, port);
+        stateStorage.delete_worker_hb(path);
+    }
+
+    @Override
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
+        String path = ClusterUtils.supervisorPath(supervisorId);
+        stateStorage.set_ephemeral_node(path, Utils.serialize(info), acls);
+    }
+
+    // if znode exists and to be not on?, delete; if exists and on?, do nothing;
+    // if not exists and to be on?, create; if not exists and not on?, do nothing;
+    @Override
+    public void workerBackpressure(String stormId, String node, Long port, boolean on) {
+        String path = ClusterUtils.backpressurePath(stormId, node, port);
+        boolean existed = stateStorage.node_exists(path, false);
+        if (existed) {
+            if (on == false)
+                stateStorage.delete_node(path);
+
+        } else {
+            if (on == true) {
+                stateStorage.set_ephemeral_node(path, null, acls);
+            }
+        }
+    }
+
+    // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
+    @Override
+    public boolean topologyBackpressure(String stormId, IFn callback) {
+        if (callback != null) {
+            backPressureCallback.put(stormId, callback);
+        }
+        String path = ClusterUtils.backpressureStormRoot(stormId);
+        List<String> childrens = stateStorage.get_children(path, callback != null);
+        return childrens.size() > 0;
+
+    }
+
+    @Override
+    public void setupBackpressure(String stormId) {
+        stateStorage.mkdirs(ClusterUtils.backpressureStormRoot(stormId), acls);
+    }
+
+    @Override
+    public void removeWorkerBackpressure(String stormId, String node, Long port) {
+        stateStorage.delete_node(ClusterUtils.backpressurePath(stormId, node, port));
+    }
+
+    @Override
+    public void activateStorm(String stormId, StormBase stormBase) {
+        String path = ClusterUtils.stormPath(stormId);
+        stateStorage.set_data(path, Utils.serialize(stormBase), acls);
+    }
+
+    // To update this function due to APersistentMap/APersistentSet is clojure's structure
+    @Override
+    public void updateStorm(String stormId, StormBase newElems) {
+
+        StormBase stormBase = stormBase(stormId, null);
+        if (stormBase.get_component_executors() != null) {
+
+            Map<String, Integer> newComponentExecutors = new HashMap<>();
+            Map<String, Integer> componentExecutors = newElems.get_component_executors();
+            // componentExecutors maybe be APersistentMap, which don't support "put"
+            for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
+                newComponentExecutors.put(entry.getKey(), entry.getValue());
+            }
+            for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
+                if (!componentExecutors.containsKey(entry.getKey())) {
+                    newComponentExecutors.put(entry.getKey(), entry.getValue());
+                }
+            }
+            if (newComponentExecutors.size() > 0)
+                newElems.set_component_executors(newComponentExecutors);
+        }
+
+        Map<String, DebugOptions> ComponentDebug = new HashMap<>();
+        Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
+
+        Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
+        /// oldComponentDebug.keySet()/ newComponentDebug.keySet() maybe be APersistentSet, which don't support addAll
+        Set<String> debugOptionsKeys = new HashSet<>();
+        debugOptionsKeys.addAll(oldComponentDebug.keySet());
+        debugOptionsKeys.addAll(newComponentDebug.keySet());
+        for (String key : debugOptionsKeys) {
+            boolean enable = false;
+            double samplingpct = 0;
+            if (oldComponentDebug.containsKey(key)) {
+                enable = oldComponentDebug.get(key).is_enable();
+                samplingpct = oldComponentDebug.get(key).get_samplingpct();
+            }
+            if (newComponentDebug.containsKey(key)) {
+                enable = newComponentDebug.get(key).is_enable();
+                samplingpct += newComponentDebug.get(key).get_samplingpct();
+            }
+            DebugOptions debugOptions = new DebugOptions();
+            debugOptions.set_enable(enable);
+            debugOptions.set_samplingpct(samplingpct);
+            ComponentDebug.put(key, debugOptions);
+        }
+        if (ComponentDebug.size() > 0) {
+            newElems.set_component_debug(ComponentDebug);
+        }
+
+        if (StringUtils.isBlank(newElems.get_name())) {
+            newElems.set_name(stormBase.get_name());
+        }
+        if (newElems.get_status() == null) {
+            newElems.set_status(stormBase.get_status());
+        }
+        if (newElems.get_num_workers() == 0) {
+            newElems.set_num_workers(stormBase.get_num_workers());
+        }
+        if (newElems.get_launch_time_secs() == 0) {
+            newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
+        }
+        if (StringUtils.isBlank(newElems.get_owner())) {
+            newElems.set_owner(stormBase.get_owner());
+        }
+        if (newElems.get_topology_action_options() == null) {
+            newElems.set_topology_action_options(stormBase.get_topology_action_options());
+        }
+        if (newElems.get_status() == null) {
+            newElems.set_status(stormBase.get_status());
+        }
+        stateStorage.set_data(ClusterUtils.stormPath(stormId), Utils.serialize(newElems), acls);
+    }
+
+    @Override
+    public void removeStormBase(String stormId) {
+        stateStorage.delete_node(ClusterUtils.stormPath(stormId));
+    }
+
+    @Override
+    public void setAssignment(String stormId, Assignment info) {
+        stateStorage.set_data(ClusterUtils.assignmentPath(stormId), Utils.serialize(info), acls);
+    }
+
+    @Override
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
+        String path = ClusterUtils.blobstorePath(key) + ClusterUtils.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
+        LOG.info("set-path: {}", path);
+        stateStorage.mkdirs(ClusterUtils.blobstorePath(key), acls);
+        stateStorage.delete_node_blobstore(ClusterUtils.blobstorePath(key), nimbusInfo.toHostPortString());
+        stateStorage.set_ephemeral_node(path, null, acls);
+    }
+
+    @Override
+    public List<String> activeKeys() {
+        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, false);
+    }
+
+    // blobstore state
+    @Override
+    public List<String> blobstore(IFn callback) {
+        if (callback != null) {
+            blobstoreCallback.set(callback);
+        }
+        stateStorage.sync_path(ClusterUtils.BLOBSTORE_SUBTREE);
+        return stateStorage.get_children(ClusterUtils.BLOBSTORE_SUBTREE, callback != null);
+
+    }
+
+    @Override
+    public void removeStorm(String stormId) {
+        stateStorage.delete_node(ClusterUtils.assignmentPath(stormId));
+        stateStorage.delete_node(ClusterUtils.credentialsPath(stormId));
+        stateStorage.delete_node(ClusterUtils.logConfigPath(stormId));
+        stateStorage.delete_node(ClusterUtils.profilerConfigPath(stormId));
+        removeStormBase(stormId);
+    }
+
+    @Override
+    public void removeBlobstoreKey(String blobKey) {
+        LOG.debug("remove key {}", blobKey);
+        stateStorage.delete_node(ClusterUtils.blobstorePath(blobKey));
+    }
+
+    @Override
+    public void removeKeyVersion(String blobKey) {
+        stateStorage.delete_node(ClusterUtils.blobstoreMaxKeySequenceNumberPath(blobKey));
+    }
+
+    @Override
+    public void reportError(String stormId, String componentId, String node, Long port, String error) {
+
+        String path = ClusterUtils.errorPath(stormId, componentId);
+        String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
+        ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
+        errorInfo.set_host(node);
+        errorInfo.set_port(port.intValue());
+        byte[] serData = Utils.serialize(errorInfo);
+        stateStorage.mkdirs(path, acls);
+        stateStorage.create_sequential(path + ClusterUtils.ZK_SEPERATOR + "e", serData, acls);
+        stateStorage.set_data(lastErrorPath, serData, acls);
+        List<String> childrens = stateStorage.get_children(path, false);
+
+        Collections.sort(childrens, new Comparator<String>() {
+            public int compare(String arg0, String arg1) {
+                return Long.compare(Long.parseLong(arg0.substring(1)), Long.parseLong(arg1.substring(1)));
+            }
+        });
+
+        while (childrens.size() > 10) {
+            stateStorage.delete_node(path + ClusterUtils.ZK_SEPERATOR + childrens.remove(0));
+        }
+    }
+
+    @Override
+    public List<ErrorInfo> errors(String stormId, String componentId) {
+        List<ErrorInfo> errorInfos = new ArrayList<>();
+        try {
+            String path = ClusterUtils.errorPath(stormId, componentId);
+            if (stateStorage.node_exists(path, false)) {
+                List<String> childrens = stateStorage.get_children(path, false);
+                for (String child : childrens) {
+                    String childPath = path + ClusterUtils.ZK_SEPERATOR + child;
+                    ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(childPath, false), ErrorInfo.class);
+                    if (errorInfo != null)
+                        errorInfos.add(errorInfo);
+                }
+            }
+            Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
+                public int compare(ErrorInfo arg0, ErrorInfo arg1) {
+                    return -Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+                }
+            });
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+
+        return errorInfos;
+    }
+
+    @Override
+    public ErrorInfo lastError(String stormId, String componentId) {
+
+        String path = ClusterUtils.lastErrorPath(stormId, componentId);
+        if (stateStorage.node_exists(path, false)) {
+            ErrorInfo errorInfo = ClusterUtils.maybeDeserialize(stateStorage.get_data(path, false), ErrorInfo.class);
+            return errorInfo;
+        }
+
+        return null;
+    }
+
+    @Override
+    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
+        List<ACL> aclList = ClusterUtils.mkTopoOnlyAcls(topoConf);
+        String path = ClusterUtils.credentialsPath(stormId);
+        stateStorage.set_data(path, Utils.serialize(creds), aclList);
+
+    }
+
+    @Override
+    public Credentials credentials(String stormId, IFn callback) {
+        if (callback != null) {
+            credentialsCallback.put(stormId, callback);
+        }
+        String path = ClusterUtils.credentialsPath(stormId);
+        return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, callback != null), Credentials.class);
+
+    }
+
+    @Override
+    public void disconnect() {
+        stateStorage.unregister(stateId);
+        if (solo)
+            stateStorage.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
deleted file mode 100644
index 3a4205b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormZkClusterState.java
+++ /dev/null
@@ -1,683 +0,0 @@
-/**
- * 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.APersistentMap;
-import clojure.lang.IFn;
-import clojure.lang.PersistentArrayMap;
-import clojure.lang.RT;
-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.Callback;
-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.data.ACL;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.UnsupportedEncodingException;
-import java.security.NoSuchAlgorithmException;
-import java.util.*;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicReference;
-
-public class StormZkClusterState implements StormClusterState {
-
-    private static Logger LOG = LoggerFactory.getLogger(StormZkClusterState.class);
-
-    private ClusterState clusterState;
-
-    private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
-    private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
-    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
-    private AtomicReference<IFn> supervisorsCallback;
-    // we want to reigister a topo directory getChildren callback for all workers of this dir
-    private ConcurrentHashMap<String, IFn> backPressureCallback;
-    private AtomicReference<IFn> assignmentsCallback;
-    private ConcurrentHashMap<String, IFn> stormBaseCallback;
-    private AtomicReference<IFn> blobstoreCallback;
-    private ConcurrentHashMap<String, IFn> credentialsCallback;
-    private ConcurrentHashMap<String, IFn> logConfigCallback;
-
-    private List<ACL> acls;
-    private String stateId;
-    private boolean solo;
-
-    public StormZkClusterState(Object clusterState, List<ACL> acls, ClusterStateContext context) throws Exception {
-
-        if (clusterState instanceof ClusterState) {
-            solo = false;
-            this.clusterState = (ClusterState) clusterState;
-        } else {
-
-            solo = true;
-            this.clusterState = new DistributedClusterState((Map) clusterState, (Map) clusterState, acls, context);
-        }
-
-        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.clusterState.register(new Callback() {
-
-            public <T> Object execute(T... args) {
-                if (args == null) {
-                    LOG.warn("Input args is null");
-                    return null;
-                } else if (args.length < 2) {
-                    LOG.warn("Input args is invalid, args length:" + args.length);
-                    return null;
-                }
-                String path = (String) args[1];
-
-                List<String> toks = Zookeeper.tokenizePath(path);
-                int size = toks.size();
-                if (size >= 1) {
-                    String params = null;
-                    String root = toks.get(0);
-                    IFn fn = null;
-                    if (root.equals(Cluster.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(Cluster.SUPERVISORS_ROOT)) {
-                        issueCallback(supervisorsCallback);
-                    } else if (root.equals(Cluster.BLOBSTORE_ROOT)) {
-                        issueCallback(blobstoreCallback);
-                    } else if (root.equals(Cluster.STORMS_ROOT) && size > 1) {
-                        issueMapCallback(stormBaseCallback, toks.get(1));
-                    } else if (root.equals(Cluster.CREDENTIALS_ROOT) && size > 1) {
-                        issueMapCallback(credentialsCallback, toks.get(1));
-                    } else if (root.equals(Cluster.LOGCONFIG_ROOT) && size > 1) {
-                        issueMapCallback(logConfigCallback, toks.get(1));
-                    } else if (root.equals(Cluster.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 null;
-            }
-
-        });
-
-        String[] pathlist = { Cluster.ASSIGNMENTS_SUBTREE, Cluster.STORMS_SUBTREE, Cluster.SUPERVISORS_SUBTREE, Cluster.WORKERBEATS_SUBTREE,
-                Cluster.ERRORS_SUBTREE, Cluster.BLOBSTORE_SUBTREE, Cluster.NIMBUSES_SUBTREE, Cluster.LOGCONFIG_SUBTREE };
-        for (String path : pathlist) {
-            this.clusterState.mkdirs(path, acls);
-        }
-
-    }
-
-    protected void issueCallback(AtomicReference<IFn> cb) {
-        IFn callback = cb.getAndSet(null);
-        if (callback != null)
-            callback.invoke();
-    }
-
-    protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
-        IFn callback = callbackConcurrentHashMap.remove(key);
-        if (callback != null)
-            callback.invoke();
-    }
-
-    @Override
-    public List<String> assignments(IFn callback) {
-        if (callback != null) {
-            assignmentsCallback.set(callback);
-        }
-        return clusterState.get_children(Cluster.ASSIGNMENTS_SUBTREE, callback != null);
-    }
-
-    @Override
-    public Assignment assignmentInfo(String stormId, IFn callback) {
-        if (callback != null) {
-            assignmentInfoCallback.put(stormId, callback);
-        }
-        byte[] serialized = clusterState.get_data(Cluster.assignmentPath(stormId), callback != null);
-        return Cluster.maybeDeserialize(serialized, Assignment.class);
-    }
-
-    @Override
-    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
-        if (callback != null) {
-            assignmentInfoWithVersionCallback.put(stormId, callback);
-        }
-        APersistentMap aPersistentMap = clusterState.get_data_with_version(Cluster.assignmentPath(stormId), callback != null);
-        Assignment assignment = Cluster.maybeDeserialize((byte[]) aPersistentMap.get("data"), Assignment.class);
-        Integer version = (Integer) aPersistentMap.get("version");
-        APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
-        return map;
-    }
-
-    @Override
-    public Integer assignmentVersion(String stormId, IFn callback) throws Exception {
-        if (callback != null) {
-            assignmentVersionCallback.put(stormId, callback);
-        }
-        return clusterState.get_version(Cluster.assignmentPath(stormId), callback != null);
-    }
-
-    // blobstore state
-    @Override
-    public List<String> blobstoreInfo(String blobKey) {
-        String path = Cluster.blobstorePath(blobKey);
-        clusterState.sync_path(path);
-        return clusterState.get_children(path, false);
-    }
-
-    @Override
-    public List nimbuses() {
-        List<NimbusSummary> nimbusSummaries = new ArrayList<>();
-        List<String> nimbusIds = clusterState.get_children(Cluster.NIMBUSES_SUBTREE, false);
-        for (String nimbusId : nimbusIds) {
-            byte[] serialized = clusterState.get_data(Cluster.nimbusPath(nimbusId), false);
-            NimbusSummary nimbusSummary = Cluster.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.
-        clusterState.delete_node(Cluster.nimbusPath(nimbusId));
-        clusterState.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");
-                    clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
-                }
-
-            }
-        });
-
-        clusterState.set_ephemeral_node(Cluster.nimbusPath(nimbusId), Utils.serialize(nimbusSummary), acls);
-    }
-
-    @Override
-    public List<String> activeStorms() {
-        return clusterState.get_children(Cluster.STORMS_SUBTREE, false);
-    }
-
-    @Override
-    public StormBase stormBase(String stormId, IFn callback) {
-        if (callback != null) {
-            stormBaseCallback.put(stormId, callback);
-        }
-        return Cluster.maybeDeserialize(clusterState.get_data(Cluster.stormPath(stormId), callback != null), StormBase.class);
-    }
-
-    @Override
-    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port) {
-        byte[] bytes = clusterState.get_worker_hb(Cluster.workerbeatPath(stormId, node, port), false);
-        if (bytes != null) {
-            return Cluster.maybeDeserialize(bytes, ClusterWorkerHeartbeat.class);
-        }
-        return null;
-    }
-
-    @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 = Cluster.profilerConfigPath(stormId);
-        if (clusterState.node_exists(path, false)) {
-            List<String> strs = clusterState.get_children(path, false);
-            for (String str : strs) {
-                String childPath = path + Cluster.ZK_SEPERATOR + str;
-                byte[] raw = clusterState.get_data(childPath, false);
-                ProfileRequest request = Cluster.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 = Cluster.profilerConfigPath(stormId, host, port, profileAction);
-        clusterState.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 = Cluster.profilerConfigPath(stormId, host, port, profileAction);
-        clusterState.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
-    @Override
-    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
-        Map<ExecutorInfo, ClusterWorkerHeartbeat> executorWhbs = new HashMap<>();
-
-        LOG.info(executorNodePort.toString());
-        Map<NodeInfo, List<List<Long>>> nodePortExecutors = Cluster.reverseMap(executorNodePort);
-        LOG.info(nodePortExecutors.toString());
-
-        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()));
-            }
-            executorWhbs.putAll(Cluster.convertExecutorBeats(executorInfoList, whb));
-        }
-        return executorWhbs;
-    }
-
-    @Override
-    public List<String> supervisors(IFn callback) {
-        if (callback != null) {
-            supervisorsCallback.set(callback);
-        }
-        return clusterState.get_children(Cluster.SUPERVISORS_SUBTREE, callback != null);
-    }
-
-    @Override
-    public SupervisorInfo supervisorInfo(String supervisorId) {
-        String path = Cluster.supervisorPath(supervisorId);
-        return Cluster.maybeDeserialize(clusterState.get_data(path, false), SupervisorInfo.class);
-    }
-
-    @Override
-    public void setupHeatbeats(String stormId) {
-        clusterState.mkdirs(Cluster.workerbeatStormRoot(stormId), acls);
-    }
-
-    @Override
-    public void teardownHeartbeats(String stormId) {
-        try {
-            clusterState.delete_worker_hb(Cluster.workerbeatStormRoot(stormId));
-        } catch (Exception e) {
-            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
-                // do nothing
-                LOG.warn("Could not teardown heartbeats for {}.", stormId);
-            } else {
-                throw e;
-            }
-        }
-    }
-
-    @Override
-    public void teardownTopologyErrors(String stormId) {
-        try {
-            clusterState.delete_node(Cluster.errorStormRoot(stormId));
-        } catch (Exception e) {
-            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
-                // do nothing
-                LOG.warn("Could not teardown errors for {}.", stormId);
-            } else {
-                throw e;
-            }
-        }
-    }
-
-    @Override
-    public List<String> heartbeatStorms() {
-        return clusterState.get_worker_hb_children(Cluster.WORKERBEATS_SUBTREE, false);
-    }
-
-    @Override
-    public List<String> errorTopologies() {
-        return clusterState.get_children(Cluster.ERRORS_SUBTREE, false);
-    }
-
-    @Override
-    public void setTopologyLogConfig(String stormId, LogConfig logConfig) {
-        clusterState.set_data(Cluster.logConfigPath(stormId), Utils.serialize(logConfig), acls);
-    }
-
-    @Override
-    public LogConfig topologyLogConfig(String stormId, IFn cb) {
-        String path = Cluster.logConfigPath(stormId);
-        return Cluster.maybeDeserialize(clusterState.get_data(path, cb != null), LogConfig.class);
-    }
-
-    @Override
-    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info) {
-        if (info != null) {
-            String path = Cluster.workerbeatPath(stormId, node, port);
-            clusterState.set_worker_hb(path, Utils.serialize(info), acls);
-        }
-    }
-
-    @Override
-    public void removeWorkerHeartbeat(String stormId, String node, Long port) {
-        String path = Cluster.workerbeatPath(stormId, node, port);
-        clusterState.delete_worker_hb(path);
-    }
-
-    @Override
-    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info) {
-        String path = Cluster.supervisorPath(supervisorId);
-        clusterState.set_ephemeral_node(path, Utils.serialize(info), acls);
-    }
-
-    // if znode exists and to be not on?, delete; if exists and on?, do nothing;
-    // if not exists and to be on?, create; if not exists and not on?, do nothing;
-    @Override
-    public void workerBackpressure(String stormId, String node, Long port, boolean on) {
-        String path = Cluster.backpressurePath(stormId, node, port);
-        boolean existed = clusterState.node_exists(path, false);
-        if (existed) {
-            if (on == false)
-                clusterState.delete_node(path);
-
-        } else {
-            if (on == true) {
-                clusterState.set_ephemeral_node(path, null, acls);
-            }
-        }
-    }
-
-    // if the backpresure/storm-id dir is empty, this topology has throttle-on, otherwise not.
-    @Override
-    public boolean topologyBackpressure(String stormId, IFn callback) {
-        if (callback != null) {
-            backPressureCallback.put(stormId, callback);
-        }
-        String path = Cluster.backpressureStormRoot(stormId);
-        List<String> childrens = clusterState.get_children(path, callback != null);
-        return childrens.size() > 0;
-
-    }
-
-    @Override
-    public void setupBackpressure(String stormId) {
-        clusterState.mkdirs(Cluster.backpressureStormRoot(stormId), acls);
-    }
-
-    @Override
-    public void removeWorkerBackpressure(String stormId, String node, Long port) {
-        clusterState.delete_node(Cluster.backpressurePath(stormId, node, port));
-    }
-
-    @Override
-    public void activateStorm(String stormId, StormBase stormBase) {
-        String path = Cluster.stormPath(stormId);
-        clusterState.set_data(path, Utils.serialize(stormBase), acls);
-    }
-
-    // maybe exit some questions for updateStorm
-    @Override
-    public void updateStorm(String stormId, StormBase newElems) {
-
-        StormBase stormBase = stormBase(stormId, null);
-        if (stormBase.get_component_executors() != null) {
-
-            Map<String, Integer> newComponentExecutors = new HashMap<>();
-            Map<String, Integer> componentExecutors = newElems.get_component_executors();
-            //componentExecutors maybe be APersistentMap, which don't support put
-            for (Map.Entry<String, Integer> entry : componentExecutors.entrySet()) {
-                    newComponentExecutors.put(entry.getKey(), entry.getValue());
-            }
-            for (Map.Entry<String, Integer> entry : stormBase.get_component_executors().entrySet()) {
-                if (!componentExecutors.containsKey(entry.getKey())) {
-                    newComponentExecutors.put(entry.getKey(), entry.getValue());
-                }
-            }
-            if (newComponentExecutors.size() > 0)
-                newElems.set_component_executors(newComponentExecutors);
-        }
-
-        Map<String, DebugOptions> ComponentDebug = new HashMap<>();
-        Map<String, DebugOptions> oldComponentDebug = stormBase.get_component_debug();
-
-        Map<String, DebugOptions> newComponentDebug = newElems.get_component_debug();
-
-        Set<String> debugOptionsKeys = oldComponentDebug.keySet();
-        debugOptionsKeys.addAll(newComponentDebug.keySet());
-        for (String key : debugOptionsKeys) {
-            boolean enable = false;
-            double samplingpct = 0;
-            if (oldComponentDebug.containsKey(key)) {
-                enable = oldComponentDebug.get(key).is_enable();
-                samplingpct = oldComponentDebug.get(key).get_samplingpct();
-            }
-            if (newComponentDebug.containsKey(key)) {
-                enable = newComponentDebug.get(key).is_enable();
-                samplingpct += newComponentDebug.get(key).get_samplingpct();
-            }
-            DebugOptions debugOptions = new DebugOptions();
-            debugOptions.set_enable(enable);
-            debugOptions.set_samplingpct(samplingpct);
-            ComponentDebug.put(key, debugOptions);
-        }
-        if (ComponentDebug.size() > 0) {
-            newElems.set_component_debug(ComponentDebug);
-        }
-
-
-        if (StringUtils.isBlank(newElems.get_name())) {
-            newElems.set_name(stormBase.get_name());
-        }
-        if (newElems.get_status() == null){
-            newElems.set_status(stormBase.get_status());
-        }
-        if (newElems.get_num_workers() == 0){
-            newElems.set_num_workers(stormBase.get_num_workers());
-        }
-        if (newElems.get_launch_time_secs() == 0) {
-            newElems.set_launch_time_secs(stormBase.get_launch_time_secs());
-        }
-        if (StringUtils.isBlank(newElems.get_owner())) {
-            newElems.set_owner(stormBase.get_owner());
-        }
-        if (newElems.get_topology_action_options() == null) {
-            newElems.set_topology_action_options(stormBase.get_topology_action_options());
-        }
-        if (newElems.get_status() == null) {
-            newElems.set_status(stormBase.get_status());
-        }
-        clusterState.set_data(Cluster.stormPath(stormId), Utils.serialize(newElems), acls);
-    }
-
-    @Override
-    public void removeStormBase(String stormId) {
-        clusterState.delete_node(Cluster.stormPath(stormId));
-    }
-
-    @Override
-    public void setAssignment(String stormId, Assignment info) {
-        clusterState.set_data(Cluster.assignmentPath(stormId), Utils.serialize(info), acls);
-    }
-
-    @Override
-    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo) {
-        String path = Cluster.blobstorePath(key) + Cluster.ZK_SEPERATOR + nimbusInfo.toHostPortString() + "-" + versionInfo;
-        LOG.info("set-path: {}", path);
-        clusterState.mkdirs(Cluster.blobstorePath(key), acls);
-        clusterState.delete_node_blobstore(Cluster.blobstorePath(key), nimbusInfo.toHostPortString());
-        clusterState.set_ephemeral_node(path, null, acls);
-    }
-
-    @Override
-    public List<String> activeKeys() {
-        return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, false);
-    }
-
-    // blobstore state
-    @Override
-    public List<String> blobstore(IFn callback) {
-        if (callback != null) {
-            blobstoreCallback.set(callback);
-        }
-        clusterState.sync_path(Cluster.BLOBSTORE_SUBTREE);
-        return clusterState.get_children(Cluster.BLOBSTORE_SUBTREE, callback != null);
-
-    }
-
-    @Override
-    public void removeStorm(String stormId) {
-        clusterState.delete_node(Cluster.assignmentPath(stormId));
-        clusterState.delete_node(Cluster.credentialsPath(stormId));
-        clusterState.delete_node(Cluster.logConfigPath(stormId));
-        clusterState.delete_node(Cluster.profilerConfigPath(stormId));
-        removeStormBase(stormId);
-    }
-
-    @Override
-    public void removeBlobstoreKey(String blobKey) {
-        LOG.debug("remove key {}", blobKey);
-        clusterState.delete_node(Cluster.blobstorePath(blobKey));
-    }
-
-    @Override
-    public void removeKeyVersion(String blobKey) {
-        clusterState.delete_node(Cluster.blobstoreMaxKeySequenceNumberPath(blobKey));
-    }
-
-    @Override
-    public void reportError(String stormId, String componentId, String node, Integer port, String error) {
-
-        try {
-            String path = Cluster.errorPath(stormId, componentId);
-            String lastErrorPath = Cluster.lastErrorPath(stormId, componentId);
-            ErrorInfo errorInfo = new ErrorInfo(error, Time.currentTimeSecs());
-            errorInfo.set_host(node);
-            errorInfo.set_port(port.intValue());
-            byte[] serData = Utils.serialize(errorInfo);
-            clusterState.mkdirs(path, acls);
-            clusterState.create_sequential(path + Cluster.ZK_SEPERATOR + "e", serData, acls);
-            clusterState.set_data(lastErrorPath, serData, acls);
-            List<String> childrens = clusterState.get_children(path, false);
-
-            Collections.sort(childrens);
-
-            while (childrens.size() >= 10) {
-                clusterState.delete_node(path + Cluster.ZK_SEPERATOR + childrens.remove(0));
-            }
-        } catch (UnsupportedEncodingException e) {
-            throw Utils.wrapInRuntime(e);
-        }
-    }
-
-    @Override
-    public List<ErrorInfo> errors(String stormId, String componentId) {
-        List<ErrorInfo> errorInfos = new ArrayList<>();
-        try {
-            String path = Cluster.errorPath(stormId, componentId);
-            if (clusterState.node_exists(path, false)) {
-                List<String> childrens = clusterState.get_children(path, false);
-                for (String child : childrens) {
-                    String childPath = path + Cluster.ZK_SEPERATOR + child;
-                    ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(childPath, false), ErrorInfo.class);
-                    if (errorInfo != null)
-                        errorInfos.add(errorInfo);
-                }
-            }
-            Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
-                public int compare(ErrorInfo arg0, ErrorInfo arg1) {
-                    return Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
-                }
-            });
-        } catch (Exception e) {
-            throw Utils.wrapInRuntime(e);
-        }
-
-        return errorInfos;
-    }
-
-    @Override
-    public ErrorInfo lastError(String stormId, String componentId) {
-        try {
-            String path = Cluster.lastErrorPath(stormId, componentId);
-            if (clusterState.node_exists(path, false)) {
-                ErrorInfo errorInfo = Cluster.maybeDeserialize(clusterState.get_data(path, false), ErrorInfo.class);
-                return errorInfo;
-            }
-        } catch (UnsupportedEncodingException e) {
-            throw Utils.wrapInRuntime(e);
-        }
-        return null;
-    }
-
-    @Override
-    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException {
-        List<ACL> aclList = Cluster.mkTopoOnlyAcls(topoConf);
-        String path = Cluster.credentialsPath(stormId);
-        clusterState.set_data(path, Utils.serialize(creds), aclList);
-
-    }
-
-    @Override
-    public Credentials credentials(String stormId, IFn callback) {
-        if (callback != null) {
-            credentialsCallback.put(stormId, callback);
-        }
-        String path = Cluster.credentialsPath(stormId);
-        return Cluster.maybeDeserialize(clusterState.get_data(path, callback != null), Credentials.class);
-
-    }
-
-    @Override
-    public void disconnect() {
-        clusterState.unregister(stateId);
-        if (solo)
-            clusterState.close();
-    }
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
new file mode 100644
index 0000000..8ac0adc
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -0,0 +1,271 @@
+/**
+ * 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.APersistentMap;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.*;
+import org.apache.curator.framework.state.ConnectionState;
+import org.apache.storm.Config;
+import org.apache.storm.callback.DefaultWatcherCallBack;
+import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.utils.Utils;
+import org.apache.storm.zookeeper.Zookeeper;
+import org.apache.zookeeper.CreateMode;
+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.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+public class ZKStateStorage implements StateStorage {
+
+    private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
+
+    private ConcurrentHashMap<String, ZKStateChangedCallback> callbacks = new ConcurrentHashMap<String, ZKStateChangedCallback>();
+    private CuratorFramework zkWriter;
+    private CuratorFramework zkReader;
+    private AtomicBoolean active;
+
+    private boolean isNimbus;
+    private Map authConf;
+    private Map<Object, Object> conf;
+
+    public ZKStateStorage(Map<Object, Object> conf, Map authConf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        this.conf = conf;
+        this.authConf = authConf;
+        if (context.getDaemonType().equals(DaemonType.NIMBUS))
+            this.isNimbus = true;
+
+        // just mkdir STORM_ZOOKEEPER_ROOT dir
+        CuratorFramework zkTemp = mkZk();
+        String rootPath = String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT));
+        Zookeeper.mkdirs(zkTemp, rootPath, acls);
+        zkTemp.close();
+
+        active = new AtomicBoolean(true);
+        zkWriter = mkZk(new WatcherCallBack() {
+            @Override
+            public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+                if (active.get()) {
+                    if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                        LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                    } else {
+                        LOG.info("Received event {} : {} : {}", state, type, path);
+                    }
+
+                    if (!type.equals(Watcher.Event.EventType.None)) {
+                        for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+                            ZKStateChangedCallback fn = e.getValue();
+                            fn.changed(type, path);
+                        }
+                    }
+                }
+            }
+        });
+        if (isNimbus) {
+            zkReader = mkZk(new WatcherCallBack() {
+                @Override
+                public void execute(Watcher.Event.KeeperState state, Watcher.Event.EventType type, String path) {
+                    if (active.get()) {
+                        if (!(state.equals(Watcher.Event.KeeperState.SyncConnected))) {
+                            LOG.warn("Received event {} : {}: {} with disconnected Zookeeper.", state, type, path);
+                        } else {
+                            LOG.debug("Received event {} : {} : {}", state, type, path);
+                        }
+
+                        if (!type.equals(Watcher.Event.EventType.None)) {
+                            for (Map.Entry<String, ZKStateChangedCallback> e : callbacks.entrySet()) {
+                                ZKStateChangedCallback fn = e.getValue();
+                                fn.changed(type, path);
+                            }
+                        }
+                    }
+                }
+            });
+        } else {
+            zkReader = zkWriter;
+        }
+
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk() throws IOException {
+        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT), "",
+                new DefaultWatcherCallBack(), authConf);
+    }
+
+    @SuppressWarnings("unchecked")
+    private CuratorFramework mkZk(WatcherCallBack watcher) throws NumberFormatException, IOException {
+        return Zookeeper.mkClient(conf, (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS), conf.get(Config.STORM_ZOOKEEPER_PORT),
+                String.valueOf(conf.get(Config.STORM_ZOOKEEPER_ROOT)), watcher, authConf);
+    }
+
+    @Override
+    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+
+    }
+
+    @Override
+    public String register(ZKStateChangedCallback callback) {
+        String id = UUID.randomUUID().toString();
+        this.callbacks.put(id, callback);
+        return id;
+    }
+
+    @Override
+    public void unregister(String id) {
+        this.callbacks.remove(id);
+    }
+
+    @Override
+    public String create_sequential(String path, byte[] data, List<ACL> acls) {
+        return Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL_SEQUENTIAL, acls);
+    }
+
+    @Override
+    public void mkdirs(String path, List<ACL> acls) {
+        Zookeeper.mkdirs(zkWriter, path, acls);
+    }
+
+    @Override
+    public void delete_node(String path) {
+        Zookeeper.deleteNode(zkWriter, path);
+    }
+
+    @Override
+    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+        Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+        if (Zookeeper.exists(zkWriter, path, false)) {
+            try {
+                Zookeeper.setData(zkWriter, path, data);
+            } catch (Exception e) {
+                if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
+                    Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+                } else {
+                    throw Utils.wrapInRuntime(e);
+                }
+            }
+
+        } else {
+            Zookeeper.createNode(zkWriter, path, data, CreateMode.EPHEMERAL, acls);
+        }
+    }
+
+    @Override
+    public Integer get_version(String path, boolean watch) throws Exception {
+        Integer ret = Zookeeper.getVersion(zkReader, path, watch);
+        return ret;
+    }
+
+    @Override
+    public boolean node_exists(String path, boolean watch) {
+        return Zookeeper.existsNode(zkWriter, path, watch);
+    }
+
+    @Override
+    public List<String> get_children(String path, boolean watch) {
+        return Zookeeper.getChildren(zkReader, path, watch);
+    }
+
+    @Override
+    public void close() {
+        this.active.set(false);
+        zkWriter.close();
+        if (isNimbus) {
+            zkReader.close();
+        }
+    }
+
+    @Override
+    public void set_data(String path, byte[] data, List<ACL> acls) {
+        if (Zookeeper.exists(zkWriter, path, false)) {
+            Zookeeper.setData(zkWriter, path, data);
+        } else {
+            Zookeeper.mkdirs(zkWriter, parentPath(path), acls);
+            Zookeeper.createNode(zkWriter, path, data, CreateMode.PERSISTENT, acls);
+        }
+    }
+
+    @Override
+    public byte[] get_data(String path, boolean watch) {
+        byte[] ret = null;
+
+        ret = Zookeeper.getData(zkReader, path, watch);
+
+        return ret;
+    }
+
+    @Override
+    public APersistentMap get_data_with_version(String path, boolean watch) {
+        return Zookeeper.getDataWithVersion(zkReader, path, watch);
+    }
+
+    @Override
+    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+        set_data(path, data, acls);
+    }
+
+    @Override
+    public byte[] get_worker_hb(String path, boolean watch) {
+        return Zookeeper.getData(zkReader, path, watch);
+    }
+
+    @Override
+    public List<String> get_worker_hb_children(String path, boolean watch) {
+        return get_children(path, watch);
+    }
+
+    @Override
+    public void delete_worker_hb(String path) {
+        delete_node(path);
+    }
+
+    @Override
+    public void add_listener(final ConnectionStateListener listener) {
+        Zookeeper.addListener(zkReader, new ConnectionStateListener() {
+            @Override
+            public void stateChanged(CuratorFramework curatorFramework, ConnectionState connectionState) {
+                listener.stateChanged(curatorFramework, connectionState);
+            }
+        });
+    }
+
+    @Override
+    public void sync_path(String path) {
+        Zookeeper.syncPath(zkWriter, path);
+    }
+
+    // To be remove when finished port Util.clj
+    public static String parentPath(String path) {
+        List<String> toks = Zookeeper.tokenizePath(path);
+        int size = toks.size();
+        if (size > 0) {
+            toks.remove(size - 1);
+        }
+        return Zookeeper.toksToPath(toks);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
new file mode 100644
index 0000000..19b04f2
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -0,0 +1,36 @@
+/**
+ * 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.APersistentMap;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
+public class ZKStateStorageFactory implements StateStorageFactory{
+
+    @Override
+    public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+        try {
+            return new ZKStateStorage(config, auth_conf, acls, context);
+        }catch (Exception e){
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
index 5d67a54..2f1440c 100644
--- a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedCluster.java
@@ -16,16 +16,16 @@
  */
 package org.apache.storm.testing.staticmocking;
 
-import org.apache.storm.cluster.Cluster;
+import org.apache.storm.cluster.ClusterUtils;
 
 public class MockedCluster implements AutoCloseable  {
 
-    public MockedCluster(Cluster inst) {
-        Cluster.setInstance(inst);
+    public MockedCluster(ClusterUtils inst) {
+        ClusterUtils.setInstance(inst);
     }
 
     @Override
     public void close() throws Exception {
-        Cluster.resetInstance();
+        ClusterUtils.resetInstance();
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index f1c7f32..c280515 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -86,19 +86,23 @@ public class Zookeeper {
         _instance = INSTANCE;
     }
 
-    public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root) {
-        return mkClient(conf, servers, port, root, new DefaultWatcherCallBack());
+    public  CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root) {
+        return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack());
     }
 
-    public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, Map authConf) {
-        return mkClient(conf, servers, port, "", new DefaultWatcherCallBack(), authConf);
+    public  CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, Map authConf) {
+        return mkClientImpl(conf, servers, port, "", new DefaultWatcherCallBack(), authConf);
     }
 
-    public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, Map authConf) {
-        return mkClient(conf, servers, port, root, new DefaultWatcherCallBack(), authConf);
+    public  CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, Map authConf) {
+        return mkClientImpl(conf, servers, port, root, new DefaultWatcherCallBack(), authConf);
     }
 
     public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher, Map authConf) {
+        return _instance.mkClientImpl(conf, servers, port, root, watcher, authConf);
+    }
+
+    public  CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher, Map authConf) {
         CuratorFramework fk;
         if (authConf != null) {
             fk = Utils.newCurator(conf, servers, port, root, new ZookeeperAuthInfo(authConf));
@@ -124,8 +128,8 @@ public class Zookeeper {
      *
      * @return
      */
-    public static CuratorFramework mkClient(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher) {
-        return mkClient(conf, servers, port, root, watcher, null);
+    public  CuratorFramework mkClientImpl(Map conf, List<String> servers, Object port, String root, final WatcherCallBack watcher) {
+        return mkClientImpl(conf, servers, port, root, watcher, null);
     }
 
     public static String createNode(CuratorFramework zk, String path, byte[] data, org.apache.zookeeper.CreateMode mode, List<ACL> acls) {
@@ -347,7 +351,7 @@ public class Zookeeper {
     protected ILeaderElector zkLeaderElectorImpl(Map conf) throws UnknownHostException {
         List<String> servers = (List<String>) conf.get(Config.STORM_ZOOKEEPER_SERVERS);
         Object port = conf.get(Config.STORM_ZOOKEEPER_PORT);
-        CuratorFramework zk = mkClient(conf, servers, port, "", conf);
+        CuratorFramework zk = mkClientImpl(conf, servers, port, "", conf);
         String leaderLockPath = conf.get(Config.STORM_ZOOKEEPER_ROOT) + "/leader-lock";
         String id = NimbusInfo.fromConf(conf).toHostPortString();
         AtomicReference<LeaderLatch> leaderLatchAtomicReference = new AtomicReference<>(new LeaderLatch(zk, leaderLockPath, id));

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
index d374511..d4fab3f 100644
--- a/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
+++ b/storm-core/test/clj/integration/org/apache/storm/integration_test.clj
@@ -21,7 +21,7 @@
   (:import [org.apache.storm.testing TestWordCounter TestWordSpout TestGlobalCount
             TestAggregatesCounter TestConfBolt AckFailMapTracker AckTracker TestPlannerSpout])
   (:import [org.apache.storm.tuple Fields])
-  (:import [org.apache.storm.cluster StormZkClusterState])
+  (:import [org.apache.storm.cluster StormClusterStateImpl])
   (:use [org.apache.storm testing config clojure util converter])
   (:use [org.apache.storm.daemon common])
   (:require [org.apache.storm [thrift :as thrift]]))
@@ -576,7 +576,7 @@
                                              (:topology tracked))
             _ (advance-cluster-time cluster 11)
             storm-id (get-storm-id state "test-errors")
-            errors-count (fn [] (count (clojurify-error (.errors state storm-id "2"))))]
+            errors-count (fn [] (count (.errors state storm-id "2")))]
 
         (is (nil? (clojurify-error (.lastError state storm-id "2"))))
 

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index d0b9882..fa34355 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,9 +23,10 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
   (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
-  (:import [org.apache.storm.cluster ClusterState DistributedClusterState ClusterStateContext StormZkClusterState])
+  (:import [org.apache.storm.cluster StateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
-  (:import [org.apache.storm.testing.staticmocking MockedZookeeper])
+  (:import [org.apache.storm.callback ZKStateChangedCallback])
+  (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
   (:require [conjure.core])
   (:use [conjure core])
   (:use [clojure test])
@@ -33,18 +34,18 @@
 
 (defn mk-config [zk-port]
   (merge (clojurify-structure (ConfigUtils/readStormConfig))
-         {STORM-ZOOKEEPER-PORT zk-port
-          STORM-ZOOKEEPER-SERVERS ["localhost"]}))
+    {STORM-ZOOKEEPER-PORT zk-port
+     STORM-ZOOKEEPER-SERVERS ["localhost"]}))
 
 (defn mk-state
   ([zk-port] (let [conf (mk-config zk-port)]
-               (DistributedClusterState. conf conf nil (ClusterStateContext.))))
+               (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))))
   ([zk-port cb]
-     (let [ret (mk-state zk-port)]
-       (.register ret cb)
-       ret )))
+    (let [ret (mk-state zk-port)]
+      (.register ret cb)
+      ret)))
 
-(defn mk-storm-state [zk-port] (StormZkClusterState. (mk-config zk-port) nil (ClusterStateContext.)))
+(defn mk-storm-state [zk-port] (ClusterUtils/mkStormClusterState (mk-config zk-port) nil (ClusterStateContext.)))
 
 (deftest test-basics
   (with-inprocess-zookeeper zk-port
@@ -99,24 +100,27 @@
 
 (defn mk-callback-tester []
   (let [last (atom nil)
-        cb (fn [type path]
-              (reset! last {:type type :path path}))]
+        cb (reify
+             ZKStateChangedCallback
+             (changed
+               [this type path]
+               (reset! last {:type type :path path})))]
     [last cb]
     ))
 
 (defn read-and-reset! [aatom]
   (let [time (System/currentTimeMillis)]
-  (loop []
-    (if-let [val @aatom]
-      (do
-        (reset! aatom nil)
-        val)
-      (do
-        (when (> (- (System/currentTimeMillis) time) 30000)
-          (throw (RuntimeException. "Waited too long for atom to change state")))
-        (Thread/sleep 10)
-        (recur))
-      ))))
+    (loop []
+      (if-let [val @aatom]
+        (do
+          (reset! aatom nil)
+          val)
+        (do
+          (when (> (- (System/currentTimeMillis) time) 30000)
+            (throw (RuntimeException. "Waited too long for atom to change state")))
+          (Thread/sleep 10)
+          (recur))
+        ))))
 
 (deftest test-callbacks
   (with-inprocess-zookeeper zk-port
@@ -189,35 +193,35 @@
       (is (= #{"storm1" "storm3"} (set (.assignments state nil))))
       (is (= assignment2 (clojurify-assignment (.assignmentInfo state "storm1" nil))))
       (is (= assignment1 (clojurify-assignment (.assignmentInfo state "storm3" nil))))
-      
-      (is (= [] (.active-storms state)))
+
+      (is (= [] (.activeStorms state)))
       (.activateStorm state "storm1" (thriftify-storm-base base1))
-      (is (= ["storm1"] (.active-storms state)))
+      (is (= ["storm1"] (.activeStorms state)))
       (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
       (is (= nil (clojurify-storm-base (.stormBase state "storm2" nil))))
       (.activateStorm state "storm2" (thriftify-storm-base base2))
       (is (= base1 (clojurify-storm-base (.stormBase state "storm1" nil))))
       (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
-      (is (= #{"storm1" "storm2"} (set (.active-storms state))))
+      (is (= #{"storm1" "storm2"} (set (.activeStorms state))))
       (.removeStormBase state "storm1")
       (is (= base2 (clojurify-storm-base (.stormBase state "storm2" nil))))
-      (is (= #{"storm2"} (set (.active-storms state))))
+      (is (= #{"storm2"} (set (.activeStorms state))))
 
       (is (nil? (clojurify-crdentials (.credentials state "storm1" nil))))
-      (.setCredentials! state "storm1" (thriftify-credentials {"a" "a"}) {})
+      (.setCredentials state "storm1" (thriftify-credentials {"a" "a"}) {})
       (is (= {"a" "a"} (clojurify-crdentials (.credentials state "storm1" nil))))
       (.setCredentials state "storm1" (thriftify-credentials {"b" "b"}) {})
       (is (= {"b" "b"} (clojurify-crdentials (.credentials state "storm1" nil))))
 
-      (is (= [] (.blobstoreInfo state nil)))
-      (.setupBlobstore state "key1" nimbusInfo1 "1")
-      (is (= ["key1"] (.blobstoreInfo state nil)))
+      (is (= [] (.blobstoreInfo state "")))
+      (.setupBlobstore state "key1" nimbusInfo1 (Integer/parseInt "1"))
+      (is (= ["key1"] (.blobstoreInfo state "")))
       (is (= [(str (.toHostPortString nimbusInfo1) "-1")] (.blobstoreInfo state "key1")))
-      (.setupBlobstore state "key1" nimbusInfo2 "1")
+      (.setupBlobstore state "key1" nimbusInfo2 (Integer/parseInt "1"))
       (is (= #{(str (.toHostPortString nimbusInfo1) "-1")
                (str (.toHostPortString nimbusInfo2) "-1")} (set (.blobstoreInfo state "key1"))))
       (.removeBlobstoreKey state "key1")
-      (is (= [] (.blobstoreInfo state nil)))
+      (is (= [] (.blobstoreInfo state "")))
 
       (is (= [] (.nimbuses state)))
       (.addNimbusHost state "nimbus1:port" nimbusSummary1)
@@ -230,11 +234,10 @@
       )))
 
 (defn- validate-errors! [state storm-id component errors-list]
-  (let [errors (clojurify-error (.errors state storm-id component))]
-    ;;(println errors)
+  (let [errors (map clojurify-error (.errors state storm-id component))]
     (is (= (count errors) (count errors-list)))
     (doseq [[error target] (map vector errors errors-list)]
-      (when-not  (.contains (:error error) target)
+      (when-not (.contains (:error error) target)
         (println target " => " (:error error)))
       (is (.contains (:error error) target))
       )))
@@ -257,8 +260,9 @@
           (.reportError state "a" "2" (local-hostname) 6700 (stringify-error (IllegalArgumentException.)))
           (advance-time-secs! 2))
         (validate-errors! state "a" "2" (concat (repeat 5 "IllegalArgumentException")
-                                                (repeat 5 "RuntimeException")
-                                                ))
+                                          (repeat 5 "RuntimeException")
+                                          ))
+
         (.disconnect state)
         ))))
 
@@ -285,23 +289,23 @@
   (with-inprocess-zookeeper zk-port
     (let [builder (Mockito/mock CuratorFrameworkFactory$Builder)
           conf (merge
-                (mk-config zk-port)
-                {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10
-                 STORM-ZOOKEEPER-SESSION-TIMEOUT 10
-                 STORM-ZOOKEEPER-RETRY-INTERVAL 5
-                 STORM-ZOOKEEPER-RETRY-TIMES 2
-                 STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15
-                 STORM-ZOOKEEPER-AUTH-SCHEME "digest"
-                 STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})]
+                 (mk-config zk-port)
+                 {STORM-ZOOKEEPER-CONNECTION-TIMEOUT 10
+                  STORM-ZOOKEEPER-SESSION-TIMEOUT 10
+                  STORM-ZOOKEEPER-RETRY-INTERVAL 5
+                  STORM-ZOOKEEPER-RETRY-TIMES 2
+                  STORM-ZOOKEEPER-RETRY-INTERVAL-CEILING 15
+                  STORM-ZOOKEEPER-AUTH-SCHEME "digest"
+                  STORM-ZOOKEEPER-AUTH-PAYLOAD "storm:thisisapoorpassword"})]
       (. (Mockito/when (.connectString builder (Mockito/anyString))) (thenReturn builder))
       (. (Mockito/when (.connectionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
       (. (Mockito/when (.sessionTimeoutMs builder (Mockito/anyInt))) (thenReturn builder))
       (TestUtils/testSetupBuilder builder (str zk-port "/") conf (ZookeeperAuthInfo. conf))
       (is (nil?
-           (try
-             (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
-             (catch MockitoAssertionError e
-               e)))))))
+            (try
+              (. (Mockito/verify builder) (authorization "digest" (.getBytes (conf STORM-ZOOKEEPER-AUTH-PAYLOAD))))
+              (catch MockitoAssertionError e
+                e)))))))
 
 (deftest test-storm-state-callbacks
   ;; TODO finish
@@ -309,13 +313,17 @@
 
 (deftest test-cluster-state-default-acls
   (testing "The default ACLs are empty."
-    (let [zk-mock (Mockito/mock Zookeeper)]
+    (let [zk-mock (Mockito/mock Zookeeper)
+          curator-frameworke (reify CuratorFramework (^void close [this] nil))]
       ;; No need for when clauses because we just want to return nil
       (with-open [_ (MockedZookeeper. zk-mock)]
-          (. (Mockito/when (Mockito/mock Zookeeper)) (thenReturn (reify CuratorFramework (^void close [this] nil))))
-          (. (Mockito/when (Mockito/mock DistributedClusterState)) (thenReturn {}))
-          (. (Mockito/when (Mockito/mock StormZkClusterState)) (thenReturn (reify ClusterState
-                                                                             (register [this callback] nil)
-                                                                             (mkdirs [this path acls] nil))))
-          (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))))
-
+        (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
+        (ClusterUtils/mkDistributedClusterState {} nil nil (ClusterStateContext.))
+        (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
+    (let [distributed-state-storage (reify StateStorage
+                                      (register [this callback] nil)
+                                      (mkdirs [this path acls] nil))
+          cluster-utils (Mockito/mock ClusterUtils)]
+      (with-open [mocked-cluster (MockedCluster. cluster-utils)]
+        (. (Mockito/when (.mkDistributedClusterStateImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+        (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
\ No newline at end of file


[26/27] storm git commit: Merge branch 'ClusterUtils' of https://github.com/hustfxj/storm into STORM-1273

Posted by bo...@apache.org.
Merge branch 'ClusterUtils' of https://github.com/hustfxj/storm into STORM-1273

STORM-1273: port backtype.storm.cluster to java


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/1b5d9a85
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/1b5d9a85
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/1b5d9a85

Branch: refs/heads/master
Commit: 1b5d9a85021ff4c7bb8e95a55a225277babb2378
Parents: 11232b5 0af6799
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 24 09:45:14 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 24 09:45:14 2016 -0600

----------------------------------------------------------------------
 conf/defaults.yaml                              |   2 +-
 .../src/clj/org/apache/storm/thrift.clj         |   2 +-
 storm-core/src/clj/org/apache/storm/cluster.clj | 700 -------------------
 .../cluster_state/zookeeper_state_factory.clj   | 165 -----
 .../clj/org/apache/storm/command/heartbeats.clj |   6 +-
 .../apache/storm/command/shell_submission.clj   |   2 +-
 .../src/clj/org/apache/storm/converter.clj      |  23 +-
 .../src/clj/org/apache/storm/daemon/common.clj  |  12 +-
 .../clj/org/apache/storm/daemon/executor.clj    |  13 +-
 .../src/clj/org/apache/storm/daemon/nimbus.clj  | 161 +++--
 .../clj/org/apache/storm/daemon/supervisor.clj  |  38 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  59 +-
 .../clj/org/apache/storm/internal/thrift.clj    |   2 +-
 .../storm/pacemaker/pacemaker_state_factory.clj | 141 ----
 storm-core/src/clj/org/apache/storm/stats.clj   |   3 +-
 storm-core/src/clj/org/apache/storm/testing.clj |  12 +-
 storm-core/src/clj/org/apache/storm/ui/core.clj |   2 +-
 storm-core/src/clj/org/apache/storm/util.clj    |  11 +
 .../src/clj/org/apache/storm/zookeeper.clj      |  74 --
 .../jvm/org/apache/storm/callback/Callback.java |  23 -
 .../storm/callback/ZKStateChangedCallback.java  |  25 +
 .../org/apache/storm/cluster/ClusterState.java  | 217 ------
 .../storm/cluster/ClusterStateContext.java      |   2 +-
 .../storm/cluster/ClusterStateFactory.java      |  28 -
 .../org/apache/storm/cluster/ClusterUtils.java  | 244 +++++++
 .../org/apache/storm/cluster/ExecutorBeat.java  |  44 ++
 .../org/apache/storm/cluster/IStateStorage.java | 222 ++++++
 .../storm/cluster/IStormClusterState.java       | 122 ++++
 .../storm/cluster/PaceMakerStateStorage.java    | 216 ++++++
 .../cluster/PaceMakerStateStorageFactory.java   |  64 ++
 .../storm/cluster/StateStorageFactory.java      |  28 +
 .../storm/cluster/StormClusterStateImpl.java    | 686 ++++++++++++++++++
 .../apache/storm/cluster/ZKStateStorage.java    | 244 +++++++
 .../storm/cluster/ZKStateStorageFactory.java    |  36 +
 .../apache/storm/pacemaker/PacemakerClient.java |   6 +-
 .../testing/staticmocking/MockedCluster.java    |  31 +
 .../MockedPaceMakerStateStorageFactory.java     |  32 +
 .../org/apache/storm/zookeeper/Zookeeper.java   |  77 +-
 .../org/apache/storm/integration_test.clj       |  13 +-
 .../test/clj/org/apache/storm/cluster_test.clj  | 202 +++---
 .../storm/messaging/netty_integration_test.clj  |   1 +
 .../test/clj/org/apache/storm/nimbus_test.clj   | 158 +++--
 .../storm/pacemaker_state_factory_test.clj      | 121 ++--
 .../storm/security/auth/nimbus_auth_test.clj    |   3 +-
 .../clj/org/apache/storm/supervisor_test.clj    | 163 +++--
 45 files changed, 2554 insertions(+), 1882 deletions(-)
----------------------------------------------------------------------



[18/27] storm git commit: removed any clojure internals

Posted by bo...@apache.org.
removed any clojure internals


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/afcd0c6c
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/afcd0c6c
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/afcd0c6c

Branch: refs/heads/master
Commit: afcd0c6c53aca7f99d39f10b91a7b45fda424fe5
Parents: d546387
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Fri Feb 19 15:21:48 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Fri Feb 19 15:22:22 2016 +0800

----------------------------------------------------------------------
 .../src/clj/org/apache/storm/converter.clj      |  9 +++
 .../src/clj/org/apache/storm/daemon/nimbus.clj  |  4 +-
 .../clj/org/apache/storm/daemon/supervisor.clj  |  8 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |  6 +-
 storm-core/src/clj/org/apache/storm/testing.clj |  2 +-
 .../org/apache/storm/cluster/ClusterUtils.java  | 41 +++-------
 .../org/apache/storm/cluster/ExecutorBeat.java  | 44 +++++++++++
 .../org/apache/storm/cluster/IStateStorage.java | 11 ++-
 .../storm/cluster/IStormClusterState.java       | 24 +++---
 .../storm/cluster/PaceMakerStateStorage.java    |  4 +-
 .../cluster/PaceMakerStateStorageFactory.java   | 12 +--
 .../storm/cluster/StateStorageFactory.java      |  6 +-
 .../storm/cluster/StormClusterStateImpl.java    | 78 ++++++++++----------
 .../apache/storm/cluster/ZKStateStorage.java    |  3 +-
 .../storm/cluster/ZKStateStorageFactory.java    |  4 +-
 .../org/apache/storm/zookeeper/Zookeeper.java   | 46 +++---------
 16 files changed, 156 insertions(+), 146 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index 18647b1..c845cd4 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -18,6 +18,7 @@
             StormBase TopologyStatus ClusterWorkerHeartbeat ExecutorInfo ErrorInfo Credentials RebalanceOptions KillOptions
             TopologyActionOptions DebugOptions ProfileRequest]
            [org.apache.storm.utils Utils])
+  (:import [org.apache.storm.cluster ExecutorBeat])
   (:use [org.apache.storm util stats log])
   (:require [org.apache.storm.daemon [common :as common]]))
 
@@ -238,6 +239,14 @@
      }
     {}))
 
+(defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
+  (if executor-hb
+    {:stats (.getStats executor-hb)
+     :uptime (.getUptime executor-hb)
+     :time-secs (.getTimeSecs executor-hb)
+     }
+    {}))
+
 (defn thriftify-zk-worker-hb [worker-hb]
   (if (not-empty (filter second (:executor-stats worker-hb)))
     (doto (ClusterWorkerHeartbeat.)

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index 6bdbdc0..beb6639 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -591,8 +591,8 @@
   (let [storm-cluster-state (:storm-cluster-state nimbus)
         executor-beats (let [executor-stats-java-map (.executorBeats storm-cluster-state storm-id (.get_executor_node_port (thriftify-assignment existing-assignment)))
                              executor-stats-clojurify (clojurify-structure executor-stats-java-map)]
-                         (->> (dofor [[^ExecutorInfo executor-info  executor-heartbeat] executor-stats-clojurify]
-                             {[(.get_task_start executor-info) (.get_task_end executor-info)] executor-heartbeat})
+                         (->> (dofor [[^ExecutorInfo executor-info  ^ExecutorBeat executor-heartbeat] executor-stats-clojurify]
+                             {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-executor-hb executor-heartbeat)})
                            (apply merge)))
 
         cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
index c1f058f..58f6291 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/supervisor.clj
@@ -20,7 +20,7 @@
                                    ConfigUtils]
            [org.apache.storm.daemon Shutdownable]
            [org.apache.storm Constants]
-           [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils]
+           [org.apache.storm.cluster ClusterStateContext DaemonType StormClusterStateImpl ClusterUtils IStateStorage]
            [java.net JarURLConnection]
            [java.net URI URLDecoder]
            [org.apache.commons.io FileUtils])
@@ -69,8 +69,8 @@
                       (if (= assignment-version recorded-version)
                         {sid (get assignment-versions sid)}
                         (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state sid callback)
-                              assignment (clojurify-assignment (:data thriftify-assignment-version))]
-                        {sid {:data assignment :version (:version thriftify-assignment-version)}}))
+                              assignment (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA)))]
+                        {sid {:data assignment :version (.get thriftify-assignment-version (IStateStorage/VERSION))}}))
                       {sid nil})))
            (apply merge)
            (filter-val not-nil?))
@@ -1184,7 +1184,7 @@
       (.readBlobTo blob-store (ConfigUtils/masterStormConfKey storm-id) (FileOutputStream. (ConfigUtils/supervisorStormConfPath tmproot)) nil)
       (finally
         (.shutdown blob-store)))
-    (try (FileUtils/moveDirectory (File. tmproot) (File. stormroot)) (catch Exception e))
+    (FileUtils/moveDirectory (File. tmproot) (File. stormroot))
 
     (setup-storm-code-dir conf (clojurify-structure (ConfigUtils/readSupervisorStormConf conf storm-id)) stormroot)
     (let [classloader (.getContextClassLoader (Thread/currentThread))

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index b80cd9e..395be23 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -38,7 +38,7 @@
   (:import [org.apache.storm.task WorkerTopologyContext])
   (:import [org.apache.storm Constants])
   (:import [org.apache.storm.security.auth AuthUtils])
-  (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils])
+  (:import [org.apache.storm.cluster ClusterStateContext DaemonType ZKStateStorage StormClusterStateImpl ClusterUtils IStateStorage])
   (:import [javax.security.auth Subject])
   (:import [java.security PrivilegedExceptionAction])
   (:import [org.apache.logging.log4j LogManager])
@@ -381,8 +381,8 @@
          (let [version (.assignmentVersion storm-cluster-state storm-id callback)
                assignment (if (= version (:version (get @(:assignment-versions worker) storm-id)))
                             (:data (get @(:assignment-versions worker) storm-id))
-                            (let [java-assignment (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
-                              new-assignment {:data (clojurify-assignment (:data java-assignment)) :version version}]
+                            (let [thriftify-assignment-version (.assignmentInfoWithVersion storm-cluster-state storm-id callback)
+                              new-assignment {:data (clojurify-assignment (.get thriftify-assignment-version (IStateStorage/DATA))) :version version}]
                               (swap! (:assignment-versions worker) assoc storm-id new-assignment)
                               (:data new-assignment)))
               my-assignment (-> assignment

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index eef7754..3dee54b 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -448,7 +448,7 @@
                            component->tasks)
         task-ids (apply concat (vals component->tasks))
         assignment (clojurify-assignment (.assignmentInfo state storm-id nil))
-        taskbeats (.taskbeats state storm-id (:task->node+port assignment))  ;hava question?
+        taskbeats (.taskbeats state storm-id (:task->node+port assignment))
         heartbeats (dofor [id task-ids] (get taskbeats id))
         stats (dofor [hb heartbeats] (if hb (stat-key (:stats hb)) 0))]
     (reduce + stats)))

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 0c663f0..aae4231 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -17,9 +17,6 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
-import clojure.lang.PersistentArrayMap;
-import clojure.lang.RT;
 import org.apache.storm.Config;
 import org.apache.storm.generated.ClusterWorkerHeartbeat;
 import org.apache.storm.generated.ExecutorInfo;
@@ -192,14 +189,15 @@ public class ClusterUtils {
      * @param workerHeartbeat
      * @return
      */
-    public static Map<ExecutorInfo, APersistentMap> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
-        Map<ExecutorInfo, APersistentMap> executorWhb = new HashMap<>();
+    public static Map<ExecutorInfo, ExecutorBeat> convertExecutorBeats(List<ExecutorInfo> executors, ClusterWorkerHeartbeat workerHeartbeat) {
+        Map<ExecutorInfo, ExecutorBeat> executorWhb = new HashMap<>();
         Map<ExecutorInfo, ExecutorStats> executorStatsMap = workerHeartbeat.get_executor_stats();
         for (ExecutorInfo executor : executors) {
             if (executorStatsMap.containsKey(executor)) {
-                APersistentMap executorBeat =
-                        new PersistentArrayMap(new Object[] { RT.keyword(null, "time-secs"), workerHeartbeat.get_time_secs(), RT.keyword(null, "uptime"),
-                                workerHeartbeat.get_uptime_secs(), RT.keyword(null, "stats"), workerHeartbeat.get_executor_stats().get(executor) });
+                int time = workerHeartbeat.get_time_secs();
+                int uptime = workerHeartbeat.get_uptime_secs();
+                ExecutorStats executorStats = workerHeartbeat.get_executor_stats().get(executor);
+                ExecutorBeat executorBeat = new ExecutorBeat(time, uptime, executorStats);
                 executorWhb.put(executor, executorBeat);
             }
         }
@@ -210,13 +208,13 @@ public class ClusterUtils {
         if (stateStorage instanceof IStateStorage) {
             return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
         } else {
-            IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
+            IStateStorage Storage = _instance.mkStateStorageImpl((Map) stateStorage, (Map) stateStorage, acls, context);
             return new StormClusterStateImpl(Storage, acls, context, true);
         }
 
     }
 
-    public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+    public IStateStorage mkStateStorageImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
         String className = null;
         IStateStorage stateStorage = null;
         if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
@@ -230,7 +228,7 @@ public class ClusterUtils {
         return stateStorage;
     }
 
-    public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+    public static IStateStorage mkStateStorage(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
         return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
     }
 
@@ -238,26 +236,7 @@ public class ClusterUtils {
         return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
     }
 
-    // TO be remove
-    public static <K, V> HashMap<V, List<K>> reverseMap(Map<K, V> map) {
-        HashMap<V, List<K>> rtn = new HashMap<V, List<K>>();
-        if (map == null) {
-            return rtn;
-        }
-        for (Map.Entry<K, V> entry : map.entrySet()) {
-            K key = entry.getKey();
-            V val = entry.getValue();
-            List<K> list = rtn.get(val);
-            if (list == null) {
-                list = new ArrayList<K>();
-                rtn.put(entry.getValue(), list);
-            }
-            list.add(key);
-        }
-        return rtn;
-    }
-
-    public static String StringifyError(Throwable error) {
+    public static String stringifyError(Throwable error) {
         String errorString = null;
         StringWriter result = null;
         PrintWriter printWriter = null;

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java b/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
new file mode 100644
index 0000000..b32615e
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ExecutorBeat.java
@@ -0,0 +1,44 @@
+/**
+ * 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 org.apache.storm.generated.ExecutorStats;
+
+public class ExecutorBeat {
+    private final int timeSecs;
+    private final int uptime;
+    private final ExecutorStats stats;
+
+    public ExecutorBeat(int timeSecs, int uptime, ExecutorStats stats) {
+        this.timeSecs = timeSecs;
+        this.uptime = uptime;
+        this.stats = stats;
+    }
+
+    public int getTimeSecs() {
+        return timeSecs;
+    }
+
+    public int getUptime() {
+        return uptime;
+    }
+
+    public ExecutorStats getStats() {
+        return stats;
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
index 1a2b14f..0b6f043 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -17,8 +17,8 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.storm.callback.ZKStateChangedCallback;
@@ -42,6 +42,9 @@ import org.apache.zookeeper.data.ACL;
  */
 public interface IStateStorage {
 
+    public static final String DATA = "data";
+    public static final String VERSION = "version";
+
     /**
      * Registers a callback function that gets called when CuratorEvents happen.
      * @param callback is a clojure IFn that accepts the type - translated to
@@ -149,14 +152,14 @@ public interface IStateStorage {
 
     /**
      * Get the data at the node along with its version. Data is returned
-     * in an APersistentMap with clojure keyword keys :data and :version.
+     * in an Map with the keys data and version.
      * @param path The path to look under
      * @param watch Whether or not to set a watch on the path. Watched paths
      * emit events which are consumed by functions registered with the
      * register method. Very useful for catching updates to nodes.
-     * @return An APersistentMap in the form {:data data :version version}
+     * @return An Map in the form {:data data :version version}
      */
-    APersistentMap get_data_with_version(String path, boolean watch);
+    Map get_data_with_version(String path, boolean watch);
 
     /**
      * Write a worker heartbeat at the path.

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
index 01cf56a..c88935e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -17,8 +17,6 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
-import clojure.lang.IFn;
 import org.apache.storm.generated.*;
 import org.apache.storm.nimbus.NimbusInfo;
 
@@ -27,13 +25,13 @@ import java.util.List;
 import java.util.Map;
 
 public interface IStormClusterState {
-    public List<String> assignments(IFn callback);
+    public List<String> assignments(Runnable callback);
 
-    public Assignment assignmentInfo(String stormId, IFn callback);
+    public Assignment assignmentInfo(String stormId, Runnable callback);
 
-    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
+    public Map assignmentInfoWithVersion(String stormId, Runnable callback);
 
-    public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
+    public Integer assignmentVersion(String stormId, Runnable callback) throws Exception;
 
     public List<String> blobstoreInfo(String blobKey);
 
@@ -43,7 +41,7 @@ public interface IStormClusterState {
 
     public List<String> activeStorms();
 
-    public StormBase stormBase(String stormId, IFn callback);
+    public StormBase stormBase(String stormId, Runnable callback);
 
     public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
 
@@ -55,9 +53,9 @@ public interface IStormClusterState {
 
     public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
 
-    public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+    public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
 
-    public List<String> supervisors(IFn callback);
+    public List<String> supervisors(Runnable callback);
 
     public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
 
@@ -73,7 +71,7 @@ public interface IStormClusterState {
 
     public void setTopologyLogConfig(String stormId, LogConfig logConfig);
 
-    public LogConfig topologyLogConfig(String stormId, IFn cb);
+    public LogConfig topologyLogConfig(String stormId, Runnable cb);
 
     public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
 
@@ -83,7 +81,7 @@ public interface IStormClusterState {
 
     public void workerBackpressure(String stormId, String node, Long port, boolean on);
 
-    public boolean topologyBackpressure(String stormId, IFn callback);
+    public boolean topologyBackpressure(String stormId, Runnable callback);
 
     public void setupBackpressure(String stormId);
 
@@ -101,7 +99,7 @@ public interface IStormClusterState {
 
     public List<String> activeKeys();
 
-    public List<String> blobstore(IFn callback);
+    public List<String> blobstore(Runnable callback);
 
     public void removeStorm(String stormId);
 
@@ -117,7 +115,7 @@ public interface IStormClusterState {
 
     public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
 
-    public Credentials credentials(String stormId, IFn callback);
+    public Credentials credentials(String stormId, Runnable callback);
 
     public void disconnect();
 

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
index a9c4d89..c29078e 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -17,7 +17,6 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.storm.callback.ZKStateChangedCallback;
 import org.apache.storm.generated.*;
@@ -28,6 +27,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.util.List;
+import java.util.Map;
 
 public class PaceMakerStateStorage implements IStateStorage {
 
@@ -104,7 +104,7 @@ public class PaceMakerStateStorage implements IStateStorage {
     }
 
     @Override
-    public APersistentMap get_data_with_version(String path, boolean watch) {
+    public Map get_data_with_version(String path, boolean watch) {
         return stateStorage.get_data_with_version(path, watch);
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
index eafd2e7..3111e04 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -17,12 +17,12 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import org.apache.storm.pacemaker.PacemakerClient;
 import org.apache.storm.utils.Utils;
 import org.apache.zookeeper.data.ACL;
 
 import java.util.List;
+import java.util.Map;
 
 public class PaceMakerStateStorageFactory implements StateStorageFactory {
 
@@ -38,7 +38,7 @@ public class PaceMakerStateStorageFactory implements StateStorageFactory {
     }
 
     @Override
-    public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+    public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
         try {
             return new PaceMakerStateStorage(initMakeClient(config), initZKstate(config, auth_conf, acls, context));
         } catch (Exception e) {
@@ -46,19 +46,19 @@ public class PaceMakerStateStorageFactory implements StateStorageFactory {
         }
     }
 
-    public static IStateStorage initZKstate(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+    public static IStateStorage initZKstate(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
         return _instance.initZKstateImpl(config, auth_conf, acls, context);
     }
 
-    public static PacemakerClient initMakeClient(APersistentMap config) {
+    public static PacemakerClient initMakeClient(Map config) {
         return _instance.initMakeClientImpl(config);
     }
 
-    public IStateStorage initZKstateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+    public IStateStorage initZKstateImpl(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
         return ClusterUtils.mkStateStorage(config, auth_conf, acls, context);
     }
 
-    public PacemakerClient initMakeClientImpl(APersistentMap config) {
+    public PacemakerClient initMakeClientImpl(Map config) {
         return new PacemakerClient(config);
     }
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index 110da41..0929750 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -17,12 +17,12 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import java.util.List;
+import java.util.Map;
+
 import org.apache.zookeeper.data.ACL;
 
 public interface StateStorageFactory {
 
-    IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
-
+    IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context);
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index 17c8641..5fa586a 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -34,8 +34,6 @@ import org.apache.zookeeper.data.ACL;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.PrintWriter;
-import java.io.StringWriter;
 import java.security.NoSuchAlgorithmException;
 import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
@@ -47,17 +45,17 @@ public class StormClusterStateImpl implements IStormClusterState {
 
     private IStateStorage stateStorage;
 
-    private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
-    private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
-    private ConcurrentHashMap<String, IFn> assignmentVersionCallback;
-    private AtomicReference<IFn> supervisorsCallback;
+    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, IFn> backPressureCallback;
-    private AtomicReference<IFn> assignmentsCallback;
-    private ConcurrentHashMap<String, IFn> stormBaseCallback;
-    private AtomicReference<IFn> blobstoreCallback;
-    private ConcurrentHashMap<String, IFn> credentialsCallback;
-    private ConcurrentHashMap<String, IFn> logConfigCallback;
+    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;
@@ -129,20 +127,20 @@ public class StormClusterStateImpl implements IStormClusterState {
 
     }
 
-    protected void issueCallback(AtomicReference<IFn> cb) {
-        IFn callback = cb.getAndSet(null);
+    protected void issueCallback(AtomicReference<Runnable> cb) {
+        Runnable callback = cb.getAndSet(null);
         if (callback != null)
-            callback.invoke();
+            callback.run();
     }
 
-    protected void issueMapCallback(ConcurrentHashMap<String, IFn> callbackConcurrentHashMap, String key) {
-        IFn callback = callbackConcurrentHashMap.remove(key);
+    protected void issueMapCallback(ConcurrentHashMap<String, Runnable> callbackConcurrentHashMap, String key) {
+        Runnable callback = callbackConcurrentHashMap.remove(key);
         if (callback != null)
-            callback.invoke();
+            callback.run();
     }
 
     @Override
-    public List<String> assignments(IFn callback) {
+    public List<String> assignments(Runnable callback) {
         if (callback != null) {
             assignmentsCallback.set(callback);
         }
@@ -150,7 +148,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public Assignment assignmentInfo(String stormId, IFn callback) {
+    public Assignment assignmentInfo(String stormId, Runnable callback) {
         if (callback != null) {
             assignmentInfoCallback.put(stormId, callback);
         }
@@ -159,23 +157,25 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback) {
+    public Map assignmentInfoWithVersion(String stormId, Runnable callback) {
+        Map map = new HashMap();
         if (callback != null) {
             assignmentInfoWithVersionCallback.put(stormId, callback);
         }
         Assignment assignment = null;
         Integer version = 0;
-        APersistentMap aPersistentMap = stateStorage.get_data_with_version(ClusterUtils.assignmentPath(stormId), callback != null);
-        if (aPersistentMap != null) {
-            assignment = ClusterUtils.maybeDeserialize((byte[]) aPersistentMap.get(RT.keyword(null, "data")), Assignment.class);
-            version = (Integer) aPersistentMap.get(RT.keyword(null, "version"));
+        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);
         }
-        APersistentMap map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), assignment, RT.keyword(null, "version"), version });
+        map.put(IStateStorage.DATA, assignment);
+        map.put(IStateStorage.VERSION, version);
         return map;
     }
 
     @Override
-    public Integer assignmentVersion(String stormId, IFn callback) throws Exception {
+    public Integer assignmentVersion(String stormId, Runnable callback) throws Exception {
         if (callback != null) {
             assignmentVersionCallback.put(stormId, callback);
         }
@@ -227,7 +227,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public StormBase stormBase(String stormId, IFn callback) {
+    public StormBase stormBase(String stormId, Runnable callback) {
         if (callback != null) {
             stormBaseCallback.put(stormId, callback);
         }
@@ -298,10 +298,10 @@ public class StormClusterStateImpl implements IStormClusterState {
      * @return
      */
     @Override
-    public Map<ExecutorInfo, APersistentMap> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
-        Map<ExecutorInfo, APersistentMap> executorWhbs = new HashMap<>();
+    public Map<ExecutorInfo, ExecutorBeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort) {
+        Map<ExecutorInfo, ExecutorBeat> executorWhbs = new HashMap<>();
 
-        Map<NodeInfo, List<List<Long>>> nodePortExecutors = ClusterUtils.reverseMap(executorNodePort);
+        Map<NodeInfo, List<List<Long>>> nodePortExecutors = Utils.reverseMap(executorNodePort);
 
         for (Map.Entry<NodeInfo, List<List<Long>>> entry : nodePortExecutors.entrySet()) {
 
@@ -319,7 +319,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public List<String> supervisors(IFn callback) {
+    public List<String> supervisors(Runnable callback) {
         if (callback != null) {
             supervisorsCallback.set(callback);
         }
@@ -342,7 +342,7 @@ public class StormClusterStateImpl implements IStormClusterState {
         try {
             stateStorage.delete_worker_hb(ClusterUtils.workerbeatStormRoot(stormId));
         } catch (Exception e) {
-            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
                 // do nothing
                 LOG.warn("Could not teardown heartbeats for {}.", stormId);
             } else {
@@ -356,7 +356,7 @@ public class StormClusterStateImpl implements IStormClusterState {
         try {
             stateStorage.delete_node(ClusterUtils.errorStormRoot(stormId));
         } catch (Exception e) {
-            if (Zookeeper.exceptionCause(KeeperException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.class, e)) {
                 // do nothing
                 LOG.warn("Could not teardown errors for {}.", stormId);
             } else {
@@ -381,7 +381,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public LogConfig topologyLogConfig(String stormId, IFn cb) {
+    public LogConfig topologyLogConfig(String stormId, Runnable cb) {
         String path = ClusterUtils.logConfigPath(stormId);
         return ClusterUtils.maybeDeserialize(stateStorage.get_data(path, cb != null), LogConfig.class);
     }
@@ -437,7 +437,7 @@ public class StormClusterStateImpl implements IStormClusterState {
      * @return
      */
     @Override
-    public boolean topologyBackpressure(String stormId, IFn callback) {
+    public boolean topologyBackpressure(String stormId, Runnable callback) {
         if (callback != null) {
             backPressureCallback.put(stormId, callback);
         }
@@ -568,7 +568,7 @@ public class StormClusterStateImpl implements IStormClusterState {
 
     // blobstore state
     @Override
-    public List<String> blobstore(IFn callback) {
+    public List<String> blobstore(Runnable callback) {
         if (callback != null) {
             blobstoreCallback.set(callback);
         }
@@ -602,7 +602,7 @@ public class StormClusterStateImpl implements IStormClusterState {
 
         String path = ClusterUtils.errorPath(stormId, componentId);
         String lastErrorPath = ClusterUtils.lastErrorPath(stormId, componentId);
-        ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.StringifyError(error), Time.currentTimeSecs());
+        ErrorInfo errorInfo = new ErrorInfo(ClusterUtils.stringifyError(error), Time.currentTimeSecs());
         errorInfo.set_host(node);
         errorInfo.set_port(port.intValue());
         byte[] serData = Utils.serialize(errorInfo);
@@ -669,7 +669,7 @@ public class StormClusterStateImpl implements IStormClusterState {
     }
 
     @Override
-    public Credentials credentials(String stormId, IFn callback) {
+    public Credentials credentials(String stormId, Runnable callback) {
         if (callback != null) {
             credentialsCallback.put(stormId, callback);
         }

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index b277751..56115ce 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -17,7 +17,6 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.state.*;
 import org.apache.curator.framework.state.ConnectionState;
@@ -220,7 +219,7 @@ public class ZKStateStorage implements IStateStorage {
     }
 
     @Override
-    public APersistentMap get_data_with_version(String path, boolean watch) {
+    public Map get_data_with_version(String path, boolean watch) {
         return Zookeeper.getDataWithVersion(zkReader, path, watch);
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index 956c20e..232488b 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -17,16 +17,16 @@
  */
 package org.apache.storm.cluster;
 
-import clojure.lang.APersistentMap;
 import org.apache.storm.utils.Utils;
 import org.apache.zookeeper.data.ACL;
 
 import java.util.List;
+import java.util.Map;
 
 public class ZKStateStorageFactory implements StateStorageFactory {
 
     @Override
-    public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+    public IStateStorage mkStore(Map config, Map auth_conf, List<ACL> acls, ClusterStateContext context) {
         try {
             return new ZKStateStorage(config, auth_conf, acls, context);
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/afcd0c6c/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
index f80b0a4..e5b2666 100644
--- a/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
+++ b/storm-core/src/jvm/org/apache/storm/zookeeper/Zookeeper.java
@@ -17,15 +17,11 @@
  */
 package org.apache.storm.zookeeper;
 
-import clojure.lang.APersistentMap;
-import clojure.lang.PersistentArrayMap;
-import clojure.lang.RT;
 import org.apache.commons.lang.StringUtils;
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.CuratorEvent;
 import org.apache.curator.framework.api.CuratorEventType;
 import org.apache.curator.framework.api.CuratorListener;
-import org.apache.curator.framework.api.UnhandledErrorListener;
 import org.apache.curator.framework.recipes.leader.LeaderLatch;
 import org.apache.curator.framework.recipes.leader.LeaderLatchListener;
 import org.apache.curator.framework.recipes.leader.Participant;
@@ -33,6 +29,7 @@ import org.apache.curator.framework.state.ConnectionStateListener;
 import org.apache.storm.Config;
 import org.apache.storm.callback.DefaultWatcherCallBack;
 import org.apache.storm.callback.WatcherCallBack;
+import org.apache.storm.cluster.IStateStorage;
 import org.apache.storm.nimbus.ILeaderElector;
 import org.apache.storm.nimbus.NimbusInfo;
 import org.apache.storm.utils.Utils;
@@ -47,17 +44,13 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.File;
-import java.io.IOException;
 import java.net.BindException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
-import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.atomic.AtomicReference;
-import java.util.Vector;
+
 
 public class Zookeeper {
     private static Logger LOG = LoggerFactory.getLogger(Zookeeper.class);
@@ -169,7 +162,7 @@ public class Zookeeper {
                 zk.delete().deletingChildrenIfNeeded().forPath(normalizePath(path));
             }
         } catch (Exception e) {
-            if (exceptionCause(KeeperException.NodeExistsException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
                 // do nothing
                 LOG.info("delete {} failed.", path, e);
             } else {
@@ -195,7 +188,7 @@ public class Zookeeper {
         try {
             createNode(zk, npath, byteArray, org.apache.zookeeper.CreateMode.PERSISTENT, acls);
         } catch (Exception e) {
-            if (exceptionCause(KeeperException.NodeExistsException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.NodeExistsException.class, e)) {
                 // this can happen when multiple clients doing mkdir at same time
             }
         }
@@ -224,7 +217,7 @@ public class Zookeeper {
                 }
             }
         } catch (Exception e) {
-            if (exceptionCause(KeeperException.NoNodeException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
                 // this is fine b/c we still have a watch from the successful exists call
             } else {
                 throw Utils.wrapInRuntime(e);
@@ -312,7 +305,7 @@ public class Zookeeper {
         }
         LOG.info("Starting inprocess zookeeper at port {} and dir {}", report, localdir);
         factory.startup(zk);
-        return Arrays.asList((Object)new Long(report), (Object)factory);
+        return Arrays.asList((Object) new Long(report), (Object) factory);
     }
 
     public static void shutdownInprocessZookeeper(NIOServerCnxnFactory handle) {
@@ -361,9 +354,8 @@ public class Zookeeper {
         return new LeaderElectorImp(conf, servers, zk, leaderLockPath, id, leaderLatchAtomicReference, leaderLatchListenerAtomicReference);
     }
 
-    // To update @return to be a Map
-    public static APersistentMap getDataWithVersion(CuratorFramework zk, String path, boolean watch) {
-        APersistentMap map = null;
+    public static Map getDataWithVersion(CuratorFramework zk, String path, boolean watch) {
+        Map map = new HashMap();
         try {
             byte[] bytes = null;
             Stat stats = new Stat();
@@ -376,11 +368,12 @@ public class Zookeeper {
                 }
                 if (bytes != null) {
                     int version = stats.getVersion();
-                    map = new PersistentArrayMap(new Object[] { RT.keyword(null, "data"), bytes, RT.keyword(null, "version"), version });
+                    map.put(IStateStorage.DATA, bytes);
+                    map.put(IStateStorage.VERSION, version);
                 }
             }
         } catch (Exception e) {
-            if (exceptionCause(KeeperException.NoNodeException.class, e)) {
+            if (Utils.exceptionCauseIsInstanceOf(KeeperException.NoNodeException.class, e)) {
                 // this is fine b/c we still have a watch from the successful exists call
             } else {
                 Utils.wrapInRuntime(e);
@@ -423,19 +416,4 @@ public class Zookeeper {
         String rtn = toksToPath(tokenizePath(path));
         return rtn;
     }
-
-    // To remove exceptionCause if port Utils.try-cause to java
-    public static boolean exceptionCause(Class klass, Throwable t) {
-        boolean ret = false;
-        Throwable throwable = t;
-        while (throwable != null) {
-            if (throwable.getClass() == klass) {
-                ret = true;
-                break;
-            }
-            throwable = throwable.getCause();
-        }
-        return ret;
-    }
-
 }


[23/27] storm git commit: convert ExecutorStats to stats of clojure by clojurify-executor-stats

Posted by bo...@apache.org.
convert ExecutorStats to stats of clojure by clojurify-executor-stats


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/d7095251
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/d7095251
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/d7095251

Branch: refs/heads/master
Commit: d70952513336532c80957e5398bfbedacff78cb7
Parents: 581cd29
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 23 22:05:50 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 23 22:05:50 2016 +0800

----------------------------------------------------------------------
 storm-core/src/clj/org/apache/storm/converter.clj     | 2 +-
 storm-core/src/clj/org/apache/storm/daemon/nimbus.clj | 9 ++++-----
 2 files changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/d7095251/storm-core/src/clj/org/apache/storm/converter.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/converter.clj b/storm-core/src/clj/org/apache/storm/converter.clj
index c845cd4..e269c5d 100644
--- a/storm-core/src/clj/org/apache/storm/converter.clj
+++ b/storm-core/src/clj/org/apache/storm/converter.clj
@@ -241,7 +241,7 @@
 
 (defn clojurify-zk-executor-hb [^ExecutorBeat executor-hb]
   (if executor-hb
-    {:stats (.getStats executor-hb)
+    {:stats (clojurify-executor-stats (.getStats executor-hb))
      :uptime (.getUptime executor-hb)
      :time-secs (.getTimeSecs executor-hb)
      }

http://git-wip-us.apache.org/repos/asf/storm/blob/d7095251/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
index e43bab9..2f6587a 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/nimbus.clj
@@ -597,7 +597,6 @@
                          (->> (dofor [[^ExecutorInfo executor-info  ^ExecutorBeat executor-heartbeat] executor-stats-clojurify]
                              {[(.get_task_start executor-info) (.get_task_end executor-info)] (clojurify-zk-executor-hb executor-heartbeat)})
                            (apply merge)))
-
         cache (update-heartbeat-cache (@(:heartbeats-cache nimbus) storm-id)
                                       executor-beats
                                       all-executors
@@ -1918,16 +1917,16 @@
               executor-summaries (dofor [[executor [node port]] (:executor->node+port assignment)]
                                         (let [host (-> assignment :node->host (get node))
                                               heartbeat (get beats executor)
-                                              stats (:stats heartbeat)
-                                              stats (if stats
-                                                      (stats/thriftify-executor-stats stats))]
+                                              excutorstats (:stats heartbeat)
+                                              excutorstats (if excutorstats
+                                                      (stats/thriftify-executor-stats excutorstats))]
                                           (doto
                                               (ExecutorSummary. (thriftify-executor-id executor)
                                                                 (-> executor first task->component)
                                                                 host
                                                                 port
                                                                 (Utils/nullToZero (:uptime heartbeat)))
-                                            (.set_stats stats))
+                                            (.set_stats excutorstats))
                                           ))
               topo-info  (TopologyInfo. storm-id
                            storm-name


[16/27] storm git commit: fix nimbus_test.clj

Posted by bo...@apache.org.
fix nimbus_test.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/e0f3cb5f
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/e0f3cb5f
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/e0f3cb5f

Branch: refs/heads/master
Commit: e0f3cb5f322c1dc09d05803e93d1fb3e6a3baff0
Parents: 419be35
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Tue Feb 16 11:40:48 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Tue Feb 16 11:40:48 2016 +0800

----------------------------------------------------------------------
 storm-core/test/clj/org/apache/storm/nimbus_test.clj | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/e0f3cb5f/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index 2a65efc..09c4371 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -1411,8 +1411,6 @@
           cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [_ (ConfigUtilsInstaller. fake-cu)
                   _ (UtilsInstaller. fake-utils)
-                  _ (proxy [ConfigUtils] []
-                      (nimbusTopoHistoryStateImpl [conf] nil))
                   zk-le (MockedZookeeper. (proxy [Zookeeper] []
                           (zkLeaderElectorImpl [conf] nil)))
                   mocked-cluster (MockedCluster. cluster-utils)]


[06/27] storm git commit: Merge branch 'master' into cluster

Posted by bo...@apache.org.
Merge branch 'master' into cluster


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2a17c455
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2a17c455
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2a17c455

Branch: refs/heads/master
Commit: 2a17c45591692b61df0be922fdec9d10a108de27
Parents: 5916b0b d236036
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Wed Feb 3 21:18:13 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Wed Feb 3 21:18:13 2016 +0800

----------------------------------------------------------------------
 CHANGELOG.md                                    |  1 +
 README.markdown                                 |  1 +
 external/storm-hdfs/README.md                   | 15 +++-
 external/storm-hdfs/pom.xml                     | 22 ++++++
 .../storm/hdfs/avro/AbstractAvroSerializer.java | 80 +++++++++++++++++++
 .../storm/hdfs/avro/AvroSchemaRegistry.java     | 28 +++++++
 .../org/apache/storm/hdfs/avro/AvroUtils.java   | 44 +++++++++++
 .../hdfs/avro/ConfluentAvroSerializer.java      | 83 ++++++++++++++++++++
 .../storm/hdfs/avro/FixedAvroSerializer.java    | 67 ++++++++++++++++
 .../storm/hdfs/avro/GenericAvroSerializer.java  | 36 +++++++++
 .../storm/hdfs/bolt/AvroGenericRecordBolt.java  |  4 -
 .../hdfs/avro/TestFixedAvroSerializer.java      | 76 ++++++++++++++++++
 .../hdfs/avro/TestGenericAvroSerializer.java    | 68 ++++++++++++++++
 .../test/resources/FixedAvroSerializer.config   |  2 +
 pom.xml                                         |  3 +
 .../clj/org/apache/storm/daemon/executor.clj    |  2 +-
 .../src/clj/org/apache/storm/daemon/task.clj    |  2 +
 17 files changed, 527 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2a17c455/storm-core/src/clj/org/apache/storm/daemon/executor.clj
----------------------------------------------------------------------


[02/27] storm git commit: delete zookeeper.clj zookeeper_state_factory.clj cluster.clj, but some tests still can't pass

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index edb161b..c98a68b 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -23,15 +23,18 @@
   (:import [org.apache.storm.scheduler ISupervisor])
   (:import [org.apache.storm.utils ConfigUtils])
   (:import [org.apache.storm.generated RebalanceOptions])
-  (:import [org.apache.storm.testing.staticmocking MockedConfigUtils])
+  (:import [org.apache.storm.testing.staticmocking MockedConfigUtils MockedCluster])
   (:import [java.util UUID])
+  (:import [org.mockito Mockito])
+  (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [java.io File])
   (:import [java.nio.file Files])
+  (:import [org.apache.storm.cluster StormZkClusterState Cluster ClusterStateContext])
   (:import [java.nio.file.attribute FileAttribute])
-  (:use [org.apache.storm config testing util timer log])
+  (:use [org.apache.storm config testing util timer log converter])
   (:use [org.apache.storm.daemon common])
   (:require [org.apache.storm.daemon [worker :as worker] [supervisor :as supervisor]]
-            [org.apache.storm [thrift :as thrift] [cluster :as cluster]])
+            [org.apache.storm [thrift :as thrift]])
   (:use [conjure core])
   (:require [clojure.java.io :as io]))
 
@@ -40,7 +43,7 @@
   [cluster supervisor-id port]
   (let [state (:storm-cluster-state cluster)
         slot-assigns (for [storm-id (.assignments state nil)]
-                        (let [executors (-> (.assignment-info state storm-id nil)
+                        (let [executors (-> (clojurify-assignment (.assignmentInfo state storm-id nil))
                                         :executor->node+port
                                         reverse-map
                                         (get [supervisor-id port] ))]
@@ -225,7 +228,7 @@
       )))
 
 (defn get-heartbeat [cluster supervisor-id]
-  (.supervisor-info (:storm-cluster-state cluster) supervisor-id))
+  (clojurify-supervisor-info (.supervisorInfo (:storm-cluster-state cluster) supervisor-id)))
 
 (defn check-heartbeat [cluster supervisor-id within-secs]
   (let [hb (get-heartbeat cluster supervisor-id)
@@ -561,18 +564,22 @@
           expected-acls supervisor/SUPERVISOR-ZK-ACLS
           fake-isupervisor (reify ISupervisor
                              (getSupervisorId [this] nil)
-                             (getAssignmentId [this] nil))]
+                             (getAssignmentId [this] nil))
+          storm-zk (Mockito/mock Cluster)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (supervisorStateImpl [conf] nil)
-                      (supervisorLocalDirImpl [conf] nil))]
+                      (supervisorLocalDirImpl [conf] nil))
+                  storm-zk-le (MockedCluster. storm-zk)]
         (stubbing [uptime-computer nil
-                 cluster/mk-storm-cluster-state nil
+              ;   cluster/mk-storm-cluster-state nil
                  local-hostname nil
                  mk-timer nil]
           (supervisor/supervisor-data auth-conf nil fake-isupervisor)
-          (verify-call-times-for cluster/mk-storm-cluster-state 1)
-          (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-                                              expected-acls))))))
+          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+        ;  (verify-call-times-for cluster/mk-storm-cluster-state 1)
+        ;  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
+        ;                                     expected-acls)
+         )))))
 
 (deftest test-write-log-metadata
   (testing "supervisor writes correct data to logs metadata file"

http://git-wip-us.apache.org/repos/asf/storm/blob/9a79fb7d/storm-core/test/jvm/org/apache/storm/ClusterTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
new file mode 100644
index 0000000..ef43afe
--- /dev/null
+++ b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
@@ -0,0 +1,22 @@
+/**
+ * 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;
+
+
+public class ClusterTest {
+}


[10/27] storm git commit: port pacemaker_state_factory.clj

Posted by bo...@apache.org.
port pacemaker_state_factory.clj


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/2ee8bec8
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/2ee8bec8
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/2ee8bec8

Branch: refs/heads/master
Commit: 2ee8bec8458b02bca6af757ce0f1052a16c660b8
Parents: 55b86ca
Author: xiaojian.fxj <xi...@alibaba-inc.com>
Authored: Mon Feb 15 09:37:32 2016 +0800
Committer: xiaojian.fxj <xi...@alibaba-inc.com>
Committed: Mon Feb 15 09:37:32 2016 +0800

----------------------------------------------------------------------
 .../clj/org/apache/storm/command/heartbeats.clj |   2 +-
 .../src/clj/org/apache/storm/daemon/worker.clj  |   2 +-
 .../storm/pacemaker/pacemaker_state_factory.clj |   6 +-
 storm-core/src/clj/org/apache/storm/testing.clj |   2 +-
 .../org/apache/storm/cluster/ClusterUtils.java  |  20 +-
 .../org/apache/storm/cluster/IStateStorage.java | 219 ++++++++++++++++++
 .../storm/cluster/IStormClusterState.java       | 129 +++++++++++
 .../storm/cluster/PaceMakerStateStorage.java    | 212 ++++++++++++++++++
 .../cluster/PaceMakerStateStorageFactory.java   |  64 ++++++
 .../org/apache/storm/cluster/StateStorage.java  | 220 -------------------
 .../storm/cluster/StateStorageFactory.java      |   2 +-
 .../apache/storm/cluster/StormClusterState.java | 129 -----------
 .../storm/cluster/StormClusterStateImpl.java    |   8 +-
 .../apache/storm/cluster/ZKStateStorage.java    |   4 +-
 .../storm/cluster/ZKStateStorageFactory.java    |   2 +-
 .../apache/storm/pacemaker/PacemakerClient.java |   1 -
 .../MockedPaceMakerStateStorageFactory.java     |  32 +++
 .../test/clj/org/apache/storm/cluster_test.clj  |  10 +-
 .../storm/pacemaker_state_factory_test.clj      |  57 ++---
 19 files changed, 715 insertions(+), 406 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
index af86b69..c4413f0 100644
--- a/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
+++ b/storm-core/src/clj/org/apache/storm/command/heartbeats.clj
@@ -27,7 +27,7 @@
 
 (defn -main [command path & args]
   (let [conf (clojurify-structure (ConfigUtils/readStormConfig))
-        cluster (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))]
+        cluster (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))]
     (println "Command: [" command "]")
     (condp = command
       "list"

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/daemon/worker.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/daemon/worker.clj b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
index a793009..ae5be57 100644
--- a/storm-core/src/clj/org/apache/storm/daemon/worker.clj
+++ b/storm-core/src/clj/org/apache/storm/daemon/worker.clj
@@ -596,7 +596,7 @@
   (let [storm-conf (ConfigUtils/readSupervisorStormConf conf storm-id)
         storm-conf (clojurify-structure (ConfigUtils/overrideLoginConfigWithSystemProperty storm-conf))
         acls (Utils/getWorkerACL storm-conf)
-        state-store (ClusterUtils/mkDistributedClusterState conf storm-conf  acls  (ClusterStateContext. DaemonType/WORKER))
+        state-store (ClusterUtils/mkStateStorage conf storm-conf  acls  (ClusterStateContext. DaemonType/WORKER))
         storm-cluster-state (ClusterUtils/mkStormClusterState state-store acls (ClusterStateContext.))
         initial-credentials (clojurify-crdentials (.credentials storm-cluster-state storm-id nil))
         auto-creds (AuthUtils/GetAutoCredentials storm-conf)

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
index 28f792d..a36da3a 100644
--- a/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
+++ b/storm-core/src/clj/org/apache/storm/pacemaker/pacemaker_state_factory.clj
@@ -23,7 +23,7 @@
   (:import [org.apache.storm.generated
             HBExecutionException HBServerMessageType HBMessage
             HBMessageData HBPulse]
-           [org.apache.storm.cluster ZKStateStorage StateStorage ClusterUtils]
+           [org.apache.storm.cluster ZKStateStorage ClusterUtils IStateStorage]
            [org.apache.storm.pacemaker PacemakerClient])
   (:gen-class
     :implements [org.apache.storm.cluster.StateStorageFactory]))
@@ -33,7 +33,7 @@
   (PacemakerClient. conf))
 
 (defn makeZKState [conf auth-conf acls context]
-  (ClusterUtils/mkDistributedClusterState conf auth-conf acls context))
+  (ClusterUtils/mkStateStorage conf auth-conf acls context))
 
 (def max-retries 10)
 
@@ -42,7 +42,7 @@
         pacemaker-client (makeClient conf)]
 
     (reify
-      StateStorage
+      IStateStorage
       ;; Let these pass through to the zk-state. We only want to handle heartbeats.
       (register [this callback] (.register zk-state callback))
       (unregister [this callback] (.unregister zk-state callback))

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/clj/org/apache/storm/testing.clj
----------------------------------------------------------------------
diff --git a/storm-core/src/clj/org/apache/storm/testing.clj b/storm-core/src/clj/org/apache/storm/testing.clj
index 470a14f..5a0bdf2 100644
--- a/storm-core/src/clj/org/apache/storm/testing.clj
+++ b/storm-core/src/clj/org/apache/storm/testing.clj
@@ -158,7 +158,7 @@
                      :port-counter port-counter
                      :daemon-conf daemon-conf
                      :supervisors (atom [])
-                     :state (ClusterUtils/mkDistributedClusterState daemon-conf nil nil (ClusterStateContext.))
+                     :state (ClusterUtils/mkStateStorage daemon-conf nil nil (ClusterStateContext.))
                      :storm-cluster-state (ClusterUtils/mkStormClusterState daemon-conf nil (ClusterStateContext.))
                      :tmp-dirs (atom [nimbus-tmp zk-tmp])
                      :zookeeper (if (not-nil? zk-handle) zk-handle)

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
index 9fd36ca..b30d1d2 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ClusterUtils.java
@@ -194,20 +194,20 @@ public class ClusterUtils {
         return executorWhb;
     }
 
-    public StormClusterState mkStormClusterStateImpl(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
-        if (StateStorage instanceof StateStorage) {
-            return new StormClusterStateImpl((StateStorage) StateStorage, acls, context, false);
+    public IStormClusterState mkStormClusterStateImpl(Object stateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+        if (stateStorage instanceof IStateStorage) {
+            return new StormClusterStateImpl((IStateStorage) stateStorage, acls, context, false);
         } else {
-            StateStorage Storage = _instance.mkDistributedClusterStateImpl((APersistentMap) StateStorage, (APersistentMap) StateStorage, acls, context);
+            IStateStorage Storage = _instance.mkStateStorageImpl((APersistentMap) stateStorage, (APersistentMap) stateStorage, acls, context);
             return new StormClusterStateImpl(Storage, acls, context, true);
         }
 
     }
 
-    public StateStorage mkDistributedClusterStateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+    public IStateStorage mkStateStorageImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
             throws Exception {
         String className = null;
-        StateStorage stateStorage = null;
+        IStateStorage stateStorage = null;
         if (config.get(Config.STORM_CLUSTER_STATE_STORE) != null) {
             className = (String) config.get(Config.STORM_CLUSTER_STATE_STORE);
         } else {
@@ -215,16 +215,16 @@ public class ClusterUtils {
         }
         Class clazz = Class.forName(className);
         StateStorageFactory storageFactory = (StateStorageFactory) clazz.newInstance();
-        stateStorage = storageFactory.mkState(config, auth_conf, acls, context);
+        stateStorage = storageFactory.mkStore(config, auth_conf, acls, context);
         return stateStorage;
     }
 
-    public static StateStorage mkDistributedClusterState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
+    public static IStateStorage mkStateStorage(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context)
             throws Exception {
-        return _instance.mkDistributedClusterStateImpl(config, auth_conf, acls, context);
+        return _instance.mkStateStorageImpl(config, auth_conf, acls, context);
     }
 
-    public static StormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
+    public static IStormClusterState mkStormClusterState(Object StateStorage, List<ACL> acls, ClusterStateContext context) throws Exception {
         return _instance.mkStormClusterStateImpl(StateStorage, acls, context);
     }
 

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
new file mode 100644
index 0000000..1a2b14f
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStateStorage.java
@@ -0,0 +1,219 @@
+/**
+ * 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.APersistentMap;
+import java.util.List;
+
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.zookeeper.data.ACL;
+
+/**
+ * StateStorage provides the API for the pluggable state store used by the
+ * Storm daemons. Data is stored in path/value format, and the store supports
+ * listing sub-paths at a given path.
+ * All data should be available across all nodes with eventual consistency.
+ *
+ * IMPORTANT NOTE:
+ * Heartbeats have different api calls used to interact with them. The root
+ * path (/) may or may not be the same as the root path for the other api calls.
+ *
+ * For example, performing these two calls:
+ *     set_data("/path", data, acls);
+ *     void set_worker_hb("/path", heartbeat, acls);
+ * may or may not cause a collision in "/path".
+ * Never use the same paths with the *_hb* methods as you do with the others.
+ */
+public interface IStateStorage {
+
+    /**
+     * Registers a callback function that gets called when CuratorEvents happen.
+     * @param callback is a clojure IFn that accepts the type - translated to
+     * clojure keyword as in zookeeper - and the path: (callback type path)
+     * @return is an id that can be passed to unregister(...) to unregister the
+     * callback.
+     */
+    String register(ZKStateChangedCallback callback);
+
+    /**
+     * Unregisters a callback function that was registered with register(...).
+     * @param id is the String id that was returned from register(...).
+     */
+    void unregister(String id);
+
+    /**
+     * Path will be appended with a monotonically increasing integer, a new node
+     * will be created there, and data will be put at that node.
+     * @param path The path that the monotonically increasing integer suffix will
+     * be added to.
+     * @param data The data that will be written at the suffixed path's node.
+     * @param acls The acls to apply to the path. May be null.
+     * @return The path with the integer suffix appended.
+     */
+    String create_sequential(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Creates nodes for path and all its parents. Path elements are separated by
+     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
+     * @param path The path to create, along with all its parents.
+     * @param acls The acls to apply to the path. May be null.
+     * @return path
+     */
+    void mkdirs(String path, List<ACL> acls);
+
+    /**
+     * Deletes the node at a given path, and any child nodes that may exist.
+     * @param path The path to delete
+     */
+    void delete_node(String path);
+
+    /**
+     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
+     * by the store when the client disconnects.
+     * @param path The path where a node will be created.
+     * @param data The data to be written at the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Gets the 'version' of the node at a path. Optionally sets a watch
+     * on that node. The version should increase whenever a write happens.
+     * @param path The path to get the version of.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The integer version of this node.
+     */
+    Integer get_version(String path, boolean watch) throws Exception;
+
+    /**
+     * Check if a node exists and optionally set a watch on the path.
+     * @param path The path to check for the existence of a node.
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return Whether or not a node exists at path.
+     */
+    boolean node_exists(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_children(String path, boolean watch);
+
+    /**
+     * Close the connection to the data store.
+     */
+    void close();
+
+    /**
+     * Set the value of the node at path to data.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_data(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the data from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The data at the node.
+     */
+    byte[] get_data(String path, boolean watch);
+
+    /**
+     * Get the data at the node along with its version. Data is returned
+     * in an APersistentMap with clojure keyword keys :data and :version.
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return An APersistentMap in the form {:data data :version version}
+     */
+    APersistentMap get_data_with_version(String path, boolean watch);
+
+    /**
+     * Write a worker heartbeat at the path.
+     * @param path The path whose node we want to set.
+     * @param data The data to put in the node.
+     * @param acls The acls to apply to the path. May be null.
+     */
+    void set_worker_hb(String path, byte[] data, List<ACL> acls);
+
+    /**
+     * Get the heartbeat from the node at path
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return The heartbeat at the node.
+     */
+    byte[] get_worker_hb(String path, boolean watch);
+
+    /**
+     * Get a list of paths of all the child nodes which exist immediately
+     * under path. This is similar to get_children, but must be used for
+     * any nodes
+     * @param path The path to look under
+     * @param watch Whether or not to set a watch on the path. Watched paths
+     * emit events which are consumed by functions registered with the
+     * register method. Very useful for catching updates to nodes.
+     * @return list of string paths under path.
+     */
+    List<String> get_worker_hb_children(String path, boolean watch);
+
+    /**
+     * Deletes the heartbeat at a given path, and any child nodes that may exist.
+     * @param path The path to delete.
+     */
+    void delete_worker_hb(String path);
+
+    /**
+     * Add a StateStorageListener to the connection.
+     * @param listener A StateStorageListener to handle changing cluster state
+     * events.
+     */
+    void add_listener(final ConnectionStateListener listener);
+
+    /**
+     * Force consistency on a path. Any writes committed on the path before
+     * this call will be completely propagated when it returns.
+     * @param path The path to synchronize.
+     */
+    void sync_path(String path);
+
+    /**
+     * Allows us to delete the znodes within /storm/blobstore/key_name
+     * whose znodes start with the corresponding nimbusHostPortInfo
+     * @param path /storm/blobstore/key_name
+     * @param nimbusHostPortInfo Contains the host port information of
+     * a nimbus node.
+     */
+    void delete_node_blobstore(String path, String nimbusHostPortInfo);
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
new file mode 100644
index 0000000..59d1af7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/IStormClusterState.java
@@ -0,0 +1,129 @@
+/**
+ * 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.APersistentMap;
+import clojure.lang.IFn;
+import org.apache.storm.generated.*;
+import org.apache.storm.nimbus.NimbusInfo;
+
+import java.security.NoSuchAlgorithmException;
+import java.util.List;
+import java.util.Map;
+
+public interface IStormClusterState {
+    public List<String> assignments(IFn callback);
+
+    public Assignment assignmentInfo(String stormId, IFn callback);
+
+    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
+
+    public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
+
+    // returns key information under /storm/blobstore/key
+    public List<String> blobstoreInfo(String blobKey);
+
+    // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
+    public List nimbuses();
+
+    // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
+    public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
+
+    public List<String> activeStorms();
+
+    public StormBase stormBase(String stormId, IFn callback);
+
+    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
+
+    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
+
+    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
+
+    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
+
+    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
+
+    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
+
+    public List<String> supervisors(IFn callback);
+
+    public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
+
+    public void setupHeatbeats(String stormId);
+
+    public void teardownHeartbeats(String stormId);
+
+    public void teardownTopologyErrors(String stormId);
+
+    public List<String> heartbeatStorms();
+
+    public List<String> errorTopologies();
+
+    public void setTopologyLogConfig(String stormId, LogConfig logConfig);
+
+    public LogConfig topologyLogConfig(String stormId, IFn cb);
+
+    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
+
+    public void removeWorkerHeartbeat(String stormId, String node, Long port);
+
+    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
+
+    public void workerBackpressure(String stormId, String node, Long port, boolean on);
+
+    public boolean topologyBackpressure(String stormId, IFn callback);
+
+    public void setupBackpressure(String stormId);
+
+    public void removeWorkerBackpressure(String stormId, String node, Long port);
+
+    public void activateStorm(String stormId, StormBase stormBase);
+
+    public void updateStorm(String stormId, StormBase newElems);
+
+    public void removeStormBase(String stormId);
+
+    public void setAssignment(String stormId, Assignment info);
+
+    // sets up information related to key consisting of nimbus
+    // host:port and version info of the blob
+    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
+
+    public List<String> activeKeys();
+
+    public List<String> blobstore(IFn callback);
+
+    public void removeStorm(String stormId);
+
+    public void removeBlobstoreKey(String blobKey);
+
+    public void removeKeyVersion(String blobKey);
+
+    public void reportError(String stormId, String componentId, String node, Long port, String error);
+
+    public List<ErrorInfo> errors(String stormId, String componentId);
+
+    public ErrorInfo lastError(String stormId, String componentId);
+
+    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
+
+    public Credentials credentials(String stormId, IFn callback);
+
+    public void disconnect();
+
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
new file mode 100644
index 0000000..1226c55
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorage.java
@@ -0,0 +1,212 @@
+/**
+ * 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.APersistentMap;
+import org.apache.curator.framework.state.ConnectionStateListener;
+import org.apache.storm.callback.ZKStateChangedCallback;
+import org.apache.storm.generated.*;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class PaceMakerStateStorage implements IStateStorage {
+
+    private static Logger LOG = LoggerFactory.getLogger(PaceMakerStateStorage.class);
+
+    private PacemakerClient pacemakerClient;
+    private IStateStorage stateStorage;
+    private static final int maxRetries = 10;
+
+    public PaceMakerStateStorage(PacemakerClient pacemakerClient, IStateStorage stateStorage) throws Exception {
+        this.pacemakerClient = pacemakerClient;
+        this.stateStorage = stateStorage;
+    }
+
+    @Override
+    public String register(ZKStateChangedCallback callback) {
+        return stateStorage.register(callback);
+    }
+
+    @Override
+    public void unregister(String id) {
+        stateStorage.unregister(id);
+    }
+
+    @Override
+    public String create_sequential(String path, byte[] data, List<ACL> acls) {
+        return stateStorage.create_sequential(path, data, acls);
+    }
+
+    @Override
+    public void mkdirs(String path, List<ACL> acls) {
+        stateStorage.mkdirs(path, acls);
+    }
+
+    @Override
+    public void delete_node(String path) {
+        stateStorage.delete_node(path);
+    }
+
+    @Override
+    public void set_ephemeral_node(String path, byte[] data, List<ACL> acls) {
+        stateStorage.set_ephemeral_node(path, data, acls);
+    }
+
+    @Override
+    public Integer get_version(String path, boolean watch) throws Exception {
+        return stateStorage.get_version(path, watch);
+    }
+
+    @Override
+    public boolean node_exists(String path, boolean watch) {
+        return stateStorage.node_exists(path, watch);
+    }
+
+    @Override
+    public List<String> get_children(String path, boolean watch) {
+        return stateStorage.get_children(path, watch);
+    }
+
+    @Override
+    public void close() {
+        stateStorage.close();
+        pacemakerClient.close();
+    }
+
+    @Override
+    public void set_data(String path, byte[] data, List<ACL> acls) {
+        stateStorage.set_data(path, data, acls);
+    }
+
+    @Override
+    public byte[] get_data(String path, boolean watch) {
+        return stateStorage.get_data(path, watch);
+    }
+
+    @Override
+    public APersistentMap get_data_with_version(String path, boolean watch) {
+        return stateStorage.get_data_with_version(path, watch);
+    }
+
+    @Override
+    public void set_worker_hb(String path, byte[] data, List<ACL> acls) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HBPulse hbPulse = new HBPulse();
+                hbPulse.set_id(path);
+                hbPulse.set_details(data);
+                HBMessage message = new HBMessage(HBServerMessageType.SEND_PULSE, HBMessageData.pulse(hbPulse));
+                HBMessage response = pacemakerClient.send(message);
+                if (response.get_type() != HBServerMessageType.SEND_PULSE_RESPONSE) {
+                    throw new HBExecutionException("Invalid Response Type");
+                }
+                LOG.debug("Successful set_worker_hb");
+                break;
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                LOG.error("{} Failed to set_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+            }
+        }
+    }
+
+    @Override
+    public byte[] get_worker_hb(String path, boolean watch) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+                HBMessage response = pacemakerClient.send(message);
+                if (response.get_type() != HBServerMessageType.GET_PULSE_RESPONSE) {
+                    throw new HBExecutionException("Invalid Response Type");
+                }
+                LOG.debug("Successful get_worker_hb");
+                return response.get_data().get_pulse().get_details();
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                LOG.error("{} Failed to get_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+            }
+        }
+    }
+
+    @Override
+    public List<String> get_worker_hb_children(String path, boolean watch) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+                HBMessage response = pacemakerClient.send(message);
+                if (response.get_type() != HBServerMessageType.GET_ALL_NODES_FOR_PATH_RESPONSE) {
+                    throw new HBExecutionException("Invalid Response Type");
+                }
+                LOG.debug("Successful get_worker_hb");
+                return response.get_data().get_nodes().get_pulseIds();
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                LOG.error("{} Failed to get_worker_hb_children. Will make {} more attempts.", e.getMessage(), retry--);
+            }
+        }
+    }
+
+    @Override
+    public void delete_worker_hb(String path) {
+        int retry = maxRetries;
+        while (true) {
+            try {
+                HBMessage message = new HBMessage(HBServerMessageType.GET_PULSE, HBMessageData.path(path));
+                HBMessage response = pacemakerClient.send(message);
+                if (response.get_type() != HBServerMessageType.DELETE_PATH_RESPONSE) {
+                    throw new HBExecutionException("Invalid Response Type");
+                }
+                LOG.debug("Successful get_worker_hb");
+                break;
+            } catch (Exception e) {
+                if (retry <= 0) {
+                    throw Utils.wrapInRuntime(e);
+                }
+                LOG.error("{} Failed to delete_worker_hb. Will make {} more attempts.", e.getMessage(), retry--);
+            }
+        }
+    }
+
+    @Override
+    public void add_listener(ConnectionStateListener listener) {
+        stateStorage.add_listener(listener);
+    }
+
+    @Override
+    public void sync_path(String path) {
+        stateStorage.sync_path(path);
+    }
+
+    @Override
+    public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
+        stateStorage.delete_node_blobstore(path, nimbusHostPortInfo);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
new file mode 100644
index 0000000..eafd2e7
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/cluster/PaceMakerStateStorageFactory.java
@@ -0,0 +1,64 @@
+/**
+ * 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.APersistentMap;
+import org.apache.storm.pacemaker.PacemakerClient;
+import org.apache.storm.utils.Utils;
+import org.apache.zookeeper.data.ACL;
+
+import java.util.List;
+
+public class PaceMakerStateStorageFactory implements StateStorageFactory {
+
+    private static final PaceMakerStateStorageFactory INSTANCE = new PaceMakerStateStorageFactory();
+    private static PaceMakerStateStorageFactory _instance = INSTANCE;
+
+    public static void setInstance(PaceMakerStateStorageFactory u) {
+        _instance = u;
+    }
+
+    public static void resetInstance() {
+        _instance = INSTANCE;
+    }
+
+    @Override
+    public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+        try {
+            return new PaceMakerStateStorage(initMakeClient(config), initZKstate(config, auth_conf, acls, context));
+        } catch (Exception e) {
+            throw Utils.wrapInRuntime(e);
+        }
+    }
+
+    public static IStateStorage initZKstate(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        return _instance.initZKstateImpl(config, auth_conf, acls, context);
+    }
+
+    public static PacemakerClient initMakeClient(APersistentMap config) {
+        return _instance.initMakeClientImpl(config);
+    }
+
+    public IStateStorage initZKstateImpl(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) throws Exception {
+        return ClusterUtils.mkStateStorage(config, auth_conf, acls, context);
+    }
+
+    public PacemakerClient initMakeClientImpl(APersistentMap config) {
+        return new PacemakerClient(config);
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
deleted file mode 100644
index 8895cd1..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorage.java
+++ /dev/null
@@ -1,220 +0,0 @@
-/**
- * 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.APersistentMap;
-import clojure.lang.IFn;
-import java.util.List;
-
-import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.storm.callback.ZKStateChangedCallback;
-import org.apache.zookeeper.data.ACL;
-
-/**
- * StateStorage provides the API for the pluggable state store used by the
- * Storm daemons. Data is stored in path/value format, and the store supports
- * listing sub-paths at a given path.
- * All data should be available across all nodes with eventual consistency.
- *
- * IMPORTANT NOTE:
- * Heartbeats have different api calls used to interact with them. The root
- * path (/) may or may not be the same as the root path for the other api calls.
- *
- * For example, performing these two calls:
- *     set_data("/path", data, acls);
- *     void set_worker_hb("/path", heartbeat, acls);
- * may or may not cause a collision in "/path".
- * Never use the same paths with the *_hb* methods as you do with the others.
- */
-public interface StateStorage {
-
-    /**
-     * Registers a callback function that gets called when CuratorEvents happen.
-     * @param callback is a clojure IFn that accepts the type - translated to
-     * clojure keyword as in zookeeper - and the path: (callback type path)
-     * @return is an id that can be passed to unregister(...) to unregister the
-     * callback.
-     */
-    String register(ZKStateChangedCallback callback);
-
-    /**
-     * Unregisters a callback function that was registered with register(...).
-     * @param id is the String id that was returned from register(...).
-     */
-    void unregister(String id);
-
-    /**
-     * Path will be appended with a monotonically increasing integer, a new node
-     * will be created there, and data will be put at that node.
-     * @param path The path that the monotonically increasing integer suffix will
-     * be added to.
-     * @param data The data that will be written at the suffixed path's node.
-     * @param acls The acls to apply to the path. May be null.
-     * @return The path with the integer suffix appended.
-     */
-    String create_sequential(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Creates nodes for path and all its parents. Path elements are separated by
-     * a "/", as in *nix filesystem notation. Equivalent to mkdir -p in *nix.
-     * @param path The path to create, along with all its parents.
-     * @param acls The acls to apply to the path. May be null.
-     * @return path
-     */
-    void mkdirs(String path, List<ACL> acls);
-
-    /**
-     * Deletes the node at a given path, and any child nodes that may exist.
-     * @param path The path to delete
-     */
-    void delete_node(String path);
-
-    /**
-     * Creates an ephemeral node at path. Ephemeral nodes are destroyed
-     * by the store when the client disconnects.
-     * @param path The path where a node will be created.
-     * @param data The data to be written at the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_ephemeral_node(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Gets the 'version' of the node at a path. Optionally sets a watch
-     * on that node. The version should increase whenever a write happens.
-     * @param path The path to get the version of.
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The integer version of this node.
-     */
-    Integer get_version(String path, boolean watch) throws Exception;
-
-    /**
-     * Check if a node exists and optionally set a watch on the path.
-     * @param path The path to check for the existence of a node.
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return Whether or not a node exists at path.
-     */
-    boolean node_exists(String path, boolean watch);
-
-    /**
-     * Get a list of paths of all the child nodes which exist immediately
-     * under path.
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return list of string paths under path.
-     */
-    List<String> get_children(String path, boolean watch);
-
-    /**
-     * Close the connection to the data store.
-     */
-    void close();
-
-    /**
-     * Set the value of the node at path to data.
-     * @param path The path whose node we want to set.
-     * @param data The data to put in the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_data(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Get the data from the node at path
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The data at the node.
-     */
-    byte[] get_data(String path, boolean watch);
-
-    /**
-     * Get the data at the node along with its version. Data is returned
-     * in an APersistentMap with clojure keyword keys :data and :version.
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return An APersistentMap in the form {:data data :version version}
-     */
-    APersistentMap get_data_with_version(String path, boolean watch);
-
-    /**
-     * Write a worker heartbeat at the path.
-     * @param path The path whose node we want to set.
-     * @param data The data to put in the node.
-     * @param acls The acls to apply to the path. May be null.
-     */
-    void set_worker_hb(String path, byte[] data, List<ACL> acls);
-
-    /**
-     * Get the heartbeat from the node at path
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return The heartbeat at the node.
-     */
-    byte[] get_worker_hb(String path, boolean watch);
-
-    /**
-     * Get a list of paths of all the child nodes which exist immediately
-     * under path. This is similar to get_children, but must be used for
-     * any nodes
-     * @param path The path to look under
-     * @param watch Whether or not to set a watch on the path. Watched paths
-     * emit events which are consumed by functions registered with the
-     * register method. Very useful for catching updates to nodes.
-     * @return list of string paths under path.
-     */
-    List<String> get_worker_hb_children(String path, boolean watch);
-
-    /**
-     * Deletes the heartbeat at a given path, and any child nodes that may exist.
-     * @param path The path to delete.
-     */
-    void delete_worker_hb(String path);
-
-    /**
-     * Add a StateStorageListener to the connection.
-     * @param listener A StateStorageListener to handle changing cluster state
-     * events.
-     */
-    void add_listener(final ConnectionStateListener listener);
-
-    /**
-     * Force consistency on a path. Any writes committed on the path before
-     * this call will be completely propagated when it returns.
-     * @param path The path to synchronize.
-     */
-    void sync_path(String path);
-
-    /**
-     * Allows us to delete the znodes within /storm/blobstore/key_name
-     * whose znodes start with the corresponding nimbusHostPortInfo
-     * @param path /storm/blobstore/key_name
-     * @param nimbusHostPortInfo Contains the host port information of
-     * a nimbus node.
-     */
-    void delete_node_blobstore(String path, String nimbusHostPortInfo);
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
index 9803dff..c2477d6 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StateStorageFactory.java
@@ -23,6 +23,6 @@ import org.apache.zookeeper.data.ACL;
 
 public interface StateStorageFactory {
     
-    StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
+    IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context);
 
 }

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
deleted file mode 100644
index 58b125b..0000000
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterState.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/**
- * 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.APersistentMap;
-import clojure.lang.IFn;
-import org.apache.storm.generated.*;
-import org.apache.storm.nimbus.NimbusInfo;
-
-import java.security.NoSuchAlgorithmException;
-import java.util.List;
-import java.util.Map;
-
-public interface StormClusterState {
-    public List<String> assignments(IFn callback);
-
-    public Assignment assignmentInfo(String stormId, IFn callback);
-
-    public APersistentMap assignmentInfoWithVersion(String stormId, IFn callback);
-
-    public Integer assignmentVersion(String stormId, IFn callback) throws Exception;
-
-    // returns key information under /storm/blobstore/key
-    public List<String> blobstoreInfo(String blobKey);
-
-    // returns list of nimbus summaries stored under /stormroot/nimbuses/<nimbus-ids> -> <data>
-    public List nimbuses();
-
-    // adds the NimbusSummary to /stormroot/nimbuses/nimbus-id
-    public void addNimbusHost(String nimbusId, NimbusSummary nimbusSummary);
-
-    public List<String> activeStorms();
-
-    public StormBase stormBase(String stormId, IFn callback);
-
-    public ClusterWorkerHeartbeat getWorkerHeartbeat(String stormId, String node, Long port);
-
-    public List<ProfileRequest> getWorkerProfileRequests(String stormId, NodeInfo nodeInfo, boolean isThrift);
-
-    public List<ProfileRequest> getTopologyProfileRequests(String stormId, boolean isThrift);
-
-    public void setWorkerProfileRequest(String stormId, ProfileRequest profileRequest);
-
-    public void deleteTopologyProfileRequests(String stormId, ProfileRequest profileRequest);
-
-    public Map<ExecutorInfo, ClusterWorkerHeartbeat> executorBeats(String stormId, Map<List<Long>, NodeInfo> executorNodePort);
-
-    public List<String> supervisors(IFn callback);
-
-    public SupervisorInfo supervisorInfo(String supervisorId); // returns nil if doesn't exist
-
-    public void setupHeatbeats(String stormId);
-
-    public void teardownHeartbeats(String stormId);
-
-    public void teardownTopologyErrors(String stormId);
-
-    public List<String> heartbeatStorms();
-
-    public List<String> errorTopologies();
-
-    public void setTopologyLogConfig(String stormId, LogConfig logConfig);
-
-    public LogConfig topologyLogConfig(String stormId, IFn cb);
-
-    public void workerHeartbeat(String stormId, String node, Long port, ClusterWorkerHeartbeat info);
-
-    public void removeWorkerHeartbeat(String stormId, String node, Long port);
-
-    public void supervisorHeartbeat(String supervisorId, SupervisorInfo info);
-
-    public void workerBackpressure(String stormId, String node, Long port, boolean on);
-
-    public boolean topologyBackpressure(String stormId, IFn callback);
-
-    public void setupBackpressure(String stormId);
-
-    public void removeWorkerBackpressure(String stormId, String node, Long port);
-
-    public void activateStorm(String stormId, StormBase stormBase);
-
-    public void updateStorm(String stormId, StormBase newElems);
-
-    public void removeStormBase(String stormId);
-
-    public void setAssignment(String stormId, Assignment info);
-
-    // sets up information related to key consisting of nimbus
-    // host:port and version info of the blob
-    public void setupBlobstore(String key, NimbusInfo nimbusInfo, Integer versionInfo);
-
-    public List<String> activeKeys();
-
-    public List<String> blobstore(IFn callback);
-
-    public void removeStorm(String stormId);
-
-    public void removeBlobstoreKey(String blobKey);
-
-    public void removeKeyVersion(String blobKey);
-
-    public void reportError(String stormId, String componentId, String node, Long port, String error);
-
-    public List<ErrorInfo> errors(String stormId, String componentId);
-
-    public ErrorInfo lastError(String stormId, String componentId);
-
-    public void setCredentials(String stormId, Credentials creds, Map topoConf) throws NoSuchAlgorithmException;
-
-    public Credentials credentials(String stormId, IFn callback);
-
-    public void disconnect();
-
-}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
index cd2bc4a..8df5885 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/StormClusterStateImpl.java
@@ -39,11 +39,11 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicReference;
 
-public class StormClusterStateImpl implements StormClusterState {
+public class StormClusterStateImpl implements IStormClusterState {
 
     private static Logger LOG = LoggerFactory.getLogger(StormClusterStateImpl.class);
 
-    private StateStorage stateStorage;
+    private IStateStorage stateStorage;
 
     private ConcurrentHashMap<String, IFn> assignmentInfoCallback;
     private ConcurrentHashMap<String, IFn> assignmentInfoWithVersionCallback;
@@ -61,7 +61,7 @@ public class StormClusterStateImpl implements StormClusterState {
     private String stateId;
     private boolean solo;
 
-    public StormClusterStateImpl(StateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
+    public StormClusterStateImpl(IStateStorage StateStorage, List<ACL> acls, ClusterStateContext context, boolean solo) throws Exception {
 
         this.stateStorage = StateStorage;
         this.solo = solo;
@@ -615,7 +615,7 @@ public class StormClusterStateImpl implements StormClusterState {
             }
             Collections.sort(errorInfos, new Comparator<ErrorInfo>() {
                 public int compare(ErrorInfo arg0, ErrorInfo arg1) {
-                    return -Integer.compare(arg0.get_error_time_secs(), arg1.get_error_time_secs());
+                    return Integer.compare(arg1.get_error_time_secs(), arg0.get_error_time_secs());
                 }
             });
         } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
index 8ac0adc..b277751 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorage.java
@@ -41,7 +41,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-public class ZKStateStorage implements StateStorage {
+public class ZKStateStorage implements IStateStorage {
 
     private static Logger LOG = LoggerFactory.getLogger(ZKStateStorage.class);
 
@@ -126,7 +126,7 @@ public class ZKStateStorage implements StateStorage {
 
     @Override
     public void delete_node_blobstore(String path, String nimbusHostPortInfo) {
-
+        Zookeeper.deleteNodeBlobstore(zkWriter, path, nimbusHostPortInfo);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
index 19b04f2..f3b9253 100644
--- a/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
+++ b/storm-core/src/jvm/org/apache/storm/cluster/ZKStateStorageFactory.java
@@ -26,7 +26,7 @@ import java.util.List;
 public class ZKStateStorageFactory implements StateStorageFactory{
 
     @Override
-    public StateStorage mkState(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
+    public IStateStorage mkStore(APersistentMap config, APersistentMap auth_conf, List<ACL> acls, ClusterStateContext context) {
         try {
             return new ZKStateStorage(config, auth_conf, acls, context);
         }catch (Exception e){

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
index 34f3665..af0e8f3 100644
--- a/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
+++ b/storm-core/src/jvm/org/apache/storm/pacemaker/PacemakerClient.java
@@ -157,7 +157,6 @@ public class PacemakerClient implements ISaslClient {
     public String secretKey() {
         return secret;
     }
-
     public HBMessage send(HBMessage m) {
         waitUntilReady();
         LOG.debug("Sending message: {}", m.toString());

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java
----------------------------------------------------------------------
diff --git a/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java
new file mode 100644
index 0000000..0253afb
--- /dev/null
+++ b/storm-core/src/jvm/org/apache/storm/testing/staticmocking/MockedPaceMakerStateStorageFactory.java
@@ -0,0 +1,32 @@
+/**
+ * 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.testing.staticmocking;
+
+
+import org.apache.storm.cluster.PaceMakerStateStorageFactory;
+
+public class MockedPaceMakerStateStorageFactory implements AutoCloseable{
+
+    public MockedPaceMakerStateStorageFactory(PaceMakerStateStorageFactory inst) {
+        PaceMakerStateStorageFactory.setInstance(inst);
+    }
+
+    @Override
+    public void close() throws Exception {
+        PaceMakerStateStorageFactory.resetInstance();
+    }
+}

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/test/clj/org/apache/storm/cluster_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/cluster_test.clj b/storm-core/test/clj/org/apache/storm/cluster_test.clj
index fa34355..39adb9e 100644
--- a/storm-core/test/clj/org/apache/storm/cluster_test.clj
+++ b/storm-core/test/clj/org/apache/storm/cluster_test.clj
@@ -23,7 +23,7 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [org.apache.curator.framework CuratorFramework CuratorFrameworkFactory CuratorFrameworkFactory$Builder])
   (:import [org.apache.storm.utils Utils TestUtils ZookeeperAuthInfo ConfigUtils])
-  (:import [org.apache.storm.cluster StateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
+  (:import [org.apache.storm.cluster IStateStorage ZKStateStorage ClusterStateContext StormClusterStateImpl ClusterUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:import [org.apache.storm.callback ZKStateChangedCallback])
   (:import [org.apache.storm.testing.staticmocking MockedZookeeper MockedCluster])
@@ -39,7 +39,7 @@
 
 (defn mk-state
   ([zk-port] (let [conf (mk-config zk-port)]
-               (ClusterUtils/mkDistributedClusterState conf conf nil (ClusterStateContext.))))
+               (ClusterUtils/mkStateStorage conf conf nil (ClusterStateContext.))))
   ([zk-port cb]
     (let [ret (mk-state zk-port)]
       (.register ret cb)
@@ -318,12 +318,12 @@
       ;; No need for when clauses because we just want to return nil
       (with-open [_ (MockedZookeeper. zk-mock)]
         (. (Mockito/when (.mkClientImpl zk-mock (Mockito/anyMap) (Mockito/anyList) (Mockito/any) (Mockito/anyString) (Mockito/any) (Mockito/anyMap))) (thenReturn curator-frameworke))
-        (ClusterUtils/mkDistributedClusterState {} nil nil (ClusterStateContext.))
+        (ClusterUtils/mkStateStorage {} nil nil (ClusterStateContext.))
         (.mkdirsImpl (Mockito/verify zk-mock (Mockito/times 1)) (Mockito/any) (Mockito/anyString) (Mockito/eq nil))))
-    (let [distributed-state-storage (reify StateStorage
+    (let [distributed-state-storage (reify IStateStorage
                                       (register [this callback] nil)
                                       (mkdirs [this path acls] nil))
           cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [mocked-cluster (MockedCluster. cluster-utils)]
-        (. (Mockito/when (.mkDistributedClusterStateImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
+        (. (Mockito/when (mkStateStorageImpl cluster-utils (Mockito/any) (Mockito/any) (Mockito/eq nil) (Mockito/any))) (thenReturn distributed-state-storage))
         (ClusterUtils/mkStormClusterState {} nil (ClusterStateContext.))))))
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/storm/blob/2ee8bec8/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
index 0925237..1a7bd2c 100644
--- a/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
+++ b/storm-core/test/clj/org/apache/storm/pacemaker_state_factory_test.clj
@@ -15,13 +15,14 @@
 ;; limitations under the License.
 (ns org.apache.storm.pacemaker-state-factory-test
   (:require [clojure.test :refer :all]
-            [conjure.core :refer :all]
-            [org.apache.storm.pacemaker [pacemaker-state-factory :as psf]])
+            [conjure.core :refer :all])
   (:import [org.apache.storm.generated
             HBExecutionException HBNodes HBRecords
             HBServerMessageType HBMessage HBMessageData HBPulse]
-           [org.apache.storm.cluster ClusterStateContext]
-           [org.mockito Mockito Matchers]))
+           [org.apache.storm.cluster ClusterStateContext  PaceMakerStateStorageFactory]
+           [org.mockito Mockito Matchers])
+(:import [org.mockito.exceptions.base MockitoAssertionError])
+(:import [org.apache.storm.testing.staticmocking MockedPaceMakerStateStorageFactory]))
 
 (defn- string-to-bytes [string]
   (byte-array (map int string)))
@@ -39,18 +40,20 @@
       (send [this something] (reset! captured something) response)
       (check-captured [this] @captured))))
 
-(defmacro with-mock-pacemaker-client-and-state [client state response & body]
-  `(let [~client (make-send-capture ~response)]
-     (stubbing [psf/makeZKState nil
-                psf/makeClient ~client]
-               (let [~state (psf/-mkState nil nil nil nil (ClusterStateContext.))]
-                 ~@body))))
+(defmacro with-mock-pacemaker-client-and-state [client state pacefactory mock response & body]
+  `(let [~client (make-send-capture ~response)
+         ~pacefactory (Mockito/mock PaceMakerStateStorageFactory)]
 
+     (with-open [~mock (MockedPaceMakerStateStorageFactory. ~pacefactory)]
+       (. (Mockito/when (.initZKstateImpl ~pacefactory (Mockito/any) (Mockito/any) (Mockito/anyList) (Mockito/any))) (thenReturn nil))
+       (. (Mockito/when (.initMakeClientImpl ~pacefactory (Mockito/any))) (thenReturn ~client))
+               (let [~state (.mkStore ~pacefactory nil nil nil (ClusterStateContext.))]
+                 ~@body))))
 
 (deftest pacemaker_state_set_worker_hb
   (testing "set_worker_hb"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/SEND_PULSE_RESPONSE nil)
 
       (.set_worker_hb state "/foo" (string-to-bytes "data") nil)
@@ -62,10 +65,10 @@
 
   (testing "set_worker_hb"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/SEND_PULSE nil)
 
-      (is (thrown? HBExecutionException      
+      (is (thrown? RuntimeException      
                    (.set_worker_hb state "/foo" (string-to-bytes "data") nil))))))
 
       
@@ -73,7 +76,7 @@
 (deftest pacemaker_state_delete_worker_hb
   (testing "delete_worker_hb"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/DELETE_PATH_RESPONSE nil)
 
       (.delete_worker_hb state "/foo/bar")
@@ -83,16 +86,16 @@
 
     (testing "delete_worker_hb"
       (with-mock-pacemaker-client-and-state
-        client state
+        client state pacefactory mock
         (HBMessage. HBServerMessageType/DELETE_PATH nil)
         
-        (is (thrown? HBExecutionException
+        (is (thrown? RuntimeException
                      (.delete_worker_hb state "/foo/bar"))))))
 
 (deftest pacemaker_state_get_worker_hb
   (testing "get_worker_hb"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE
                 (HBMessageData/pulse
                  (doto (HBPulse.)
@@ -106,24 +109,24 @@
 
   (testing "get_worker_hb - fail (bad response)"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/GET_PULSE nil)
       
-      (is (thrown? HBExecutionException
+      (is (thrown? RuntimeException
                    (.get_worker_hb state "/foo" false)))))
   
   (testing "get_worker_hb - fail (bad data)"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/GET_PULSE_RESPONSE nil)
       
-      (is (thrown? HBExecutionException
+      (is (thrown? RuntimeException
                    (.get_worker_hb state "/foo" false))))))
 
 (deftest pacemaker_state_get_worker_hb_children
   (testing "get_worker_hb_children"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE
                 (HBMessageData/nodes
                  (HBNodes. [])))
@@ -135,16 +138,16 @@
 
   (testing "get_worker_hb_children - fail (bad response)"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/DELETE_PATH nil)
 
-      (is (thrown? HBExecutionException
+      (is (thrown? RuntimeException
                    (.get_worker_hb_children state "/foo" false)))))
 
     (testing "get_worker_hb_children - fail (bad data)"
     (with-mock-pacemaker-client-and-state
-      client state
+      client state pacefactory mock
       (HBMessage. HBServerMessageType/GET_ALL_NODES_FOR_PATH_RESPONSE nil)
-      
-      (is (thrown? HBExecutionException
+      ;need been update due to HBExecutionException
+      (is (thrown? RuntimeException
                    (.get_worker_hb_children state "/foo" false))))))


[07/27] storm git commit: update class hierarchy about cluster

Posted by bo...@apache.org.
http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/nimbus_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/nimbus_test.clj b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
index d4402fb..772a232 100644
--- a/storm-core/test/clj/org/apache/storm/nimbus_test.clj
+++ b/storm-core/test/clj/org/apache/storm/nimbus_test.clj
@@ -36,7 +36,7 @@
   (:import [org.apache.storm.utils Time Utils ConfigUtils])
   (:import [org.apache.storm.zookeeper Zookeeper])
   (:import [org.apache.commons.io FileUtils])
-  (:import [org.apache.storm.cluster StormZkClusterState ClusterStateContext Cluster])
+  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
   (:use [org.apache.storm testing MockAutoCred util config log timer converter])
   (:use [org.apache.storm.daemon common])
   (:require [conjure.core])
@@ -126,7 +126,7 @@
   (let [state (:storm-cluster-state cluster)
         executor->node+port (:executor->node+port (clojurify-assignment (.assignmentInfo state storm-id nil)))
         [node port] (get executor->node+port executor)
-        curr-beat (clojurify-zk-worker-hb (.getworkerHeartbeat state storm-id node port))
+        curr-beat (clojurify-zk-worker-hb (.getWorkerHeartbeat state storm-id node port))
         stats (:executor-stats curr-beat)]
     (.workerHeartbeat state storm-id node port
       (thriftify-zk-worker-hb {:storm-id storm-id :time-secs (current-time-secs) :uptime 10 :executor-stats (merge stats {executor (stats/render-stats! (stats/mk-bolt-stats 20))})})
@@ -1031,7 +1031,7 @@
                             STORM-CLUSTER-MODE "local"
                             STORM-ZOOKEEPER-PORT zk-port
                             STORM-LOCAL-DIR nimbus-dir}))
-         (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+         (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
          (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
          (bind topology (thrift/mk-topology
                          {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1103,7 +1103,7 @@
                         STORM-CLUSTER-MODE "local"
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir}))
-          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind topology (thrift/mk-topology
                            {"1" (thrift/mk-spout-spec (TestPlannerSpout. true) :parallelism-hint 3)}
@@ -1113,7 +1113,7 @@
                           (zkLeaderElectorImpl [conf] (mock-leader-elector :is-leader false))))]
 
             (letlocals
-              (bind non-leader-cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+              (bind non-leader-cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
               (bind non-leader-nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
 
               ;first we verify that the master nimbus can perform all actions, even with another nimbus present.
@@ -1309,7 +1309,7 @@
                         :status {:type bogus-type}}
                 }
         ]
-      (stubbing [topology-bases bogus-bases
+      (stubbing [nimbus/nimbus-topology-bases bogus-bases
                  nimbus/get-blob-replication-count 1]
         (let [topos (.get_topologies (.getClusterInfo nimbus))]
           ; The number of topologies in the summary is correct.
@@ -1350,14 +1350,13 @@
                      NIMBUS-THRIFT-PORT 6666})
           expected-acls nimbus/NIMBUS-ZK-ACLS
           fake-inimbus (reify INimbus (getForcedScheduler [this] nil))
-          storm-zk (Mockito/mock Cluster)]
+          cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (nimbusTopoHistoryStateImpl [conf] nil))
                   zk-le (MockedZookeeper. (proxy [Zookeeper] []
                           (zkLeaderElectorImpl [conf] nil)))
-                  storm-zk-le (MockedCluster. storm-zk)]
+                  mocked-cluster (MockedCluster. cluster-utils)]
         (stubbing [mk-authorization-handler nil
-               ;  cluster/mk-storm-cluster-state nil
                  nimbus/file-cache-map nil
                  nimbus/mk-blob-cache-map nil
                  nimbus/mk-bloblist-cache-map nil
@@ -1366,10 +1365,7 @@
                  mk-timer nil
                  nimbus/mk-scheduler nil]
           (nimbus/nimbus-data auth-conf fake-inimbus)
-          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
-          ;   (verify-call-times-for cluster/mk-storm-cluster-state 1)
-       ; (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
-       ; expected-acls)
+          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
           )))))
 
 (deftest test-file-bogus-download
@@ -1401,7 +1397,7 @@
                       STORM-CLUSTER-MODE "local"
                       STORM-ZOOKEEPER-PORT zk-port
                       STORM-LOCAL-DIR nimbus-dir}))
-        (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+        (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
         (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
         (sleep-secs 1)
         (bind topology (thrift/mk-topology
@@ -1433,7 +1429,7 @@
                         STORM-ZOOKEEPER-PORT zk-port
                         STORM-LOCAL-DIR nimbus-dir
                         NIMBUS-TOPOLOGY-ACTION-NOTIFIER-PLUGIN (.getName InMemoryTopologyActionNotifier)}))
-          (bind cluster-state (StormZkClusterState. conf nil (ClusterStateContext.)))
+          (bind cluster-state (ClusterUtils/mkStormClusterState conf nil (ClusterStateContext.)))
           (bind nimbus (nimbus/service-handler conf (nimbus/standalone-nimbus)))
           (bind notifier (InMemoryTopologyActionNotifier.))
           (sleep-secs 1)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/clj/org/apache/storm/supervisor_test.clj
----------------------------------------------------------------------
diff --git a/storm-core/test/clj/org/apache/storm/supervisor_test.clj b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
index c98a68b..b89b7bb 100644
--- a/storm-core/test/clj/org/apache/storm/supervisor_test.clj
+++ b/storm-core/test/clj/org/apache/storm/supervisor_test.clj
@@ -29,7 +29,7 @@
   (:import [org.mockito.exceptions.base MockitoAssertionError])
   (:import [java.io File])
   (:import [java.nio.file Files])
-  (:import [org.apache.storm.cluster StormZkClusterState Cluster ClusterStateContext])
+  (:import [org.apache.storm.cluster StormClusterStateImpl ClusterStateContext ClusterUtils])
   (:import [java.nio.file.attribute FileAttribute])
   (:use [org.apache.storm config testing util timer log converter])
   (:use [org.apache.storm.daemon common])
@@ -565,17 +565,17 @@
           fake-isupervisor (reify ISupervisor
                              (getSupervisorId [this] nil)
                              (getAssignmentId [this] nil))
-          storm-zk (Mockito/mock Cluster)]
+          cluster-utils (Mockito/mock ClusterUtils)]
       (with-open [_ (proxy [MockedConfigUtils] []
                       (supervisorStateImpl [conf] nil)
                       (supervisorLocalDirImpl [conf] nil))
-                  storm-zk-le (MockedCluster. storm-zk)]
+                  mocked-cluster (MockedCluster. cluster-utils)]
         (stubbing [uptime-computer nil
               ;   cluster/mk-storm-cluster-state nil
                  local-hostname nil
                  mk-timer nil]
           (supervisor/supervisor-data auth-conf nil fake-isupervisor)
-          (.mkStormClusterStateImpl (Mockito/verify storm-zk (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
+          (.mkStormClusterStateImpl (Mockito/verify cluster-utils (Mockito/times 1)) (Mockito/any) (Mockito/eq expected-acls) (Mockito/any))
         ;  (verify-call-times-for cluster/mk-storm-cluster-state 1)
         ;  (verify-first-call-args-for-indices cluster/mk-storm-cluster-state [2]
         ;                                     expected-acls)

http://git-wip-us.apache.org/repos/asf/storm/blob/55b86ca4/storm-core/test/jvm/org/apache/storm/ClusterTest.java
----------------------------------------------------------------------
diff --git a/storm-core/test/jvm/org/apache/storm/ClusterTest.java b/storm-core/test/jvm/org/apache/storm/ClusterTest.java
deleted file mode 100644
index ef43afe..0000000
--- a/storm-core/test/jvm/org/apache/storm/ClusterTest.java
+++ /dev/null
@@ -1,22 +0,0 @@
-/**
- * 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;
-
-
-public class ClusterTest {
-}


[27/27] storm git commit: Added STORM-1273 to Changelog

Posted by bo...@apache.org.
Added STORM-1273 to Changelog


Project: http://git-wip-us.apache.org/repos/asf/storm/repo
Commit: http://git-wip-us.apache.org/repos/asf/storm/commit/71d615b7
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/71d615b7
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/71d615b7

Branch: refs/heads/master
Commit: 71d615b7cc9a96b6667b976a25dc86ef54a66169
Parents: 1b5d9a8
Author: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Authored: Wed Feb 24 09:46:03 2016 -0600
Committer: Robert (Bobby) Evans <ev...@yahoo-inc.com>
Committed: Wed Feb 24 09:46:03 2016 -0600

----------------------------------------------------------------------
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/71d615b7/CHANGELOG.md
----------------------------------------------------------------------
diff --git a/CHANGELOG.md b/CHANGELOG.md
index feef9a3..cc11139 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -1,4 +1,5 @@
 ## 2.0.0
+ * STORM-1273: port backtype.storm.cluster to java
  * STORM-1479: use a simple implemention for IntSerializer
  * STORM-1255: port storm_utils.clj to java and split Time tests into its
  * STORM-1566: Worker exits with error o.a.s.d.worker [ERROR] Error on initialization of server mk-worker