mirror of https://github.com/apache/lucene.git
SOLR-8279: Add a new SolrCloud test that stops and starts the cluster while indexing data.
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1714216 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
15ac4b0e83
commit
31da6a3535
|
@ -0,0 +1,148 @@
|
|||
package org.apache.solr.cloud;
|
||||
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.Slow;
|
||||
import org.apache.lucene.util.LuceneTestCase.Nightly;
|
||||
import org.apache.solr.client.solrj.SolrServerException;
|
||||
import org.apache.solr.common.SolrInputDocument;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@Slow
|
||||
@Nightly
|
||||
public class RestartWhileUpdatingTest extends AbstractFullDistribZkTestBase {
|
||||
|
||||
//private static final String DISTRIB_UPDATE_CHAIN = "distrib-update-chain";
|
||||
private static Logger log = LoggerFactory.getLogger(RestartWhileUpdatingTest.class);
|
||||
private List<StoppableIndexingThread> threads;
|
||||
|
||||
public RestartWhileUpdatingTest() {
|
||||
super();
|
||||
sliceCount = 1;
|
||||
fixShardCount(3);
|
||||
schemaString = "schema15.xml"; // we need a string id
|
||||
}
|
||||
|
||||
public static String[] fieldNames = new String[]{"f_i", "f_f", "f_d", "f_l", "f_dt"};
|
||||
public static RandVal[] randVals = new RandVal[]{rint, rfloat, rdouble, rlong, rdate};
|
||||
|
||||
protected String[] getFieldNames() {
|
||||
return fieldNames;
|
||||
}
|
||||
|
||||
protected RandVal[] getRandValues() {
|
||||
return randVals;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void test() throws Exception {
|
||||
handle.clear();
|
||||
handle.put("timestamp", SKIPVAL);
|
||||
|
||||
// start a couple indexing threads
|
||||
|
||||
int[] maxDocList = new int[] {5000, 10000};
|
||||
|
||||
|
||||
int maxDoc = maxDocList[random().nextInt(maxDocList.length - 1)];
|
||||
|
||||
int numThreads = random().nextInt(4) + 1;
|
||||
|
||||
threads = new ArrayList<>(2);
|
||||
|
||||
StoppableIndexingThread indexThread;
|
||||
for (int i = 0; i < numThreads; i++) {
|
||||
indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true, maxDoc, 1, true);
|
||||
threads.add(indexThread);
|
||||
indexThread.start();
|
||||
}
|
||||
|
||||
Thread.sleep(2000);
|
||||
|
||||
int restartTimes = random().nextInt(4) + 1;;
|
||||
for (int i = 0; i < restartTimes; i++) {
|
||||
stopAndStartAllReplicas();
|
||||
}
|
||||
|
||||
Thread.sleep(2000);
|
||||
|
||||
// stop indexing threads
|
||||
for (StoppableIndexingThread thread : threads) {
|
||||
thread.safeStop();
|
||||
thread.safeStop();
|
||||
}
|
||||
|
||||
Thread.sleep(1000);
|
||||
|
||||
waitForThingsToLevelOut(120);
|
||||
|
||||
Thread.sleep(2000);
|
||||
|
||||
waitForThingsToLevelOut(30);
|
||||
|
||||
Thread.sleep(5000);
|
||||
|
||||
waitForRecoveriesToFinish(DEFAULT_COLLECTION, cloudClient.getZkStateReader(), false, true);
|
||||
|
||||
|
||||
checkShardConsistency(false, false);
|
||||
}
|
||||
|
||||
public void stopAndStartAllReplicas() throws Exception, InterruptedException {
|
||||
chaosMonkey.stopAll(random().nextInt(2000));
|
||||
|
||||
Thread.sleep(1000);
|
||||
|
||||
chaosMonkey.startAll();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void indexDoc(SolrInputDocument doc) throws IOException,
|
||||
SolrServerException {
|
||||
cloudClient.add(doc);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void distribTearDown() throws Exception {
|
||||
// make sure threads have been stopped...
|
||||
if (threads != null) {
|
||||
for (StoppableIndexingThread thread : threads) {
|
||||
thread.safeStop();
|
||||
thread.safeStop();
|
||||
}
|
||||
}
|
||||
|
||||
super.distribTearDown();
|
||||
}
|
||||
|
||||
// skip the randoms - they can deadlock...
|
||||
@Override
|
||||
protected void indexr(Object... fields) throws Exception {
|
||||
SolrInputDocument doc = new SolrInputDocument();
|
||||
addFields(doc, fields);
|
||||
addFields(doc, "rnd_b", true);
|
||||
indexDoc(doc);
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ import java.util.ArrayList;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
@ -226,6 +227,27 @@ public class ChaosMonkey {
|
|||
kill(cjetty.jetty);
|
||||
}
|
||||
|
||||
public void stopAll(int pauseBetweenMs) throws Exception {
|
||||
Set<String> keys = shardToJetty.keySet();
|
||||
for (String key : keys) {
|
||||
List<CloudJettyRunner> jetties = shardToJetty.get(key);
|
||||
for (CloudJettyRunner jetty : jetties) {
|
||||
Thread.sleep(pauseBetweenMs);
|
||||
stopJetty(jetty);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void startAll() throws Exception {
|
||||
Set<String> keys = shardToJetty.keySet();
|
||||
for (String key : keys) {
|
||||
List<CloudJettyRunner> jetties = shardToJetty.get(key);
|
||||
for (CloudJettyRunner jetty : jetties) {
|
||||
start(jetty.jetty);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void stopShard(String slice) throws Exception {
|
||||
List<CloudJettyRunner> jetties = shardToJetty.get(slice);
|
||||
for (CloudJettyRunner jetty : jetties) {
|
||||
|
|
Loading…
Reference in New Issue