mirror of https://github.com/apache/lucene.git
more black hole avoidance
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1365437 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
3208495042
commit
b77c7e684d
solr/core/src/test/org/apache/solr/cloud
|
@ -685,19 +685,21 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
// kill a shard
|
||||
CloudJettyRunner deadShard = chaosMonkey.stopShard(SHARD2, 0);
|
||||
cloudClient.connect();
|
||||
int tries = 0;
|
||||
while (cloudClient
|
||||
.getZkStateReader()
|
||||
.getCloudState()
|
||||
.liveNodesContain(
|
||||
shardToJetty.get(SHARD2).get(0).info
|
||||
.get(ZkStateReader.NODE_NAME_PROP))) {
|
||||
if (tries++ == 60) {
|
||||
fail("Shard still reported as live in zk");
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
|
||||
// we are careful to make sure the downed node is no longer in the state,
|
||||
// because on some systems (especially freebsd w/ blackhole enabled), trying
|
||||
// to talk to a downed node causes grief
|
||||
Set<CloudJettyRunner> jetties = new HashSet<CloudJettyRunner>();
|
||||
jetties.addAll(shardToJetty.get(SHARD2));
|
||||
jetties.remove(deadShard);
|
||||
|
||||
for (CloudJettyRunner cjetty : jetties) {
|
||||
waitToSeeNotLive(((SolrDispatchFilter) cjetty.jetty.getDispatchFilter()
|
||||
.getFilter()).getCores().getZkController().getZkStateReader(),
|
||||
deadShard);
|
||||
}
|
||||
|
||||
waitToSeeNotLive(cloudClient.getZkStateReader(), deadShard);
|
||||
|
||||
// ensure shard is dead
|
||||
try {
|
||||
index_specific(deadShard.client.solrClient, id, 999, i1, 107, t1,
|
||||
|
@ -715,25 +717,7 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
// System.out.println("clouddocs:" + cloudClientDocs);
|
||||
|
||||
// try to index to a living shard at shard2
|
||||
|
||||
// we are careful to make sure the downed node is no longer in the state,
|
||||
// because on some systems (especially freebsd w/ blackhole enabled), trying
|
||||
// to talk to a downed node causes grief
|
||||
tries = 0;
|
||||
while (((SolrDispatchFilter) shardToJetty.get(SHARD2).get(1).jetty
|
||||
.getDispatchFilter().getFilter())
|
||||
.getCores()
|
||||
.getZkController()
|
||||
.getZkStateReader()
|
||||
.getCloudState()
|
||||
.liveNodesContain(
|
||||
shardToJetty.get(SHARD2).get(0).info
|
||||
.get(ZkStateReader.NODE_NAME_PROP))) {
|
||||
if (tries++ == 120) {
|
||||
fail("Shard still reported as live in zk");
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
|
||||
|
||||
long numFound1 = cloudClient.query(new SolrQuery("*:*")).getResults().getNumFound();
|
||||
|
||||
|
@ -812,6 +796,13 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
// recover over 100 docs so we do more than just peer sync (replicate recovery)
|
||||
chaosMonkey.stopJetty(deadShard);
|
||||
|
||||
for (CloudJettyRunner cjetty : jetties) {
|
||||
waitToSeeNotLive(((SolrDispatchFilter) cjetty.jetty.getDispatchFilter()
|
||||
.getFilter()).getCores().getZkController().getZkStateReader(),
|
||||
deadShard);
|
||||
}
|
||||
waitToSeeNotLive(cloudClient.getZkStateReader(), deadShard);
|
||||
|
||||
for (int i = 0; i < 226; i++) {
|
||||
doc = new SolrInputDocument();
|
||||
doc.addField("id", 2000 + i);
|
||||
|
@ -1496,4 +1487,16 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
protected void waitToSeeNotLive(ZkStateReader zkStateReader,
|
||||
CloudJettyRunner cjetty) throws InterruptedException {
|
||||
int tries = 0;
|
||||
while (zkStateReader.getCloudState()
|
||||
.liveNodesContain(cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) {
|
||||
if (tries++ == 120) {
|
||||
fail("Shard still reported as live in zk");
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -143,9 +143,7 @@ public class SyncSliceTest extends FullSolrCloudTest {
|
|||
CloudJettyRunner leaderJetty = shardToLeaderJetty.get("shard1");
|
||||
|
||||
Set<CloudJettyRunner> jetties = new HashSet<CloudJettyRunner>();
|
||||
for (int i = 0; i < shardCount; i++) {
|
||||
jetties.add(shardToJetty.get("shard1").get(i));
|
||||
}
|
||||
jetties.addAll(shardToJetty.get("shard1"));
|
||||
jetties.remove(leaderJetty);
|
||||
|
||||
chaosMonkey.killJetty(leaderJetty);
|
||||
|
@ -168,18 +166,6 @@ public class SyncSliceTest extends FullSolrCloudTest {
|
|||
assertEquals(5, cloudClientDocs);
|
||||
}
|
||||
|
||||
private void waitToSeeNotLive(ZkStateReader zkStateReader,
|
||||
CloudJettyRunner cjetty) throws InterruptedException {
|
||||
int tries = 0;
|
||||
while (zkStateReader.getCloudState()
|
||||
.liveNodesContain(cjetty.info.get(ZkStateReader.NODE_NAME_PROP))) {
|
||||
if (tries++ == 120) {
|
||||
fail("Shard still reported as live in zk");
|
||||
}
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
}
|
||||
|
||||
private void waitForThingsToLevelOut() throws Exception {
|
||||
int cnt = 0;
|
||||
boolean retry = false;
|
||||
|
|
Loading…
Reference in New Issue