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 2014/01/10 15:52:17 UTC

svn commit: r1557140 - in /lucene/dev/branches/lucene_solr_4_6: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/update/processor/ solr/solrj/ solr/solrj/src/java/org/apache/solr/common/cloud/

Author: markrmiller
Date: Fri Jan 10 14:52:17 2014
New Revision: 1557140

URL: http://svn.apache.org/r1557140
Log:
SOLR-5577: Likely ZooKeeper expiration should not slow down updates a given amount, but instead cut off updates after a given time.

Modified:
    lucene/dev/branches/lucene_solr_4_6/   (props changed)
    lucene/dev/branches/lucene_solr_4_6/solr/   (props changed)
    lucene/dev/branches/lucene_solr_4_6/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/lucene_solr_4_6/solr/core/   (props changed)
    lucene/dev/branches/lucene_solr_4_6/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
    lucene/dev/branches/lucene_solr_4_6/solr/solrj/   (props changed)
    lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
    lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java

Modified: lucene/dev/branches/lucene_solr_4_6/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/solr/CHANGES.txt?rev=1557140&r1=1557139&r2=1557140&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/solr/CHANGES.txt (original)
+++ lucene/dev/branches/lucene_solr_4_6/solr/CHANGES.txt Fri Jan 10 14:52:17 2014
@@ -57,6 +57,10 @@ Bug Fixes
 * SOLR-5547: Creating a collection alias using SolrJ's CollectionAdminRequest
   sets the alias name and the collections to alias to the same value.
   (Aaron Schram, Mark Miller)
+
+* SOLR-5577: Likely ZooKeeper expiration should not slow down updates a given
+  amount, but instead cut off updates after a given time. 
+  (Mark Miller, Christine Poerschke)
   
 * SOLR-5580: NPE when creating a core with both explicit shard and coreNodeName.
   (YouPeng Yang, Mark Miller)

Modified: lucene/dev/branches/lucene_solr_4_6/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java?rev=1557140&r1=1557139&r2=1557140&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java (original)
+++ lucene/dev/branches/lucene_solr_4_6/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java Fri Jan 10 14:52:17 2014
@@ -1123,22 +1123,10 @@ public class DistributedUpdateProcessor 
       return;
     }
 
-    if (zkController.isConnected()) {
+    if (!zkController.getZkClient().getConnectionManager().isLikelyExpired()) {
       return;
     }
     
-    long timeoutAt = System.currentTimeMillis() + zkController.getClientTimeout();
-    while (System.currentTimeMillis() < timeoutAt) {
-      if (zkController.isConnected()) {
-        return;
-      }
-      try {
-        Thread.sleep(100);
-      } catch (InterruptedException e) {
-        Thread.currentThread().interrupt();
-        break;
-      }
-    }
     throw new SolrException(ErrorCode.SERVICE_UNAVAILABLE, "Cannot talk to ZooKeeper - Updates are disabled.");
   }
 

Modified: lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java?rev=1557140&r1=1557139&r2=1557140&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java (original)
+++ lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java Fri Jan 10 14:52:17 2014
@@ -17,6 +17,8 @@ package org.apache.solr.common.cloud;
  * limitations under the License.
  */
 
+import java.util.Timer;
+import java.util.TimerTask;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeoutException;
 
