You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2015/04/21 07:49:39 UTC
svn commit: r1675030 - in /lucene/dev/trunk/solr: CHANGES.txt
core/src/java/org/apache/solr/cloud/ZkController.java
core/src/test/org/apache/solr/cloud/ZkControllerTest.java
Author: shalin
Date: Tue Apr 21 05:49:39 2015
New Revision: 1675030
URL: http://svn.apache.org/r1675030
Log:
SOLR-6665: ZkController.publishAndWaitForDownStates can return before all local cores are marked as 'down' if multiple replicas with the same core name exist in the cluster
Modified:
lucene/dev/trunk/solr/CHANGES.txt
lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1675030&r1=1675029&r2=1675030&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Tue Apr 21 05:49:39 2015
@@ -140,6 +140,10 @@ Bug Fixes
* SOLR-7426: SolrConfig#getConfigOverlay does not clean up it's resources. (Mark Miller)
+* SOLR-6665: ZkController.publishAndWaitForDownStates can return before all local cores are
+ marked as 'down' if multiple replicas with the same core name exist in the cluster.
+ (shalin)
+
Optimizations
----------------------
Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1675030&r1=1675029&r2=1675030&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/cloud/ZkController.java Tue Apr 21 05:49:39 2015
@@ -107,6 +107,7 @@ import org.slf4j.MDC;
public final class ZkController {
private static final Logger log = LoggerFactory.getLogger(ZkController.class);
+ static final int WAIT_DOWN_STATES_TIMEOUT_SECONDS = 60;
private final boolean SKIP_AUTO_RECOVERY = Boolean.getBoolean("solrcloud.skip.autorecovery");
@@ -662,7 +663,7 @@ public final class ZkController {
ClusterState clusterState = zkStateReader.getClusterState();
Set<String> collections = clusterState.getCollections();
- List<String> updatedNodes = new ArrayList<>();
+ Set<String> updatedCoreNodeNames = new HashSet<>();
for (String collectionName : collections) {
DocCollection collection = clusterState.getCollection(collectionName);
Collection<Slice> slices = collection.getSlices();
@@ -683,7 +684,7 @@ public final class ZkController {
replica.getStr(ZkStateReader.SHARD_ID_PROP),
ZkStateReader.COLLECTION_PROP, collectionName,
ZkStateReader.CORE_NODE_NAME_PROP, replica.getName());
- updatedNodes.add(replica.getStr(ZkStateReader.CORE_NAME_PROP));
+ updatedCoreNodeNames.add(replica.getName());
overseerJobQueue.offer(ZkStateReader.toJSON(m));
}
}
@@ -692,7 +693,7 @@ public final class ZkController {
// now wait till the updates are in our state
long now = System.nanoTime();
- long timeout = now + TimeUnit.NANOSECONDS.convert(60, TimeUnit.SECONDS);
+ long timeout = now + TimeUnit.NANOSECONDS.convert(WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
boolean foundStates = false;
while (System.nanoTime() < timeout) {
clusterState = zkStateReader.getClusterState();
@@ -704,14 +705,14 @@ public final class ZkController {
Collection<Replica> replicas = slice.getReplicas();
for (Replica replica : replicas) {
if (replica.getState() == Replica.State.DOWN) {
- updatedNodes.remove(replica.getStr(ZkStateReader.CORE_NAME_PROP));
+ updatedCoreNodeNames.remove(replica.getName());
}
}
}
}
- if (updatedNodes.size() == 0) {
+ if (updatedCoreNodeNames.size() == 0) {
foundStates = true;
Thread.sleep(1000);
break;
Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1675030&r1=1675029&r2=1675030&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Tue Apr 21 05:49:39 2015
@@ -19,7 +19,11 @@ package org.apache.solr.cloud;
import org.apache.lucene.util.LuceneTestCase.Slow;
import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocRouter;
import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.SolrZkClient;
import org.apache.solr.common.cloud.ZkConfigManager;
import org.apache.solr.common.cloud.ZkCoreNodeProps;
@@ -41,6 +45,7 @@ import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
+import java.util.concurrent.TimeUnit;
@Slow
public class ZkControllerTest extends SolrTestCaseJ4 {
@@ -173,7 +178,7 @@ public class ZkControllerTest extends So
props.put("configName", actualConfigName);
ZkNodeProps zkProps = new ZkNodeProps(props);
zkClient.makePath(ZkStateReader.COLLECTIONS_ZKNODE + "/"
- + COLLECTION_NAME, ZkStateReader.toJSON(zkProps),
+ + COLLECTION_NAME, ZkStateReader.toJSON(zkProps),
CreateMode.PERSISTENT, true);
if (DEBUG) {
@@ -246,6 +251,14 @@ public class ZkControllerTest extends So
}
}
+ /*
+ Test that:
+ 1) LIR state to 'down' is not set unless publishing node is a leader
+ 1a) Test that leader can publish when LIR node already exists in zk
+ 1b) Test that leader can publish when LIR node does not exist - TODO
+ 2) LIR state to 'active' or 'recovery' can be set regardless of whether publishing
+ node is leader or not - TODO
+ */
public void testEnsureReplicaInLeaderInitiatedRecovery() throws Exception {
String zkDir = createTempDir("testEnsureReplicaInLeaderInitiatedRecovery").toFile().getAbsolutePath();
CoreContainer cc = null;
@@ -296,14 +309,69 @@ public class ZkControllerTest extends So
}
}
- /*
- Test that:
- 1) LIR state to 'down' is not set unless publishing node is a leader
- 1a) Test that leader can publish when LIR node already exists in zk
- 1b) Test that leader can publish when LIR node does not exist
- 2) LIR state to 'active' or 'recovery' can be set regardless of whether publishing
- node is leader or not
- */
+ public void testPublishAndWaitForDownStates() throws Exception {
+ String zkDir = createTempDir("testPublishAndWaitForDownStates").toFile().getAbsolutePath();
+ CoreContainer cc = null;
+
+ ZkTestServer server = new ZkTestServer(zkDir);
+ try {
+ server.run();
+
+ AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
+ AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
+
+ cc = getCoreContainer();
+ ZkController zkController = null;
+
+ try {
+ CloudConfig cloudConfig = new CloudConfig.CloudConfigBuilder("127.0.0.1", 8983, "solr").build();
+ zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, cloudConfig, new CurrentCoreDescriptorProvider() {
+
+ @Override
+ public List<CoreDescriptor> getCurrentDescriptors() {
+ // do nothing
+ return null;
+ }
+ });
+
+ HashMap<String, DocCollection> collectionStates = new HashMap<>();
+ HashMap<String, Replica> replicas = new HashMap<>();
+ // add two replicas with the same core name but one of them should be on a different node
+ // than this ZkController instance
+ for (int i=1; i<=2; i++) {
+ Replica r = new Replica("core_node" + i,
+ map(ZkStateReader.STATE_PROP, i == 1 ? "active" : "down",
+ ZkStateReader.NODE_NAME_PROP, i == 1 ? "127.0.0.1:8983_solr" : "non_existent_host",
+ ZkStateReader.CORE_NAME_PROP, "collection1"));
+ replicas.put("core_node" + i, r);
+ }
+ HashMap<String, Object> sliceProps = new HashMap<>();
+ sliceProps.put("state", Slice.State.ACTIVE.toString());
+ Slice slice = new Slice("shard1", replicas, sliceProps);
+ DocCollection c = new DocCollection("testPublishAndWaitForDownStates", map("shard1", slice), Collections.emptyMap(), DocRouter.DEFAULT);
+ ClusterState state = new ClusterState(0, Collections.emptySet(), map("testPublishAndWaitForDownStates", c));
+ byte[] bytes = ZkStateReader.toJSON(state);
+ zkController.getZkClient().makePath(ZkStateReader.getCollectionPath("testPublishAndWaitForDownStates"), bytes, CreateMode.PERSISTENT, true);
+
+ zkController.getZkStateReader().updateClusterState(true);
+ assertTrue(zkController.getZkStateReader().getClusterState().hasCollection("testPublishAndWaitForDownStates"));
+ assertNotNull(zkController.getZkStateReader().getClusterState().getCollection("testPublishAndWaitForDownStates"));
+
+ long now = System.nanoTime();
+ long timeout = now + TimeUnit.NANOSECONDS.convert(ZkController.WAIT_DOWN_STATES_TIMEOUT_SECONDS, TimeUnit.SECONDS);
+ zkController.publishAndWaitForDownStates();
+ assertTrue("The ZkController.publishAndWaitForDownStates should have timed out but it didn't", System.nanoTime() >= timeout);
+ } finally {
+ if (zkController != null)
+ zkController.close();
+ }
+ } finally {
+ if (cc != null) {
+ cc.shutdown();
+ }
+ server.shutdown();
+ }
+ }
private CoreContainer getCoreContainer() {
return new MockCoreContainer();