From 0656e8ac444673abc791121aeac8956d1d87120f Mon Sep 17 00:00:00 2001 From: Noble Paul Date: Thu, 26 Mar 2015 00:33:49 +0000 Subject: [PATCH] SOLR-6924: removing double listening of nodes git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1669245 13f79535-47bb-0310-9956-ffa450edef68 --- solr/core/src/java/org/apache/solr/cloud/ZkController.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java index d4fa58ea917..15f1118770a 100644 --- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java +++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java @@ -2295,9 +2295,9 @@ public final class ZkController { private final Map> confDirectoryListeners = new HashMap<>(); void watchZKConfDir(final String zkDir) { - log.info("watch zkdir " + zkDir); + log.info("watch zkdir {}" , zkDir); if (!confDirectoryListeners.containsKey(zkDir)) { - confDirectoryListeners.put(zkDir, new HashSet()); + confDirectoryListeners.put(zkDir, new HashSet<>()); setConfWatcher(zkDir, new WatcherImpl(zkDir), null); } } @@ -2348,6 +2348,7 @@ public final class ZkController { new Thread() { //run these in a separate thread because this can be long running public void run() { + log.info("Running listeners for {}", zkDir); for (final Runnable listener : listenersCopy) { try { listener.run(); @@ -2371,7 +2372,6 @@ public final class ZkController { //so fire the event listeners fireEventListeners(zkDir); } - zkClient.exists(zkDir, watcher, true); } catch (KeeperException e) { log.error("failed to set watcher for conf dir {} ", zkDir); } catch (InterruptedException e) {