mirror of https://github.com/apache/lucene.git
SOLR-12923: Harden TestSimLargeCluster.
This commit is contained in:
parent
bf0e9b367a
commit
202dad8c88
|
@ -889,6 +889,7 @@ public class SimCloudManager implements SolrCloudManager {
|
|||
IOUtils.closeQuietly(stateManager);
|
||||
triggerThread.interrupt();
|
||||
IOUtils.closeQuietly(triggerThread);
|
||||
triggerThread.interrupt();
|
||||
try {
|
||||
triggerThread.join();
|
||||
} catch (InterruptedException e) {
|
||||
|
|
|
@ -1369,7 +1369,7 @@ public class SimClusterStateProvider implements ClusterStateProvider {
|
|||
OverseerCollectionMessageHandler.NUM_SLICES, "1",
|
||||
CommonAdminParams.WAIT_FOR_FINAL_STATE, "true");
|
||||
simCreateCollection(props, new NamedList());
|
||||
CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL, 90, TimeUnit.SECONDS,
|
||||
CloudTestUtils.waitForState(cloudManager, CollectionAdminParams.SYSTEM_COLL, 120, TimeUnit.SECONDS,
|
||||
CloudTestUtils.clusterShape(1, Integer.parseInt(repFactor), false, true));
|
||||
} catch (Exception e) {
|
||||
throw new IOException(e);
|
||||
|
|
|
@ -53,7 +53,6 @@ import org.apache.solr.cloud.autoscaling.TriggerListenerBase;
|
|||
import org.apache.solr.cloud.autoscaling.TriggerValidationException;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.apache.solr.common.cloud.Replica;
|
||||
import org.apache.solr.common.params.CollectionAdminParams;
|
||||
import org.apache.solr.common.params.CollectionParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.Pair;
|
||||
|
@ -78,7 +77,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
public static final int SPEED = 50;
|
||||
|
||||
public static final int NUM_NODES = 100;
|
||||
|
||||
|
||||
static Map<String, List<CapturedEvent>> listenerEvents = new ConcurrentHashMap<>();
|
||||
static AtomicInteger triggerFinishedCount = new AtomicInteger();
|
||||
static AtomicInteger triggerStartedCount = new AtomicInteger();
|
||||
|
@ -133,12 +132,6 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
// do this in advance if missing
|
||||
if (!cluster.getSimClusterStateProvider().simListCollections().contains(CollectionAdminParams.SYSTEM_COLL)) {
|
||||
cluster.getSimClusterStateProvider().createSystemCollection();
|
||||
CloudTestUtils.waitForState(cluster, CollectionAdminParams.SYSTEM_COLL, 120, TimeUnit.SECONDS,
|
||||
CloudTestUtils.clusterShape(1, 3, false, true));
|
||||
}
|
||||
}
|
||||
|
||||
public static class TestTriggerListener extends TriggerListenerBase {
|
||||
|
@ -310,7 +303,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
create.setAutoAddReplicas(false);
|
||||
create.process(solrClient);
|
||||
|
||||
log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 20 * NUM_NODES, TimeUnit.SECONDS,
|
||||
log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 90 * NUM_NODES, TimeUnit.SECONDS,
|
||||
CloudTestUtils.clusterShape(NUM_NODES / 10, NUM_NODES / 8 * 3, false, true)) + " ms");
|
||||
|
||||
// start adding nodes
|
||||
|
@ -325,7 +318,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
assertTrue("trigger did not fire", await);
|
||||
|
||||
// wait until started == finished
|
||||
TimeOut timeOut = new TimeOut(20 * waitForSeconds * NUM_NODES, TimeUnit.SECONDS, cluster.getTimeSource());
|
||||
TimeOut timeOut = new TimeOut(45 * waitForSeconds * NUM_NODES, TimeUnit.SECONDS, cluster.getTimeSource());
|
||||
while (!timeOut.hasTimedOut()) {
|
||||
if (triggerStartedCount.get() == triggerFinishedCount.get()) {
|
||||
break;
|
||||
|
@ -510,7 +503,7 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
create.setAutoAddReplicas(false);
|
||||
create.process(solrClient);
|
||||
|
||||
log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 30 * NUM_NODES, TimeUnit.SECONDS,
|
||||
log.info("Ready after " + CloudTestUtils.waitForState(cluster, collectionName, 60 * NUM_NODES, TimeUnit.SECONDS,
|
||||
CloudTestUtils.clusterShape(NUM_NODES / 5, NUM_NODES / 10, false, true)) + " ms");
|
||||
|
||||
// start killing nodes
|
||||
|
@ -701,12 +694,15 @@ public class TestSimLargeCluster extends SimSolrCloudTestCase {
|
|||
assertEquals(response.get("result").toString(), "success");
|
||||
|
||||
|
||||
boolean await = triggerFinishedLatch.await(waitForSeconds * 20000 / SPEED, TimeUnit.MILLISECONDS);
|
||||
boolean await = triggerFinishedLatch.await(waitForSeconds * 45000 / SPEED, TimeUnit.MILLISECONDS);
|
||||
assertTrue("The trigger did not fire at all", await);
|
||||
// wait for listener to capture the SUCCEEDED stage
|
||||
cluster.getTimeSource().sleep(5000);
|
||||
assertNotNull(listenerEvents.toString(), listenerEvents.get("srt"));
|
||||
assertEquals(listenerEvents.toString(), 1, listenerEvents.get("srt").size());
|
||||
cluster.getTimeSource().sleep(15000);
|
||||
|
||||
assertNotNull(listenerEvents.entrySet().toString(), listenerEvents.get("srt"));
|
||||
|
||||
assertTrue(listenerEvents.toString(), listenerEvents.get("srt").size() >= 1);
|
||||
|
||||
CapturedEvent ev = listenerEvents.get("srt").get(0);
|
||||
assertEquals(TriggerEventType.SEARCHRATE, ev.event.getEventType());
|
||||
Map<String, Number> m = (Map<String, Number>)ev.event.getProperty(SearchRateTrigger.HOT_NODES);
|
||||
|
|
Loading…
Reference in New Issue