@@ -27,7 +29,7 @@ import org.apache.zookeeper.Watcher.Even
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-class ConnectionManager implements Watcher {
+public class ConnectionManager implements Watcher {
   protected static final Logger log = LoggerFactory
       .getLogger(ConnectionManager.class);
 
@@ -35,25 +37,25 @@ class ConnectionManager implements Watch
   private CountDownLatch clientConnected;
   private KeeperState state;
   private boolean connected;
+  private boolean likelyExpired = true;
 
   private final ZkClientConnectionStrategy connectionStrategy;
 
   private final String zkServerAddress;
 
-  private final int zkClientTimeout;
-
   private final SolrZkClient client;
 
   private final OnReconnect onReconnect;
 
   private volatile boolean isClosed = false;
+  
+  private volatile Timer disconnectedTimer;
 
   public ConnectionManager(String name, SolrZkClient client, String zkServerAddress, int zkClientTimeout, ZkClientConnectionStrategy strat, OnReconnect onConnect) {
     this.name = name;
     this.client = client;
     this.connectionStrategy = strat;
     this.zkServerAddress = zkServerAddress;
-    this.zkClientTimeout = zkClientTimeout;
     this.onReconnect = onConnect;
     reset();
   }
@@ -61,6 +63,35 @@ class ConnectionManager implements Watch
   private synchronized void reset() {
     clientConnected = new CountDownLatch(1);
     state = KeeperState.Disconnected;
+    disconnected();
+  }
+  
+  private synchronized void connected() {
+    connected = true;
+    if (disconnectedTimer != null) {
+      disconnectedTimer.cancel();
+      disconnectedTimer = null;
+    }
+    likelyExpired = false;
+  }
+
+  private synchronized void disconnected() {
+    if (disconnectedTimer != null) {
+      disconnectedTimer.cancel();
+      disconnectedTimer = null;
+    }
+    
+    disconnectedTimer = new Timer();
+    disconnectedTimer.schedule(new TimerTask() {
+      
+      @Override
+      public void run() {
+        synchronized (ConnectionManager.this) {
+          likelyExpired = true;
+        }
+      }
+      
+    }, (long) (client.getZkClientTimeout() * 0.90));
     connected = false;
   }
 
@@ -78,15 +109,15 @@ class ConnectionManager implements Watch
 
     state = event.getState();
     if (state == KeeperState.SyncConnected) {
-      connected = true;
+      connected();
       clientConnected.countDown();
       connectionStrategy.connected();
     } else if (state == KeeperState.Expired) {
-      connected = false;
+      disconnected();
       log.info("Our previous ZooKeeper session was expired. Attempting to reconnect to recover relationship with ZooKeeper...");
       
       try {
-        connectionStrategy.reconnect(zkServerAddress, zkClientTimeout, this,
+        connectionStrategy.reconnect(zkServerAddress, client.getZkClientTimeout(), this,
             new ZkClientConnectionStrategy.ZkUpdate() {
               @Override
               public void update(SolrZooKeeper keeper) {
@@ -114,9 +145,7 @@ class ConnectionManager implements Watch
                   onReconnect.command();
                 }
                 
-                synchronized (ConnectionManager.this) {
-                  ConnectionManager.this.connected = true;
-                }
+                connected();
                 
               }
             });
@@ -126,10 +155,10 @@ class ConnectionManager implements Watch
       log.info("Connected:" + connected);
     } else if (state == KeeperState.Disconnected) {
       log.info("zkClient has disconnected");
-      connected = false;
+      disconnected();
       connectionStrategy.disconnected();
     } else {
-      connected = false;
+      disconnected();
     }
     notifyAll();
   }
@@ -142,11 +171,19 @@ class ConnectionManager implements Watch
   // to avoid deadlock on shutdown
   public void close() {
     this.isClosed = true;
+    if (this.disconnectedTimer != null) {
+      this.disconnectedTimer.cancel();
+      this.disconnectedTimer = null;
+    }
   }
 
   public synchronized KeeperState state() {
     return state;
   }
+  
+  public synchronized boolean isLikelyExpired() {
+    return likelyExpired;
+  }
 
   public synchronized void waitForConnected(long waitForConnection)
       throws TimeoutException {

Modified: lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1557140&r1=1557139&r2=1557140&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/lucene_solr_4_6/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Fri Jan 10 14:52:17 2014
@@ -144,6 +144,10 @@ public class SolrZkClient {
     numOpens.incrementAndGet();
   }
 
+  public ConnectionManager getConnectionManager() {
+    return connManager;
+  }
+  
   public ZkClientConnectionStrategy getZkClientConnectionStrategy() {
     return zkClientConnectionStrategy;
   }