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 2012/11/21 16:02:48 UTC

svn commit: r1412140 - in /lucene/dev/trunk/solr: CHANGES.txt core/src/java/org/apache/solr/cloud/DistributedQueue.java

Author: markrmiller
Date: Wed Nov 21 15:02:47 2012
New Revision: 1412140

URL: http://svn.apache.org/viewvc?rev=1412140&view=rev
Log:
SOLR-4099: Allow the collection api work queue to make forward progress even when it's watcher is not fired for some reason.

Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1412140&r1=1412139&r2=1412140&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Wed Nov 21 15:02:47 2012
@@ -232,6 +232,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
 ----------------------

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java?rev=1412140&r1=1412139&r2=1412140&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/DistributedQueue.java Wed Nov 21 15:02:47 2012
@@ -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;
       }