Repository: storm
Updated Branches:
  refs/heads/1.x-branch 0164e522e -> ec25fad1b


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/ed4a55f5
Tree: http://git-wip-us.apache.org/repos/asf/storm/tree/ed4a55f5
Diff: http://git-wip-us.apache.org/repos/asf/storm/diff/ed4a55f5

Branch: refs/heads/1.x-branch
Commit: ed4a55f50f5e73589e2a59c99236651edf0f18b1
Parents: 0164e52
Author: Jungtaek Lim <[email protected]>
Authored: Tue Feb 9 11:26:06 2016 +0900
Committer: Jungtaek Lim <[email protected]>
Committed: Tue Feb 9 11:26:06 2016 +0900

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


http://git-wip-us.apache.org/repos/asf/storm/blob/ed4a55f5/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 3104c52..7eca328 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
@@ -18,6 +18,7 @@
   (:import [org.apache.curator.framework.state ConnectionStateListener])
   (:import [org.apache.zookeeper KeeperException$NoNodeException]
            [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
@@ -142,7 +143,7 @@
        (let [curator-listener (reify ConnectionStateListener
                                 (stateChanged
                                   [this client newState]
-                                  (.stateChanged listener client newState)))]
+                                  (.stateChanged listener 
(StormConnectionStateConverter/convert newState))))]
          (zk/add-listener zk-reader curator-listener)))
 
      (sync-path

http://git-wip-us.apache.org/repos/asf/storm/blob/ed4a55f5/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..587b5eb
--- /dev/null
+++ 
b/storm-core/src/jvm/org/apache/storm/utils/StormConnectionStateConverter.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.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);
+    }
+}
\ No newline at end of file

Reply via email to