You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2015/02/26 13:05:07 UTC
svn commit: r1662429 - in /lucene/dev/branches/lucene_solr_4_10: ./ solr/
solr/CHANGES.txt solr/core/
solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
Author: shalin
Date: Thu Feb 26 12:05:06 2015
New Revision: 1662429
URL: http://svn.apache.org/r1662429
Log:
SOLR-6631: DistributedQueue spinning on calling zookeeper getChildren()
Added:
lucene/dev/branches/lucene_solr_4_10/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
- copied unchanged from r1635576, lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/DistributedQueueTest.java
Modified:
lucene/dev/branches/lucene_solr_4_10/ (props changed)
lucene/dev/branches/lucene_solr_4_10/solr/ (props changed)
lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt (contents, props changed)
lucene/dev/branches/lucene_solr_4_10/solr/core/ (props changed)
lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
Modified: lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt?rev=1662429&r1=1662428&r2=1662429&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene_solr_4_10/solr/CHANGES.txt Thu Feb 26 12:05:06 2015
@@ -43,6 +43,9 @@ Bug Fixes
* SOLR-6481: CLUSTERSTATUS should check if the node hosting a replica is live when
reporting replica status (Timothy Potter)
+* SOLR-6631: DistributedQueue spinning on calling zookeeper getChildren()
+ (Jessica Cheng Mallet, Mark Miller, Timothy Potter)
+
================== 4.10.3 ==================
Bug Fixes
Modified: lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java?rev=1662429&r1=1662428&r2=1662429&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java (original)
+++ lucene/dev/branches/lucene_solr_4_10/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java Thu Feb 26 12:05:06 2015
@@ -45,8 +45,7 @@ import java.util.TreeMap;
* A distributed queue from zk recipes.
*/
public class DistributedQueue {
- private static final Logger LOG = LoggerFactory
- .getLogger(DistributedQueue.class);
+ private static final Logger LOG = LoggerFactory.getLogger(DistributedQueue.class);
private static long DEFAULT_TIMEOUT = 5*60*1000;
@@ -235,38 +234,50 @@ public class DistributedQueue {
time.stop();
}
}
-
-
- private class LatchChildWatcher implements Watcher {
-
- final Object lock;
- private WatchedEvent event = null;
-
- public LatchChildWatcher() {
- this.lock = new Object();
+
+ /**
+ * Watcher that blocks until a WatchedEvent occurs for a znode.
+ */
+ private final class LatchWatcher implements Watcher {
+
+ private final Object lock;
+ private WatchedEvent event;
+ private Event.EventType latchEventType;
+
+ LatchWatcher(Object lock) {
+ this(lock, null);
+ }
+
+ LatchWatcher(Event.EventType eventType) {
+ this(new Object(), eventType);
}
- public LatchChildWatcher(Object lock) {
+ LatchWatcher(Object lock, Event.EventType eventType) {
this.lock = lock;
+ this.latchEventType = eventType;
}
-
+
@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();
+ Event.EventType eventType = event.getType();
+ // None events are ignored
+ // If latchEventType is not null, only fire if the type matches
+ if (eventType != Event.EventType.None && (latchEventType == null || eventType == latchEventType)) {
+ LOG.info("{} fired on path {} state {}", eventType, event.getPath(), event.getState());
+ synchronized (lock) {
+ this.event = event;
+ lock.notifyAll();
+ }
}
}
-
+
public void await(long timeout) throws InterruptedException {
synchronized (lock) {
if (this.event != null) return;
lock.wait(timeout);
}
}
-
+
public WatchedEvent getWatchedEvent() {
return event;
}
@@ -274,13 +285,13 @@ public class DistributedQueue {
// we avoid creating *many* watches in some cases
// by saving the childrenWatcher and the children associated - see SOLR-6336
- private LatchChildWatcher childrenWatcher;
+ private LatchWatcher childrenWatcher;
private TreeMap<Long,String> fetchedChildren;
private final Object childrenWatcherLock = new Object();
private Map<Long, String> getChildren(long wait) throws InterruptedException, KeeperException
{
- LatchChildWatcher watcher;
+ LatchWatcher watcher;
TreeMap<Long,String> children;
synchronized (childrenWatcherLock) {
watcher = childrenWatcher;
@@ -288,7 +299,8 @@ public class DistributedQueue {
}
if (watcher == null || watcher.getWatchedEvent() != null) {
- watcher = new LatchChildWatcher();
+ // this watcher is only interested in child change events
+ watcher = new LatchWatcher(Watcher.Event.EventType.NodeChildrenChanged);
while (true) {
try {
children = orderedChildren(watcher);
@@ -390,8 +402,9 @@ public class DistributedQueue {
String watchID = createData(
dir + "/" + response_prefix + path.substring(path.lastIndexOf("-") + 1),
null, CreateMode.EPHEMERAL);
+
Object lock = new Object();
- LatchChildWatcher watcher = new LatchChildWatcher(lock);
+ LatchWatcher watcher = new LatchWatcher(lock);
synchronized (lock) {
if (zookeeper.exists(watchID, watcher, true) != null) {
watcher.await(timeout);