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/01/06 20:00:15 UTC

svn commit: r1228347 - in /lucene/dev/branches/solrcloud/solr: core/src/java/org/apache/solr/cloud/ core/src/java/org/apache/solr/core/ solrj/src/java/org/apache/solr/common/cloud/

Author: markrmiller
Date: Fri Jan  6 19:00:15 2012
New Revision: 1228347

URL: http://svn.apache.org/viewvc?rev=1228347&view=rev
Log:
some hardening against connection loss exceptions

Added:
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java   (contents, props changed)
      - copied, changed from r1228294, lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkCmdExecutor.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZooKeeperOperation.java   (contents, props changed)
      - copied, changed from r1228294, lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZooKeeperOperation.java
Removed:
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkCmdExecutor.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZooKeeperOperation.java
Modified:
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
    lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1228347&r1=1228346&r2=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Fri Jan  6 19:00:15 2012
@@ -1,8 +1,10 @@
 package org.apache.solr.cloud;
 
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.cloud.ZooKeeperOperation;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java?rev=1228347&r1=1228346&r2=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/LeaderElector.java Fri Jan  6 19:00:15 2012
@@ -28,7 +28,9 @@ import java.util.regex.Pattern;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkCmdExecutor;
 import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.cloud.ZooKeeperOperation;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;

Modified: lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1228347&r1=1228346&r2=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/core/CoreContainer.java Fri Jan  6 19:00:15 2012
@@ -437,7 +437,9 @@ public class CoreContainer 
     synchronized(cores) {
       try {
         for(SolrCore core : cores.values()) {
-          core.close();
+          if (!core.isClosed()) {
+            core.close();
+          }
         }
         cores.clear();
       } finally {

Copied: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java (from r1228294, lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkCmdExecutor.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java?p2=lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java&p1=lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkCmdExecutor.java&r1=1228294&r2=1228347&rev=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZkCmdExecutor.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkCmdExecutor.java Fri Jan  6 19:00:15 2012
@@ -1,4 +1,4 @@
-package org.apache.solr.cloud;
+package org.apache.solr.common.cloud;
 
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -20,7 +20,6 @@ package org.apache.solr.cloud;
 import java.util.List;
 
 import org.apache.log4j.Logger;
-import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooDefs;
@@ -83,7 +82,7 @@ public class ZkCmdExecutor {
    * @return object. it needs to be cast to the callee's expected return type.
    */
   @SuppressWarnings("unchecked")
-  protected <T> T retryOperation(ZooKeeperOperation operation)
+  public <T> T retryOperation(ZooKeeperOperation operation)
       throws KeeperException, InterruptedException {
     KeeperException exception = null;
     for (int i = 0; i < retryCount; i++) {

Modified: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java?rev=1228347&r1=1228346&r2=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java Fri Jan  6 19:00:15 2012
@@ -119,9 +119,12 @@ public class ZkStateReader {
   private SolrZkClient zkClient;
   
   private boolean closeClient = false;
+
+  private ZkCmdExecutor cmdExecutor;
   
   public ZkStateReader(SolrZkClient zkClient) {
     this.zkClient = zkClient;
+    cmdExecutor = new ZkCmdExecutor(zkClient);
   }
   
   public ZkStateReader(String zkServerAddress, int zkClientTimeout, int zkClientConnectTimeout) throws InterruptedException, TimeoutException, IOException {
@@ -147,6 +150,7 @@ public class ZkStateReader {
 
           }
         });
+    cmdExecutor = new ZkCmdExecutor(zkClient);
   }
   
   // load and publish a new CollectionInfo
@@ -164,9 +168,26 @@ public class ZkStateReader {
     // We need to fetch the current cluster state and the set of live nodes
     
     synchronized (getUpdateLock()) {
-      if (!zkClient.exists(CLUSTER_STATE)) {
+      Boolean exists = cmdExecutor.retryOperation(new ZooKeeperOperation() {
+        
+        @Override
+        public Boolean execute() throws KeeperException, InterruptedException {
+          return zkClient.exists(CLUSTER_STATE);
+        }
+      });
+      
+      if (!exists) {
         try {
-          zkClient.create(CLUSTER_STATE, null, CreateMode.PERSISTENT);
+          cmdExecutor.retryOperation(new ZooKeeperOperation() {
+            
+            @Override
+            public Object execute() throws KeeperException, InterruptedException {
+              zkClient.create(CLUSTER_STATE, null, CreateMode.PERSISTENT);
+              return null;
+            }
+          });
+          
+          
         } catch (NodeExistsException e) {
           // if someone beats us to creating this ignore it
         }
@@ -186,7 +207,15 @@ public class ZkStateReader {
           // ZkStateReader.this.updateCloudState(false, false);
           synchronized (ZkStateReader.this.getUpdateLock()) {
             // remake watch
-            byte[] data = zkClient.getData(CLUSTER_STATE, this, null);
+            final Watcher thisWatch = this;
+            byte[] data = cmdExecutor.retryOperation(new ZooKeeperOperation() {
+              @Override
+              public byte[] execute() throws KeeperException,
+                  InterruptedException {
+                return zkClient.getData(CLUSTER_STATE, thisWatch, null);
+              }
+            });
+            
             CloudState clusterState = CloudState.load(data,
                 ZkStateReader.this.cloudState.getLiveNodes());
             // update volatile

Copied: lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZooKeeperOperation.java (from r1228294, lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZooKeeperOperation.java)
URL: http://svn.apache.org/viewvc/lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZooKeeperOperation.java?p2=lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZooKeeperOperation.java&p1=lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZooKeeperOperation.java&r1=1228294&r2=1228347&rev=1228347&view=diff
==============================================================================
--- lucene/dev/branches/solrcloud/solr/core/src/java/org/apache/solr/cloud/ZooKeeperOperation.java (original)
+++ lucene/dev/branches/solrcloud/solr/solrj/src/java/org/apache/solr/common/cloud/ZooKeeperOperation.java Fri Jan  6 19:00:15 2012
@@ -15,13 +15,13 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.solr.cloud;
+package org.apache.solr.common.cloud;
 
 import org.apache.zookeeper.KeeperException;
 
 /**
  * A callback object which can be used for implementing retry-able operations in the 
- * {@link org.apache.zookeeper.ZkCmdExecutor.lock.ProtocolSupport} class
+ * {@link org.apache.solr.common.cloud.ZkCmdExecutor.lock.ProtocolSupport} class
  *
  */
 public interface ZooKeeperOperation {