You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2016/02/19 20:47:20 UTC
[1/2] lucene-solr git commit: SOLR-8694: DistributedMap/Queue can
create too many Watchers and some code simplification.
Repository: lucene-solr
Updated Branches:
refs/heads/master 189e985b5 -> e30d638c5
SOLR-8694: DistributedMap/Queue can create too many Watchers and some code simplification.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/32fbca6e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/32fbca6e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/32fbca6e
Branch: refs/heads/master
Commit: 32fbca6ea7b65043041e622660e07915f04090fe
Parents: 189e985
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 19 14:33:50 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Fri Feb 19 14:33:50 2016 -0500
----------------------------------------------------------------------
solr/CHANGES.txt | 3 +
.../org/apache/solr/cloud/DistributedMap.java | 152 +------------------
.../org/apache/solr/cloud/DistributedQueue.java | 6 +-
.../java/org/apache/solr/cloud/Overseer.java | 6 +-
.../solr/cloud/SizeLimitedDistributedMap.java | 29 ++--
.../solr/handler/admin/CollectionsHandler.java | 8 +-
6 files changed, 34 insertions(+), 170 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 953b2d4..c0c8856 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -182,6 +182,9 @@ Bug Fixes
* SOLR-8701: CloudSolrClient decides that there are no healthy nodes to handle a request too early.
(Mark Miller)
+* SOLR-8694: DistributedMap/Queue can create too many Watchers and some code simplification.
+ (Scott Blum via Mark Miller)
+
Optimizations
----------------------
* SOLR-7876: Speed up queries and operations that use many terms when timeAllowed has not been
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
index 8434eb8..c3b5690 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DistributedMap.java
@@ -22,15 +22,9 @@ import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkCmdExecutor;
import org.apache.zookeeper.CreateMode;
import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooDefs;
import org.apache.zookeeper.data.ACL;
import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import java.lang.invoke.MethodHandles;
import java.util.List;
/**
@@ -39,19 +33,13 @@ import java.util.List;
* don't have to be ordered i.e. DistributedQueue.
*/
public class DistributedMap {
- private static final Logger LOG = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-
- protected static long DEFAULT_TIMEOUT = 5*60*1000;
-
protected final String dir;
protected SolrZkClient zookeeper;
protected final String prefix = "mn-";
- protected final String response_prefix = "mnr-" ;
-
- public DistributedMap(SolrZkClient zookeeper, String dir, List<ACL> acl) {
+ public DistributedMap(SolrZkClient zookeeper, String dir) {
this.dir = dir;
ZkCmdExecutor cmdExecutor = new ZkCmdExecutor(zookeeper.getZkClientTimeout());
@@ -67,89 +55,13 @@ public class DistributedMap {
this.zookeeper = zookeeper;
}
- protected class LatchChildWatcher implements Watcher {
-
- Object lock = new Object();
- private WatchedEvent event = null;
-
- public LatchChildWatcher() {}
-
- public LatchChildWatcher(Object lock) {
- this.lock = lock;
- }
-
- @Override
- public void process(WatchedEvent event) {
- LOG.info("LatchChildWatcher fired on path: " + event.getPath() + " state: "
- + event.getState() + " type " + event.getType());
- synchronized (lock) {
- this.event = event;
- lock.notifyAll();
- }
- }
-
- public void await(long timeout) throws InterruptedException {
- synchronized (lock) {
- lock.wait(timeout);
- }
- }
-
- public WatchedEvent getWatchedEvent() {
- return event;
- }
- }
-
- /**
- * Inserts data into zookeeper.
- *
- * @return true if data was successfully added
- */
- protected String createData(String path, byte[] data, CreateMode mode)
- throws KeeperException, InterruptedException {
- for (;;) {
- try {
- return zookeeper.create(path, data, mode, true);
- } catch (KeeperException.NoNodeException e) {
- try {
- zookeeper.create(dir, new byte[0], CreateMode.PERSISTENT, true);
- } catch (KeeperException.NodeExistsException ne) {
- // someone created it
- }
- }
- }
- }
-
-
- public boolean put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
- return createData(dir + "/" + prefix + trackingId, data,
- CreateMode.PERSISTENT) != null;
- }
- /**
- * Offer the data and wait for the response
- *
- */
- public MapEvent put(String trackingId, byte[] data, long timeout) throws KeeperException,
- InterruptedException {
- String path = createData(dir + "/" + prefix + trackingId, data,
- CreateMode.PERSISTENT);
- String watchID = createData(
- dir + "/" + response_prefix + path.substring(path.lastIndexOf("-") + 1),
- null, CreateMode.EPHEMERAL);
- Object lock = new Object();
- LatchChildWatcher watcher = new LatchChildWatcher(lock);
- synchronized (lock) {
- if (zookeeper.exists(watchID, watcher, true) != null) {
- watcher.await(timeout);
- }
- }
- byte[] bytes = zookeeper.getData(watchID, null, null, true);
- zookeeper.delete(watchID, -1, true);
- return new MapEvent(watchID, bytes, watcher.getWatchedEvent());
+ public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
+ zookeeper.makePath(dir + "/" + prefix + trackingId, data, CreateMode.PERSISTENT, null, false, true);
}
- public MapEvent get(String trackingId) throws KeeperException, InterruptedException {
- return new MapEvent(trackingId, zookeeper.getData(dir + "/" + prefix + trackingId, null, null, true), null);
+ public byte[] get(String trackingId) throws KeeperException, InterruptedException {
+ return zookeeper.getData(dir + "/" + prefix + trackingId, null, null, true);
}
public boolean contains(String trackingId) throws KeeperException, InterruptedException {
@@ -187,58 +99,4 @@ public class DistributedMap {
}
- public static class MapEvent {
- @Override
- public int hashCode() {
- final int prime = 31;
- int result = 1;
- result = prime * result + ((id == null) ? 0 : id.hashCode());
- return result;
- }
-
- @Override
- public boolean equals(Object obj) {
- if (this == obj) return true;
- if (obj == null) return false;
- if (getClass() != obj.getClass()) return false;
- MapEvent other = (MapEvent) obj;
- if (id == null) {
- if (other.id != null) return false;
- } else if (!id.equals(other.id)) return false;
- return true;
- }
-
- private WatchedEvent event = null;
- private String id;
- private byte[] bytes;
-
- MapEvent(String id, byte[] bytes, WatchedEvent event) {
- this.id = id;
- this.bytes = bytes;
- this.event = event;
- }
-
- public void setId(String id) {
- this.id = id;
- }
-
- public String getId() {
- return id;
- }
-
- public void setBytes(byte[] bytes) {
- this.bytes = bytes;
- }
-
- public byte[] getBytes() {
- return bytes;
- }
-
- public WatchedEvent getWatchedEvent() {
- return event;
- }
-
- }
-
-
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java b/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
index 87d92c6..e424b7e 100644
--- a/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
@@ -318,7 +318,7 @@ public class DistributedQueue {
}
return orderedChildren;
} catch (KeeperException.NoNodeException e) {
- zookeeper.create(dir, new byte[0], CreateMode.PERSISTENT, true);
+ zookeeper.makePath(dir, false, true);
// go back to the loop and try again
}
}
@@ -408,6 +408,10 @@ public class DistributedQueue {
@Override
public void process(WatchedEvent event) {
+ // session events are not change events, and do not remove the watcher; except for Expired
+ if (Event.EventType.None.equals(event.getType()) && !Event.KeeperState.Expired.equals(event.getState())) {
+ return;
+ }
updateLock.lock();
try {
// this watcher is automatically cleared when fired
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/core/src/java/org/apache/solr/cloud/Overseer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/Overseer.java b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
index 3fe2e5c..8dfacb1 100644
--- a/solr/core/src/java/org/apache/solr/cloud/Overseer.java
+++ b/solr/core/src/java/org/apache/solr/cloud/Overseer.java
@@ -912,19 +912,19 @@ public class Overseer implements Closeable {
/* Internal map for failed tasks, not to be used outside of the Overseer */
static DistributedMap getRunningMap(final SolrZkClient zkClient) {
createOverseerNode(zkClient);
- return new DistributedMap(zkClient, "/overseer/collection-map-running", null);
+ return new DistributedMap(zkClient, "/overseer/collection-map-running");
}
/* Size-limited map for successfully completed tasks*/
static DistributedMap getCompletedMap(final SolrZkClient zkClient) {
createOverseerNode(zkClient);
- return new SizeLimitedDistributedMap(zkClient, "/overseer/collection-map-completed", null, NUM_RESPONSES_TO_STORE);
+ return new SizeLimitedDistributedMap(zkClient, "/overseer/collection-map-completed", NUM_RESPONSES_TO_STORE);
}
/* Map for failed tasks, not to be used outside of the Overseer */
static DistributedMap getFailureMap(final SolrZkClient zkClient) {
createOverseerNode(zkClient);
- return new SizeLimitedDistributedMap(zkClient, "/overseer/collection-map-failure", null, NUM_RESPONSES_TO_STORE);
+ return new SizeLimitedDistributedMap(zkClient, "/overseer/collection-map-failure", NUM_RESPONSES_TO_STORE);
}
/* Collection creation queue */
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/core/src/java/org/apache/solr/cloud/SizeLimitedDistributedMap.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/SizeLimitedDistributedMap.java b/solr/core/src/java/org/apache/solr/cloud/SizeLimitedDistributedMap.java
index 418eb66..3326dca 100644
--- a/solr/core/src/java/org/apache/solr/cloud/SizeLimitedDistributedMap.java
+++ b/solr/core/src/java/org/apache/solr/cloud/SizeLimitedDistributedMap.java
@@ -33,41 +33,40 @@ public class SizeLimitedDistributedMap extends DistributedMap {
private final int maxSize;
- public SizeLimitedDistributedMap(SolrZkClient zookeeper, String dir, List<ACL> acl, int maxSize) {
- super(zookeeper, dir, acl);
+ public SizeLimitedDistributedMap(SolrZkClient zookeeper, String dir, int maxSize) {
+ super(zookeeper, dir);
this.maxSize = maxSize;
}
-
+
@Override
- public boolean put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
- if(this.size() >= maxSize) {
+ public void put(String trackingId, byte[] data) throws KeeperException, InterruptedException {
+ if (this.size() >= maxSize) {
// Bring down the size
List<String> children = zookeeper.getChildren(dir, null, true);
int cleanupSize = maxSize / 10;
-
+
final PriorityQueue priorityQueue = new PriorityQueue<Long>(cleanupSize) {
@Override
protected boolean lessThan(Long a, Long b) {
return (a > b);
}
};
-
- for(String child: children) {
+
+ for (String child : children) {
Stat stat = zookeeper.exists(dir + "/" + child, null, true);
priorityQueue.insertWithOverflow(stat.getMzxid());
}
-
+
long topElementMzxId = (Long) priorityQueue.top();
-
- for(String child:children) {
+
+ for (String child : children) {
Stat stat = zookeeper.exists(dir + "/" + child, null, true);
- if(stat.getMzxid() <= topElementMzxId)
+ if (stat.getMzxid() <= topElementMzxId)
zookeeper.delete(dir + "/" + child, -1, true);
}
}
-
- return createData(dir + "/" + prefix + trackingId, data,
- CreateMode.PERSISTENT) != null;
+
+ super.put(trackingId, data);
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/32fbca6e/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index c9698ae..74507d1 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -564,12 +564,12 @@ public class CollectionsHandler extends RequestHandlerBase {
final NamedList<Object> results = new NamedList<>();
if (zkController.getOverseerCompletedMap().contains(requestId)) {
- final DistributedMap.MapEvent mapEvent = zkController.getOverseerCompletedMap().get(requestId);
- rsp.getValues().addAll(SolrResponse.deserialize(mapEvent.getBytes()).getResponse());
+ final byte[] mapEntry = zkController.getOverseerCompletedMap().get(requestId);
+ rsp.getValues().addAll(SolrResponse.deserialize(mapEntry).getResponse());
addStatusToResponse(results, COMPLETED, "found [" + requestId + "] in completed tasks");
} else if (zkController.getOverseerFailureMap().contains(requestId)) {
- final DistributedMap.MapEvent mapEvent = zkController.getOverseerFailureMap().get(requestId);
- rsp.getValues().addAll(SolrResponse.deserialize(mapEvent.getBytes()).getResponse());
+ final byte[] mapEntry = zkController.getOverseerFailureMap().get(requestId);
+ rsp.getValues().addAll(SolrResponse.deserialize(mapEntry).getResponse());
addStatusToResponse(results, FAILED, "found [" + requestId + "] in failed tasks");
} else if (zkController.getOverseerRunningMap().contains(requestId)) {
addStatusToResponse(results, RUNNING, "found [" + requestId + "] in running tasks");
[2/2] lucene-solr git commit: SOLR-8695: Ensure ZK watchers are not
triggering our watch logic on connection events and make this handling more
consistent.
Posted by ma...@apache.org.
SOLR-8695: Ensure ZK watchers are not triggering our watch logic on connection events and make this handling more consistent.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/e30d638c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e30d638c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e30d638c
Branch: refs/heads/master
Commit: e30d638c51f9c6cf9d462741d05e91302ff4b56d
Parents: 32fbca6
Author: markrmiller <ma...@apache.org>
Authored: Fri Feb 19 14:36:53 2016 -0500
Committer: markrmiller <ma...@apache.org>
Committed: Fri Feb 19 14:36:53 2016 -0500
----------------------------------------------------------------------
solr/CHANGES.txt | 3 +++
.../org/apache/solr/cloud/LeaderElector.java | 3 +--
.../apache/solr/cloud/OverseerTaskQueue.java | 10 +++++---
.../org/apache/solr/cloud/ZkController.java | 3 ++-
.../solr/handler/CdcrBufferStateManager.java | 1 +
.../solr/handler/CdcrLeaderStateManager.java | 1 +
.../solr/handler/CdcrProcessStateManager.java | 1 +
.../handler/admin/ZookeeperInfoHandler.java | 4 +++
.../apache/solr/common/cloud/ZkStateReader.java | 26 ++++++++------------
9 files changed, 29 insertions(+), 23 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index c0c8856..f64f552 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -184,6 +184,9 @@ Bug Fixes
* SOLR-8694: DistributedMap/Queue can create too many Watchers and some code simplification.
(Scott Blum via Mark Miller)
+
+* SOLR-8695: Ensure ZK watchers are not triggering our watch logic on connection events and
+ make this handling more consistent. (Scott Blum via Mark Miller)
Optimizations
----------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
index 056b4b9..af27f00 100644
--- a/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
+++ b/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
@@ -367,8 +367,7 @@ public class LeaderElector {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
index ea219d3..cf9d583 100644
--- a/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
+++ b/solr/core/src/java/org/apache/solr/cloud/OverseerTaskQueue.java
@@ -129,11 +129,13 @@ public class OverseerTaskQueue extends DistributedQueue {
@Override
public void process(WatchedEvent event) {
- Event.EventType eventType = event.getType();
- // None events are ignored
+ // session events are not change events, and do not remove the watcher
+ if (Event.EventType.None.equals(event.getType())) {
+ return;
+ }
// If latchEventType is not null, only fire if the type matches
- LOG.info("{} fired on path {} state {} latchEventType {}", eventType, event.getPath(), event.getState(), latchEventType);
- if (eventType != Event.EventType.None && (latchEventType == null || eventType == latchEventType)) {
+ LOG.info("{} fired on path {} state {} latchEventType {}", event.getType(), event.getPath(), event.getState(), latchEventType);
+ if (latchEventType == null || event.getType() == latchEventType) {
synchronized (lock) {
this.event = event;
lock.notifyAll();
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
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 e436716..7874e2b 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -2365,7 +2365,8 @@ public final class ZkController {
@Override
public void process(WatchedEvent event) {
- if (event.getState() == Event.KeeperState.Disconnected || event.getState() == Event.KeeperState.Expired) {
+ // session events are not change events, and do not remove the watcher
+ if (Event.EventType.None.equals(event.getType())) {
return;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
index 18c7d05..713d6fc 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrBufferStateManager.java
@@ -154,6 +154,7 @@ class CdcrBufferStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.info("The CDCR buffer state has changed: {} @ {}:{}", event, collectionName, shard);
+ // session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
index 59313d3..7d8ddc4 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrLeaderStateManager.java
@@ -137,6 +137,7 @@ class CdcrLeaderStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.debug("The leader state has changed: {} @ {}:{}", event, collectionName, shard);
+ // session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java b/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
index 7630702..b1c8dda 100644
--- a/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
+++ b/solr/core/src/java/org/apache/solr/handler/CdcrProcessStateManager.java
@@ -154,6 +154,7 @@ class CdcrProcessStateManager extends CdcrStateManager {
String shard = core.getCoreDescriptor().getCloudDescriptor().getShardId();
log.info("The CDCR process state has changed: {} @ {}:{}", event, collectionName, shard);
+ // session events are not change events, and do not remove the watcher
if (Event.EventType.None.equals(event.getType())) {
return;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
index 8bc2130..e2ce63d 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/ZookeeperInfoHandler.java
@@ -248,6 +248,10 @@ public final class ZookeeperInfoHandler extends RequestHandlerBase {
*/
@Override
public void process(WatchedEvent event) {
+ // session events are not change events, and do not remove the watcher
+ if (Event.EventType.None.equals(event.getType())) {
+ return;
+ }
synchronized (this) {
cachedCollections = null;
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e30d638c/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index bfe89e3..910b47e 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -306,8 +306,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@@ -359,8 +358,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@@ -797,15 +795,14 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- if (!interestingCollections.contains(coll)) {
- // This collection is no longer interesting, stop watching.
- LOG.info("Uninteresting collection [{}]", coll);
+ // session events are not change events, and do not remove the watcher
+ if (EventType.None.equals(event.getType())) {
return;
}
- // session events are not change events,
- // and do not remove the watcher
- if (EventType.None.equals(event.getType())) {
+ if (!interestingCollections.contains(coll)) {
+ // This collection is no longer interesting, stop watching.
+ LOG.info("Uninteresting collection {}", coll);
return;
}
@@ -852,8 +849,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@@ -890,8 +886,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}
@@ -924,8 +919,7 @@ public class ZkStateReader implements Closeable {
@Override
public void process(WatchedEvent event) {
- // session events are not change events,
- // and do not remove the watcher
+ // session events are not change events, and do not remove the watcher
if (EventType.None.equals(event.getType())) {
return;
}