mirror of https://github.com/apache/lucene.git
SOLR-4099: Allow the collection api work queue to make forward progress even when it's watcher is not fired for some reason.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1412140 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
dea0747084
commit
258baa7069
|
@ -233,6 +233,9 @@ Bug Fixes
|
|||
* SOLR-4097: Race can cause NPE in logging line on first cluster state update.
|
||||
(Mark Miller)
|
||||
|
||||
* SOLR-4099: Allow the collection api work queue to make forward progress even
|
||||
when it's watcher is not fired for some reason. (Raintung Li via Mark Miller)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.solr.cloud;
|
|||
import java.util.List;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
|
||||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.zookeeper.CreateMode;
|
||||
|
@ -40,6 +39,8 @@ public class DistributedQueue {
|
|||
private static final Logger LOG = LoggerFactory
|
||||
.getLogger(DistributedQueue.class);
|
||||
|
||||
private static long DEFAULT_TIMEOUT = 5*60*1000;
|
||||
|
||||
private final String dir;
|
||||
|
||||
private SolrZkClient zookeeper;
|
||||
|
@ -163,20 +164,22 @@ public class DistributedQueue {
|
|||
|
||||
private class LatchChildWatcher implements Watcher {
|
||||
|
||||
CountDownLatch latch;
|
||||
Object lock = new Object();
|
||||
|
||||
public LatchChildWatcher() {
|
||||
latch = new CountDownLatch(1);
|
||||
}
|
||||
public LatchChildWatcher() {}
|
||||
|
||||
public void process(WatchedEvent event) {
|
||||
LOG.debug("Watcher fired on path: " + event.getPath() + " state: "
|
||||
LOG.info("Watcher fired on path: " + event.getPath() + " state: "
|
||||
+ event.getState() + " type " + event.getType());
|
||||
latch.countDown();
|
||||
synchronized (lock) {
|
||||
lock.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
public void await() throws InterruptedException {
|
||||
latch.await();
|
||||
public void await(long timeout) throws InterruptedException {
|
||||
synchronized (lock) {
|
||||
lock.wait(timeout);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -197,7 +200,7 @@ public class DistributedQueue {
|
|||
continue;
|
||||
}
|
||||
if (orderedChildren.size() == 0) {
|
||||
childWatcher.await();
|
||||
childWatcher.await(DEFAULT_TIMEOUT);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
@ -274,7 +277,7 @@ public class DistributedQueue {
|
|||
continue;
|
||||
}
|
||||
if (orderedChildren.size() == 0) {
|
||||
childWatcher.await();
|
||||
childWatcher.await(DEFAULT_TIMEOUT);
|
||||
continue;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue