mirror of https://github.com/apache/lucene.git
SOLR-13860: Enable back TestTlogReplica (#965)
* Wait for leader in testBasicLeaderElection It can take some time (>4 seconds) to elect a new leader, and if the update is attempted immediately it'll fail. Need to either wait for the leader or retry the udpate in case of failure (which is what clients would do) * Wait for leader to be active in testKillTlogReplica * Add hack to prevent unrelated failure * Reduce the time wait time for replica state change * A more robust attempt to add replicas in the tests * Wait for replication for 2 times the replication time
This commit is contained in:
parent
3a3df47840
commit
98cdac82a1
|
@ -39,7 +39,6 @@ import org.apache.http.client.methods.HttpGet;
|
|||
import org.apache.http.client.methods.HttpPost;
|
||||
import org.apache.http.entity.StringEntity;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.solr.client.solrj.SolrClient;
|
||||
import org.apache.solr.client.solrj.SolrQuery;
|
||||
|
@ -76,7 +75,6 @@ import org.slf4j.Logger;
|
|||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Slow
|
||||
@AwaitsFix(bugUrl = "https://issues.apache.org/jira/browse/SOLR-12313")
|
||||
public class TestTlogReplica extends SolrCloudTestCase {
|
||||
|
||||
private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
|
||||
|
@ -90,6 +88,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
|
||||
@BeforeClass
|
||||
public static void setupCluster() throws Exception {
|
||||
System.setProperty("solr.waitToSeeReplicasInStateTimeoutSeconds", "30");
|
||||
configureCluster(2) // 2 + random().nextInt(3)
|
||||
.addConfig("conf", configset("cloud-minimal-inplace-updates"))
|
||||
.configure();
|
||||
|
@ -149,80 +148,75 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
@Repeat(iterations=2) // 2 times to make sure cleanup is complete and we can create the same collection
|
||||
// commented out on: 17-Feb-2019 @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028") // added 09-Aug-2018
|
||||
public void testCreateDelete() throws Exception {
|
||||
try {
|
||||
switch (random().nextInt(3)) {
|
||||
case 0:
|
||||
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 0, 4, 0)
|
||||
.setMaxShardsPerNode(100)
|
||||
.process(cluster.getSolrClient());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
case 1:
|
||||
// Sometimes don't use SolrJ
|
||||
String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&collection.configName=%s&numShards=%s&tlogReplicas=%s&maxShardsPerNode=%s",
|
||||
cluster.getRandomJetty(random()).getBaseUrl(),
|
||||
collectionName, "conf",
|
||||
2, // numShards
|
||||
4, // tlogReplicas
|
||||
100); // maxShardsPerNode
|
||||
HttpGet createCollectionGet = new HttpGet(url);
|
||||
HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionGet);
|
||||
assertEquals(200, httpResponse.getStatusLine().getStatusCode());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
case 2:
|
||||
// Sometimes use V2 API
|
||||
url = cluster.getRandomJetty(random()).getBaseUrl().toString() + "/____v2/c";
|
||||
String requestBody = String.format(Locale.ROOT, "{create:{name:%s, config:%s, numShards:%s, tlogReplicas:%s, maxShardsPerNode:%s}}",
|
||||
collectionName, "conf",
|
||||
2, // numShards
|
||||
4, // tlogReplicas
|
||||
100); // maxShardsPerNode
|
||||
HttpPost createCollectionPost = new HttpPost(url);
|
||||
createCollectionPost.setHeader("Content-type", "application/json");
|
||||
createCollectionPost.setEntity(new StringEntity(requestBody));
|
||||
httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionPost);
|
||||
assertEquals(200, httpResponse.getStatusLine().getStatusCode());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
}
|
||||
switch (random().nextInt(3)) {
|
||||
case 0:
|
||||
CollectionAdminRequest.createCollection(collectionName, "conf", 2, 0, 4, 0)
|
||||
.setMaxShardsPerNode(100)
|
||||
.process(cluster.getSolrClient());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
case 1:
|
||||
// Sometimes don't use SolrJ
|
||||
String url = String.format(Locale.ROOT, "%s/admin/collections?action=CREATE&name=%s&collection.configName=%s&numShards=%s&tlogReplicas=%s&maxShardsPerNode=%s",
|
||||
cluster.getRandomJetty(random()).getBaseUrl(),
|
||||
collectionName, "conf",
|
||||
2, // numShards
|
||||
4, // tlogReplicas
|
||||
100); // maxShardsPerNode
|
||||
HttpGet createCollectionGet = new HttpGet(url);
|
||||
HttpResponse httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionGet);
|
||||
assertEquals(200, httpResponse.getStatusLine().getStatusCode());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
case 2:
|
||||
// Sometimes use V2 API
|
||||
url = cluster.getRandomJetty(random()).getBaseUrl().toString() + "/____v2/c";
|
||||
String requestBody = String.format(Locale.ROOT, "{create:{name:%s, config:%s, numShards:%s, tlogReplicas:%s, maxShardsPerNode:%s}}",
|
||||
collectionName, "conf",
|
||||
2, // numShards
|
||||
4, // tlogReplicas
|
||||
100); // maxShardsPerNode
|
||||
HttpPost createCollectionPost = new HttpPost(url);
|
||||
createCollectionPost.setHeader("Content-type", "application/json");
|
||||
createCollectionPost.setEntity(new StringEntity(requestBody));
|
||||
httpResponse = cluster.getSolrClient().getHttpClient().execute(createCollectionPost);
|
||||
assertEquals(200, httpResponse.getStatusLine().getStatusCode());
|
||||
cluster.waitForActiveCollection(collectionName, 2, 8);
|
||||
break;
|
||||
}
|
||||
|
||||
boolean reloaded = false;
|
||||
while (true) {
|
||||
DocCollection docCollection = getCollectionState(collectionName);
|
||||
assertNotNull(docCollection);
|
||||
assertEquals("Expecting 2 shards",
|
||||
2, docCollection.getSlices().size());
|
||||
assertEquals("Expecting 4 relpicas per shard",
|
||||
8, docCollection.getReplicas().size());
|
||||
assertEquals("Expecting 8 tlog replicas, 4 per shard",
|
||||
8, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
|
||||
assertEquals("Expecting no nrt replicas",
|
||||
0, docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
|
||||
assertEquals("Expecting no pull replicas",
|
||||
0, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
|
||||
for (Slice s:docCollection.getSlices()) {
|
||||
assertTrue(s.getLeader().getType() == Replica.Type.TLOG);
|
||||
List<String> shardElectionNodes = cluster.getZkClient().getChildren(ZkStateReader.getShardLeadersElectPath(collectionName, s.getName()), null, true);
|
||||
assertEquals("Unexpected election nodes for Shard: " + s.getName() + ": " + Arrays.toString(shardElectionNodes.toArray()),
|
||||
4, shardElectionNodes.size());
|
||||
}
|
||||
assertUlogPresence(docCollection);
|
||||
if (reloaded) {
|
||||
break;
|
||||
} else {
|
||||
// reload
|
||||
CollectionAdminResponse response = CollectionAdminRequest.reloadCollection(collectionName)
|
||||
.process(cluster.getSolrClient());
|
||||
assertEquals(0, response.getStatus());
|
||||
waitForState("failed waiting for active colletion", collectionName, clusterShape(2, 8));
|
||||
reloaded = true;
|
||||
}
|
||||
boolean reloaded = false;
|
||||
while (true) {
|
||||
DocCollection docCollection = getCollectionState(collectionName);
|
||||
assertNotNull(docCollection);
|
||||
assertEquals("Expecting 2 shards",
|
||||
2, docCollection.getSlices().size());
|
||||
assertEquals("Expecting 4 relpicas per shard",
|
||||
8, docCollection.getReplicas().size());
|
||||
assertEquals("Expecting 8 tlog replicas, 4 per shard",
|
||||
8, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)).size());
|
||||
assertEquals("Expecting no nrt replicas",
|
||||
0, docCollection.getReplicas(EnumSet.of(Replica.Type.NRT)).size());
|
||||
assertEquals("Expecting no pull replicas",
|
||||
0, docCollection.getReplicas(EnumSet.of(Replica.Type.PULL)).size());
|
||||
for (Slice s:docCollection.getSlices()) {
|
||||
assertTrue(s.getLeader().getType() == Replica.Type.TLOG);
|
||||
List<String> shardElectionNodes = cluster.getZkClient().getChildren(ZkStateReader.getShardLeadersElectPath(collectionName, s.getName()), null, true);
|
||||
assertEquals("Unexpected election nodes for Shard: " + s.getName() + ": " + Arrays.toString(shardElectionNodes.toArray()),
|
||||
4, shardElectionNodes.size());
|
||||
}
|
||||
assertUlogPresence(docCollection);
|
||||
if (reloaded) {
|
||||
break;
|
||||
} else {
|
||||
// reload
|
||||
CollectionAdminResponse response = CollectionAdminRequest.reloadCollection(collectionName)
|
||||
.process(cluster.getSolrClient());
|
||||
assertEquals(0, response.getStatus());
|
||||
waitForState("failed waiting for active colletion", collectionName, clusterShape(2, 8));
|
||||
reloaded = true;
|
||||
}
|
||||
} finally {
|
||||
zkClient().printLayoutToStream(System.out);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -409,17 +403,8 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
docCollection = assertNumberOfReplicas(0, 1, 0, true, true);
|
||||
|
||||
// Wait until a new leader is elected
|
||||
TimeOut t = new TimeOut(30, TimeUnit.SECONDS, TimeSource.NANO_TIME);
|
||||
while (!t.hasTimedOut()) {
|
||||
docCollection = getCollectionState(collectionName);
|
||||
Replica leader = docCollection.getSlice("shard1").getLeader();
|
||||
if (leader != null && leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
|
||||
break;
|
||||
}
|
||||
Thread.sleep(500);
|
||||
}
|
||||
assertFalse("Timeout waiting for a new leader to be elected", t.hasTimedOut());
|
||||
|
||||
waitForLeaderChange(leaderJetty, "shard1");
|
||||
|
||||
// There is a new leader, I should be able to add and commit
|
||||
cluster.getSolrClient().add(collectionName, new SolrInputDocument("id", "2", "foo", "zoo"));
|
||||
cluster.getSolrClient().commit(collectionName);
|
||||
|
@ -428,7 +413,8 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
waitForNumDocsInAllReplicas(2, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)), REPLICATION_TIMEOUT_SECS);
|
||||
// Start back the node
|
||||
if (removeReplica) {
|
||||
CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
|
||||
addReplicaWithRetries();
|
||||
|
||||
} else {
|
||||
leaderJetty.start();
|
||||
}
|
||||
|
@ -437,6 +423,22 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
waitForNumDocsInAllReplicas(2, docCollection.getReplicas(EnumSet.of(Replica.Type.TLOG)), REPLICATION_TIMEOUT_SECS);
|
||||
}
|
||||
|
||||
private void addReplicaWithRetries() throws SolrServerException, IOException {
|
||||
int maxAttempts = 3;
|
||||
for (int i = 0; i < maxAttempts ; i++) {
|
||||
try {
|
||||
CollectionAdminResponse respone = CollectionAdminRequest.addReplicaToShard(collectionName, "shard1", Replica.Type.TLOG).process(cluster.getSolrClient());
|
||||
// This is an unfortunate hack. There are cases where the ADDREPLICA fails, will create a Jira to address that separately. for now, we'll retry
|
||||
if (respone.isSuccess()) {
|
||||
break;
|
||||
}
|
||||
log.error("Unsuccessful atempt to add replica. Attempt: %d/%d", i, maxAttempts);
|
||||
} catch (SolrException e) {
|
||||
log.error("Exception while adding replica. Attempt: " + i + "/" + maxAttempts, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void testKillTlogReplica() throws Exception {
|
||||
DocCollection docCollection = createAndWaitForCollection(1, 0, 2, 0);
|
||||
|
||||
|
@ -448,6 +450,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
JettySolrRunner pullReplicaJetty = cluster.getReplicaJetty(docCollection.getSlice("shard1").getReplicas(EnumSet.of(Replica.Type.TLOG)).get(0));
|
||||
pullReplicaJetty.stop();
|
||||
waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
|
||||
waitForLeaderChange(pullReplicaJetty, "shard1");
|
||||
// // Also wait for the replica to be placed in state="down"
|
||||
// waitForState("Didn't update state", collectionName, clusterStateReflectsActiveAndDownReplicas());
|
||||
|
||||
|
@ -461,7 +464,6 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
}
|
||||
|
||||
@Test
|
||||
// Removed BadApple on 2018-05-21
|
||||
public void testOnlyLeaderIndexes() throws Exception {
|
||||
createAndWaitForCollection(1, 0, 2, 0);
|
||||
|
||||
|
@ -525,7 +527,7 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
}
|
||||
}
|
||||
checkRTG(120,150, cluster.getJettySolrRunners());
|
||||
waitForReplicasCatchUp(20);
|
||||
waitForReplicasCatchUp(4 * REPLICATION_TIMEOUT_SECS);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
@ -557,7 +559,6 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
for (int i = 0; i < 3; i++) {
|
||||
UpdateRequest ureq = new UpdateRequest().add(sdoc("id", "7"));
|
||||
ureq.setParam("collection", collectionName);
|
||||
ureq.setParam(UpdateRequest.MIN_REPFACT, "2");
|
||||
NamedList<Object> response = cloudClient.request(ureq);
|
||||
if ((Integer)((NamedList<Object>)response.get("responseHeader")).get(UpdateRequest.REPFACT) >= 2) {
|
||||
break;
|
||||
|
@ -595,7 +596,6 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
for (int i = 0; i < 3; i++) {
|
||||
UpdateRequest ureq = new UpdateRequest().add(sdoc("id", "8"));
|
||||
ureq.setParam("collection", collectionName);
|
||||
ureq.setParam(UpdateRequest.MIN_REPFACT, "2");
|
||||
NamedList<Object> response = cloudClient.request(ureq);
|
||||
if ((Integer)((NamedList<Object>)response.get("responseHeader")).get(UpdateRequest.REPFACT) >= 2) {
|
||||
break;
|
||||
|
@ -658,7 +658,11 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
|
||||
oldLeaderJetty.stop();
|
||||
waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
|
||||
new UpdateRequest()
|
||||
|
||||
// Even after the replica is gone, a leader may not be elected yet. Wait for it.
|
||||
waitForLeaderChange(oldLeaderJetty, "shard1");
|
||||
|
||||
new UpdateRequest()
|
||||
.add(sdoc("id", "3"))
|
||||
.add(sdoc("id", "4"))
|
||||
.process(cloudClient, collectionName);
|
||||
|
@ -670,6 +674,18 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
waitForNumDocsInAllActiveReplicas(4, 0);
|
||||
}
|
||||
|
||||
private void waitForLeaderChange(JettySolrRunner oldLeaderJetty, String shardName) {
|
||||
waitForState("Expect new leader", collectionName,
|
||||
(liveNodes, collectionState) -> {
|
||||
Replica leader = collectionState.getLeader(shardName);
|
||||
if (leader == null || !leader.isActive(cluster.getSolrClient().getZkStateReader().getClusterState().getLiveNodes())) {
|
||||
return false;
|
||||
}
|
||||
return oldLeaderJetty == null || !leader.getNodeName().equals(oldLeaderJetty.getNodeName());
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
public void testOutOfOrderDBQWithInPlaceUpdates() throws Exception {
|
||||
createAndWaitForCollection(1,0,2,0);
|
||||
assertFalse(getSolrCore(true).get(0).getLatestSchema().getField("inplace_updatable_int").indexed());
|
||||
|
@ -687,16 +703,9 @@ public class TestTlogReplica extends SolrCloudTestCase {
|
|||
}
|
||||
}
|
||||
JettySolrRunner oldLeaderJetty = getSolrRunner(true).get(0);
|
||||
String oldLeaderNodeName = oldLeaderJetty.getNodeName();
|
||||
oldLeaderJetty.stop();
|
||||
waitForState("Replica not removed", collectionName, activeReplicaCount(0, 1, 0));
|
||||
waitForState("Expect new leader", collectionName,
|
||||
(liveNodes, collectionState) -> {
|
||||
Replica leader = collectionState.getLeader("shard1");
|
||||
if (leader == null) return false;
|
||||
return !leader.getNodeName().equals(oldLeaderNodeName);
|
||||
}
|
||||
);
|
||||
waitForLeaderChange(oldLeaderJetty, "shard1");
|
||||
oldLeaderJetty.start();
|
||||
waitForState("Replica not added", collectionName, activeReplicaCount(0, 2, 0));
|
||||
checkRTG(1,1, cluster.getJettySolrRunners());
|
||||
|
|
Loading…
Reference in New Issue