You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2014/12/24 16:56:55 UTC

svn commit: r1647812 - in /lucene/dev/branches/branch_5x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/cloud/ solr/core/src/test/org/apache/solr/cloud/

Author: sarowe
Date: Wed Dec 24 15:56:54 2014
New Revision: 1647812

URL: http://svn.apache.org/r1647812
Log:
SOLR-6879: Have an option to disable autoAddReplicas temporarily for all collections. (merged trunk r1647811)

Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1647812&r1=1647811&r2=1647812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Wed Dec 24 15:56:54 2014
@@ -212,6 +212,10 @@ New Features
 
 * SOLR-6729: createNodeSet.shuffle=(true|false) support for /admin/collections?action=CREATE.
   (Christine Poerschke, Ramkumar Aiyengar via Mark Miller)
+
+* SOLR-6879: Have an option to disable autoAddReplicas temporarily for all collections.
+  (Varun Thacker via Steve Rowe)
+
   
 Bug Fixes
 ----------------------

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java?rev=1647812&r1=1647811&r2=1647812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerAutoReplicaFailoverThread.java Wed Dec 24 15:56:54 2014
@@ -145,6 +145,11 @@ public class OverseerAutoReplicaFailover
     
     // TODO: extract to configurable strategy class ??
     ClusterState clusterState = zkStateReader.getClusterState();
+    //check if we have disabled autoAddReplicas cluster wide
+    String autoAddReplicas = (String) zkStateReader.getClusterProps().get(ZkStateReader.AUTO_ADD_REPLICAS);
+    if (autoAddReplicas !=null && autoAddReplicas.equals("false")) {
+      return;
+    }
     if (clusterState != null) {
       if (lastClusterStateVersion == clusterState.getZkClusterStateVersion() && baseUrlForBadNodes.size() == 0) {
         // nothing has changed, no work to do

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1647812&r1=1647811&r2=1647812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Wed Dec 24 15:56:54 2014
@@ -165,7 +165,8 @@ public class OverseerCollectionProcessor
 
   public int maxParallelThreads = 10;
 
-  public static final Set<String> KNOWN_CLUSTER_PROPS = ImmutableSet.of(ZkStateReader.LEGACY_CLOUD, ZkStateReader.URL_SCHEME);
+  public static final Set<String> KNOWN_CLUSTER_PROPS = ImmutableSet.of(ZkStateReader.LEGACY_CLOUD, ZkStateReader.URL_SCHEME,
+      ZkStateReader.AUTO_ADD_REPLICAS);
 
   public static final Map<String,Object> COLL_PROPS = ZkNodeProps.makeMap(
       ROUTER, DocRouter.DEFAULT_NAME,

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java?rev=1647812&r1=1647811&r2=1647812&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/cloud/SharedFSAutoReplicaFailoverTest.java Wed Dec 24 15:56:54 2014
@@ -19,6 +19,7 @@ package org.apache.solr.cloud;
 
 import java.util.Collection;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
@@ -27,25 +28,29 @@ import java.util.concurrent.SynchronousQ
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
+import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.lucene.util.LuceneTestCase.Nightly;
+import org.apache.lucene.util.LuceneTestCase.Slow;
 import org.apache.solr.SolrTestCaseJ4.SuppressSSL;
-import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
+import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.cloud.hdfs.HdfsTestUtil;
 import org.apache.solr.common.cloud.ClusterStateUtil;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionParams;
+import org.apache.solr.common.params.MapSolrParams;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope.Scope;
+import static org.apache.solr.common.cloud.ZkNodeProps.makeMap;
 
 @Nightly
 @Slow
@@ -109,7 +114,7 @@ public class SharedFSAutoReplicaFailover
       }
     }
   }
-  
+
   // very slow tests, especially since jetty is started and stopped
   // serially
   private void testBasics() throws Exception {
@@ -143,19 +148,19 @@ public class SharedFSAutoReplicaFailover
     assertTrue(response2.isSuccess());
     
     waitForRecoveriesToFinish(collection2, false);
-    
+
     ChaosMonkey.stop(jettys.get(1));
     ChaosMonkey.stop(jettys.get(2));
-    
+
     Thread.sleep(3000);
-    
+
     assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLive(cloudClient.getZkStateReader(), collection1, 120000));
-    
+
     assertSliceAndReplicaCount(collection1);
-    
+
     assertEquals(4, getLiveAndActiveCount(collection1));
     assertTrue(getLiveAndActiveCount(collection2) < 4);
-    
+
     ChaosMonkey.stop(jettys);
     ChaosMonkey.stop(controlJetty);
 
@@ -163,18 +168,47 @@ public class SharedFSAutoReplicaFailover
 
     ChaosMonkey.start(jettys);
     ChaosMonkey.start(controlJetty);
-    
+
     assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLive(cloudClient.getZkStateReader(), collection1, 120000));
 
     assertSliceAndReplicaCount(collection1);
-    
-    
+
     int jettyIndex = random().nextInt(jettys.size());
     ChaosMonkey.stop(jettys.get(jettyIndex));
     ChaosMonkey.start(jettys.get(jettyIndex));
-    
+
+    assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLive(cloudClient.getZkStateReader(), collection1, 60000));
+
+    //disable autoAddReplicas
+    Map m = makeMap(
+        "action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
+        "name", ZkStateReader.AUTO_ADD_REPLICAS,
+        "val", "false");
+
+    SolrRequest request = new QueryRequest(new MapSolrParams(m));
+    request.setPath("/admin/collections");
+    cloudClient.request(request);
+
+    int currentCount = getLiveAndActiveCount(collection1);
+
+    ChaosMonkey.stop(jettys.get(3));
+
+    //solr-no-core.xml has defined workLoopDelay=10s and waitAfterExpiration=10s
+    //Hence waiting for 30 seconds to be on the safe side.
+    Thread.sleep(30000);
+    //Ensures that autoAddReplicas has not kicked in.
+    assertTrue(currentCount > getLiveAndActiveCount(collection1));
+
+    //enable autoAddReplicas
+    m = makeMap(
+        "action", CollectionParams.CollectionAction.CLUSTERPROP.toLower(),
+        "name", ZkStateReader.AUTO_ADD_REPLICAS);
+
+    request = new QueryRequest(new MapSolrParams(m));
+    request.setPath("/admin/collections");
+    cloudClient.request(request);
+
     assertTrue("Timeout waiting for all live and active", ClusterStateUtil.waitForAllActiveAndLive(cloudClient.getZkStateReader(), collection1, 60000));
-    
     assertSliceAndReplicaCount(collection1);
   }