[
https://issues.apache.org/jira/browse/STORM-1520?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15132018#comment-15132018
]
ASF GitHub Bot commented on STORM-1520:
---------------------------------------
GitHub user HeartSaVioR opened a pull request:
https://github.com/apache/storm/pull/1077
STORM-1520 [for master branch] Nimbus Clojure/Zookeeper issue
("stateChanged" method not found)
Please take a look at the mapping between Curator ConnectionState and Storm
ConnectionState.
When pull request is merged, I'll craft the pull request which can be
applied to 1.x-branch.
(It would be nearly same to this pull request but I've seen small conflict
since we ported ZK to Java.)
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/HeartSaVioR/storm STORM-1520
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/storm/pull/1077.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #1077
----
commit 25f8b2af7ac007ca4da304e3e25813a4f0079af4
Author: Jungtaek Lim <[email protected]>
Date: 2016-02-04T09:17:15Z
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
----
> Nimbus Clojure/Zookeeper issue ("stateChanged" method not found)
> ----------------------------------------------------------------
>
> Key: STORM-1520
> URL: https://issues.apache.org/jira/browse/STORM-1520
> Project: Apache Storm
> Issue Type: Bug
> Components: storm-core
> Affects Versions: 1.0.0
> Reporter: P. Taylor Goetz
> Assignee: Jungtaek Lim
> Priority: Blocker
>
> Placeholder until I can gather more information for reproducing the issue.
> The following appears in nimbus.log after deploying/undeploying topologies:
> {code}
> 2016-02-02 21:34:04.308 o.a.s.s.o.a.c.f.l.ListenerContainer [ERROR] Listener
> (org.apache.storm.cluster_state.zookeeper_state_factory$_mkState$reify$reify__12660@22587507)
> threw an exception
> java.lang.IllegalArgumentException: No matching method found: stateChanged
> for class org.apache.storm.cluster$mk_storm_cluster_state$reify$reify__6413
> at clojure.lang.Reflector.invokeMatchingMethod(Reflector.java:53)
> at clojure.lang.Reflector.invokeInstanceMethod(Reflector.java:28)
> at
> org.apache.storm.cluster_state.zookeeper_state_factory$_mkState$reify$reify__12660.stateChanged(zookeeper_state_factory.clj:145)
> at
> org.apache.storm.shade.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:259)
> at
> org.apache.storm.shade.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:255)
> at
> org.apache.storm.shade.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:92)
> at
> org.apache.storm.shade.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at
> org.apache.storm.shade.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:84)
> at
> org.apache.storm.shade.org.apache.curator.framework.state.ConnectionStateManager.processEvents(ConnectionStateManager.java:253)
> at
> org.apache.storm.shade.org.apache.curator.framework.state.ConnectionStateManager.access$000(ConnectionStateManager.java:43)
> at
> org.apache.storm.shade.org.apache.curator.framework.state.ConnectionStateManager$1.call(ConnectionStateManager.java:111)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {code}
> -Basic functionality does not seem to be affected.-
> Nimbus becomes unresponsive and needs to be manually restarted.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)