Repository: storm
Updated Branches:
  refs/heads/master b89af960a -> 53e44ff3d


STORM-1520 Nimbus Clojure/Zookeeper issue ("stateChanged" method not found)

* fix a bug which passes wrong type of parameter to 
ClusterStateListener.stateChanged()
  * we passed ConnectionState from Curator which method needs storm's 
ConnectionState


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

Branch: refs/heads/master
Commit: 25f8b2af7ac007ca4da304e3e25813a4f0079af4
Parents: 695f8c9
Author: Jungtaek Lim <[email protected]>
Authored: Thu Feb 4 18:17:15 2016 +0900
Committer: Jungtaek Lim <[email protected]>
Committed: Thu Feb 4 18:17:15 2016 +0900

----------------------------------------------------------------------
 .../cluster_state/zookeeper_state_factory.clj   |  3 ++-
 .../utils/StormConnectionStateConverter.java    | 26 ++++++++++++++++++++
 2 files changed, 28 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/storm/blob/25f8b2af/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
index dcfa8d8..624d75c 100644
--- 
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
@@ -20,6 +20,7 @@
   (:import [org.apache.zookeeper KeeperException$NoNodeException CreateMode
              Watcher$Event$EventType Watcher$Event$KeeperState]
            [org.apache.storm.cluster ClusterState DaemonType])
+  (:import [org.apache.storm.utils StormConnectionStateConverter])
   (:use [org.apache.storm cluster config log util])
   (:require [org.apache.storm [zookeeper :as zk]])
   (:gen-class
@@ -144,7 +145,7 @@
        (let [curator-listener (reify ConnectionStateListener
                                 (stateChanged
                                   [this client newState]
-                                  (.stateChanged listener client newState)))]
+                                  (.stateChanged listener 
(StormConnectionStateConverter/convert newState))))]
          (Zookeeper/addListener zk-reader curator-listener)))
 
      (sync-path

http://git-wip-us.apache.org/repos/asf/storm/blob/25f8b2af/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java
----------------------------------------------------------------------
diff --git 
a/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java 
b/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java
new file mode 100644
index 0000000..0374756
--- /dev/null
+++ 
b/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.java
@@ -0,0 +1,26 @@
+package org.apache.storm.utils;
+
+import org.apache.storm.cluster.ConnectionState;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class StormConnectionStateConverter {
+
+    private static final 
Map<org.apache.curator.framework.state.ConnectionState, ConnectionState> 
mapCuratorToStorm = new HashMap<>();
+    static {
+        
mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.CONNECTED,
 ConnectionState.CONNECTED);
+        
mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.LOST, 
ConnectionState.LOST);
+        
mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.RECONNECTED,
 ConnectionState.RECONNECTED);
+        
mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.READ_ONLY,
 ConnectionState.LOST);
+        
mapCuratorToStorm.put(org.apache.curator.framework.state.ConnectionState.SUSPENDED,
 ConnectionState.LOST);
+    }
+
+    public static ConnectionState 
convert(org.apache.curator.framework.state.ConnectionState state) {
+        ConnectionState stormState = mapCuratorToStorm.get(state);
+        if (stormState != null) {
+            return stormState;
+        }
+        throw new IllegalStateException("Unknown ConnectionState from Curator: 
" + state);
+    }
+}

Reply via email to