You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ab...@apache.org on 2017/06/15 10:33:59 UTC
lucene-solr:jira/solr-10878: SOLR-10878 Initial patch,
still not working quite right.
Repository: lucene-solr
Updated Branches:
refs/heads/jira/solr-10878 [created] 08dadc7e4
SOLR-10878 Initial patch, still not working quite right.
Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/08dadc7e
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/08dadc7e
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/08dadc7e
Branch: refs/heads/jira/solr-10878
Commit: 08dadc7e4e6c61efbc6dfc24cc128b747bfc56a0
Parents: 232eff0
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Thu Jun 15 12:33:28 2017 +0200
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Thu Jun 15 12:33:28 2017 +0200
----------------------------------------------------------------------
.../org/apache/solr/cloud/MoveReplicaCmd.java | 38 +++++++++--
.../org/apache/solr/cloud/ReplaceNodeCmd.java | 43 +++++++++---
.../org/apache/solr/cloud/MoveReplicaTest.java | 72 +++++++++++++++++++-
3 files changed, 132 insertions(+), 21 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/08dadc7e/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
index fed1398..42bc8b3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/MoveReplicaCmd.java
@@ -22,6 +22,8 @@ import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.cloud.ClusterState;
@@ -29,6 +31,7 @@ import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.util.NamedList;
import org.apache.solr.common.util.Utils;
@@ -60,6 +63,7 @@ public class MoveReplicaCmd implements Cmd{
ocmh.checkRequired(message, COLLECTION_PROP, "targetNode");
String collection = message.getStr(COLLECTION_PROP);
String targetNode = message.getStr("targetNode");
+ int timeout = message.getInt("timeout", 10 * 60); // 10 minutes
String async = message.getStr(ASYNC);
@@ -103,14 +107,14 @@ public class MoveReplicaCmd implements Cmd{
assert slice != null;
Object dataDir = replica.get("dataDir");
if (dataDir != null && dataDir.toString().startsWith("hdfs:/")) {
- moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice);
+ moveHdfsReplica(clusterState, results, dataDir.toString(), targetNode, async, coll, replica, slice, timeout);
} else {
- moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice);
+ moveNormalReplica(clusterState, results, targetNode, async, coll, replica, slice, timeout);
}
}
private void moveHdfsReplica(ClusterState clusterState, NamedList results, String dataDir, String targetNode, String async,
- DocCollection coll, Replica replica, Slice slice) throws Exception {
+ DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception {
String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
ZkNodeProps removeReplicasProps = new ZkNodeProps(
@@ -154,7 +158,7 @@ public class MoveReplicaCmd implements Cmd{
}
private void moveNormalReplica(ClusterState clusterState, NamedList results, String targetNode, String async,
- DocCollection coll, Replica replica, Slice slice) throws Exception {
+ DocCollection coll, Replica replica, Slice slice, int timeout) throws Exception {
String newCoreName = Assign.buildCoreName(coll, slice.getName(), replica.getType());
ZkNodeProps addReplicasProps = new ZkNodeProps(
COLLECTION_PROP, coll.getName(),
@@ -163,7 +167,7 @@ public class MoveReplicaCmd implements Cmd{
CoreAdminParams.NAME, newCoreName);
if(async!=null) addReplicasProps.getProperties().put(ASYNC, async);
NamedList addResult = new NamedList();
- ocmh.addReplica(clusterState, addReplicasProps, addResult, null);
+ ZkNodeProps newReplica = ocmh.addReplica(clusterState, addReplicasProps, addResult, null);
if (addResult.get("failure") != null) {
String errorString = String.format(Locale.ROOT, "Failed to create replica for collection=%s shard=%s" +
" on node=%s", coll.getName(), slice.getName(), targetNode);
@@ -171,12 +175,32 @@ public class MoveReplicaCmd implements Cmd{
results.add("failure", errorString);
return;
}
-
+ // wait for the other replica to be active if the source replica was a leader
+ if (replica.equals(slice.getLeader())) {
+ log.info("==== Waiting for leader replica to recover.");
+ CountDownLatch countDownLatch = new CountDownLatch(1);
+ ReplaceNodeCmd.RecoveryWatcher watcher = new ReplaceNodeCmd.RecoveryWatcher(coll.getName(), slice.getName(),
+ replica.getName(), newReplica.getStr(ZkStateReader.CORE_NAME_PROP), countDownLatch);
+ try {
+ ocmh.zkStateReader.registerCollectionStateWatcher(coll.getName(), watcher);
+ if (!countDownLatch.await(timeout, TimeUnit.SECONDS)) {
+ String errorString = String.format(Locale.ROOT, "Timed out waiting for leader replica to recover, collection=%s shard=%s" +
+ " on node=%s", coll.getName(), slice.getName(), targetNode);
+ log.warn(errorString);
+ results.add("failure", errorString);
+ return;
+ } else {
+ log.info("Replica " + watcher.getRecoveredReplica() + " is active - deleting the source...");
+ }
+ } finally {
+ ocmh.zkStateReader.registerCollectionStateWatcher(coll.getName(), watcher);
+ }
+ }
ZkNodeProps removeReplicasProps = new ZkNodeProps(
COLLECTION_PROP, coll.getName(),
SHARD_ID_PROP, slice.getName(),
REPLICA_PROP, replica.getName());
- if(async!=null) removeReplicasProps.getProperties().put(ASYNC, async);
+ if (async != null) removeReplicasProps.getProperties().put(ASYNC, async);
NamedList deleteResult = new NamedList();
ocmh.deleteReplica(clusterState, removeReplicasProps, deleteResult, null);
if (deleteResult.get("failure") != null) {
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/08dadc7e/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
index 5adbe8c..ba60908 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ReplaceNodeCmd.java
@@ -93,15 +93,6 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
CountDownLatch replicasToRecover = new CountDownLatch(numLeaders);
for (ZkNodeProps sourceReplica : sourceReplicas) {
- if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false)) {
- String shardName = sourceReplica.getStr(SHARD_ID_PROP);
- String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
- String collectionName = sourceReplica.getStr(COLLECTION_PROP);
- String key = collectionName + "_" + replicaName;
- RecoveryWatcher watcher = new RecoveryWatcher(collectionName, shardName, replicaName, replicasToRecover);
- watchers.put(key, watcher);
- zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
- }
NamedList nl = new NamedList();
log.info("Going to create replica for collection={} shard={} on node={}", sourceReplica.getStr(COLLECTION_PROP), sourceReplica.getStr(SHARD_ID_PROP), target);
ZkNodeProps msg = sourceReplica.plus("parallel", String.valueOf(parallel)).plus(CoreAdminParams.NODE, target);
@@ -128,6 +119,16 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
if (addedReplica != null) {
createdReplicas.add(addedReplica);
+ if (sourceReplica.getBool(ZkStateReader.LEADER_PROP, false)) {
+ String shardName = sourceReplica.getStr(SHARD_ID_PROP);
+ String replicaName = sourceReplica.getStr(ZkStateReader.REPLICA_PROP);
+ String collectionName = sourceReplica.getStr(COLLECTION_PROP);
+ String key = collectionName + "_" + replicaName;
+ RecoveryWatcher watcher = new RecoveryWatcher(collectionName, shardName, replicaName,
+ addedReplica.getStr(ZkStateReader.CORE_NAME_PROP), replicasToRecover);
+ watchers.put(key, watcher);
+ zkStateReader.registerCollectionStateWatcher(collectionName, watcher);
+ }
}
}
@@ -208,16 +209,27 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
}
// we use this watcher to wait for replicas to recover
- private static class RecoveryWatcher implements CollectionStateWatcher {
+ static class RecoveryWatcher implements CollectionStateWatcher {
String collectionId;
String shardId;
String replicaId;
+ String targetCore;
CountDownLatch countDownLatch;
+ Replica recovered;
- RecoveryWatcher(String collectionId, String shardId, String replicaId, CountDownLatch countDownLatch) {
+ /**
+ * Watch for recovery of a replica
+ * @param collectionId collection name
+ * @param shardId shard id
+ * @param replicaId source replica name (coreNodeName)
+ * @param targetCore specific target core name - if null then any active replica will do
+ * @param countDownLatch countdown when recovered
+ */
+ RecoveryWatcher(String collectionId, String shardId, String replicaId, String targetCore, CountDownLatch countDownLatch) {
this.collectionId = collectionId;
this.shardId = shardId;
this.replicaId = replicaId;
+ this.targetCore = targetCore;
this.countDownLatch = countDownLatch;
}
@@ -241,7 +253,12 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
continue;
}
// check its state
+ String coreName = replica.getStr(ZkStateReader.CORE_NAME_PROP);
+ if (targetCore != null && !targetCore.equals(coreName)) {
+ continue;
+ }
if (replica.isActive(liveNodes)) { // recovered - stop waiting
+ recovered = replica;
countDownLatch.countDown();
return true;
}
@@ -250,5 +267,9 @@ public class ReplaceNodeCmd implements OverseerCollectionMessageHandler.Cmd {
// set the watch again to wait for the new replica to recover
return false;
}
+
+ public Replica getRecoveredReplica() {
+ return recovered;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/08dadc7e/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
index 4368fea..93aaaaa 100644
--- a/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/MoveReplicaTest.java
@@ -31,8 +31,10 @@ import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.CoreAdminRequest;
import org.apache.solr.client.solrj.response.CoreAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
+import org.apache.solr.common.cloud.DocCollection;
import org.apache.solr.common.cloud.Replica;
import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
@@ -56,10 +58,11 @@ public class MoveReplicaTest extends SolrCloudTestCase {
cluster.waitForAllNodes(5000);
String coll = "movereplicatest_coll";
log.info("total_jettys: " + cluster.getJettySolrRunners().size());
+ int REPLICATION = 1;
CloudSolrClient cloudClient = cluster.getSolrClient();
- CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, 2);
+ CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(coll, "conf1", 2, REPLICATION);
create.setMaxShardsPerNode(2);
cloudClient.request(create);
@@ -98,12 +101,75 @@ public class MoveReplicaTest extends SolrCloudTestCase {
}
assertTrue(success);
checkNumOfCores(cloudClient, replica.getNodeName(), 0);
- checkNumOfCores(cloudClient, targetNode, 2);
+ checkNumOfCores(cloudClient, targetNode, REPLICATION);
+ // wait for recovery
+ boolean recovered = false;
+ for (int i = 0; i < 300; i++) {
+ DocCollection collState = getCollectionState(coll);
+ List<Replica> replicas = collState.getReplicas(targetNode);
+ boolean allActive = true;
+ boolean hasLeaders = true;
+ if (replicas != null && !replicas.isEmpty()) {
+ for (Replica r : replicas) {
+ if (!r.isActive(Collections.singleton(targetNode))) {
+ log.info("Not active yet: " + r);
+ allActive = false;
+ }
+ }
+ } else {
+ allActive = false;
+ }
+ for (Slice slice : collState.getSlices()) {
+ if (slice.getLeader() == null) {
+ hasLeaders = false;
+ }
+ }
+ if (allActive && hasLeaders) {
+ assertEquals("total number of replicas", REPLICATION, replicas.size());
+ recovered = true;
+ break;
+ } else {
+ log.info("--- waiting, allActive=" + allActive + ", hasLeaders=" + hasLeaders);
+ Thread.sleep(1000);
+ }
+ }
+ assertTrue("replica never fully recovered", recovered);
moveReplica = new CollectionAdminRequest.MoveReplica(coll, shardId, targetNode, replica.getNodeName());
moveReplica.process(cloudClient);
checkNumOfCores(cloudClient, replica.getNodeName(), 1);
- checkNumOfCores(cloudClient, targetNode, 1);
+ checkNumOfCores(cloudClient, targetNode, REPLICATION - 1);
+ // wait for recovery
+ recovered = false;
+ for (int i = 0; i < 300; i++) {
+ DocCollection collState = getCollectionState(coll);
+ List<Replica> replicas = collState.getReplicas(targetNode);
+ boolean allActive = true;
+ boolean hasLeaders = true;
+ if (replicas != null && !replicas.isEmpty()) {
+ for (Replica r : replicas) {
+ if (!r.isActive(Collections.singleton(targetNode))) {
+ log.info("Not active yet: " + r);
+ allActive = false;
+ }
+ }
+ } else {
+ allActive = false;
+ }
+ for (Slice slice : collState.getSlices()) {
+ if (slice.getLeader() == null) {
+ hasLeaders = false;
+ }
+ }
+ if (allActive && hasLeaders) {
+ assertEquals("total number of replicas", 1, replicas.size());
+ recovered = true;
+ break;
+ } else {
+ Thread.sleep(1000);
+ }
+ }
+ assertTrue("replica never fully recovered", recovered);
}
private Replica getRandomReplica(String coll, CloudSolrClient cloudClient) {