[
https://issues.apache.org/jira/browse/STORM-1257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15119429#comment-15119429
]
ASF GitHub Bot commented on STORM-1257:
---------------------------------------
Github user revans2 commented on a diff in the pull request:
https://github.com/apache/storm/pull/1047#discussion_r50997086
--- Diff: storm-core/src/clj/org/apache/storm/zookeeper.clj ---
@@ -57,15 +46,15 @@
: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 (zk-keeper-states (.getState event))
- (zk-event-types (.getType event))
- (.getPath event))))))))
+ (getCuratorListenable)
+ (addListener
+ (reify CuratorListener
+ (^void eventReceived [this ^CuratorFramework _fk ^CuratorEvent e]
+ (when (= (.getType e) CuratorEventType/WATCHED)
+ (let [^WatchedEvent event (.getWatchedEvent e)]
+ (watcher (ZkKeeperStates/getStateName (.getState event))
+ (ZkEventTypes/getTypeName (.getType event))
--- End diff --
These are returning Strings, not symbols. We probably want to just return
the enum itself. Also anyone who is using those symbols needs to be updated.
When I run the tests I am getting NPEs because of this.
It looks like you will mostly need to update
./storm-core/src/clj/org/apache/storm/cluster_state/zookeeper_state_factory.clj
and
./storm-core/test/clj/org/apache/storm/cluster_test.clj
> port backtype.storm.zookeeper to java
> -------------------------------------
>
> Key: STORM-1257
> URL: https://issues.apache.org/jira/browse/STORM-1257
> Project: Apache Storm
> Issue Type: New Feature
> Components: storm-core
> Reporter: Robert Joseph Evans
> Assignee: John Fang
> Labels: java-migration, jstorm-merger
>
> A wrapper around zookeeper/curator.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)