mirror of https://github.com/apache/lucene.git
add a StopableSearchThread
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1365313 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ab2a7e389f
commit
1ff7e05b8f
|
@ -87,7 +87,7 @@ public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
|
|||
// as it's not supported for recovery
|
||||
// del("*:*");
|
||||
|
||||
List<StopableIndexingThread> threads = new ArrayList<StopableIndexingThread>();
|
||||
List<StopableThread> threads = new ArrayList<StopableThread>();
|
||||
int threadCount = 1;
|
||||
int i = 0;
|
||||
for (i = 0; i < threadCount; i++) {
|
||||
|
@ -97,6 +97,14 @@ public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
|
|||
indexThread.start();
|
||||
}
|
||||
|
||||
threadCount = 1;
|
||||
i = 0;
|
||||
for (i = 0; i < threadCount; i++) {
|
||||
StopableSearchThread searchThread = new StopableSearchThread();
|
||||
threads.add(searchThread);
|
||||
searchThread.start();
|
||||
}
|
||||
|
||||
FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread(
|
||||
clients, i * 50000, true);
|
||||
threads.add(ftIndexThread);
|
||||
|
@ -110,12 +118,12 @@ public class ChaosMonkeyNothingIsSafeTest extends FullSolrCloudTest {
|
|||
chaosMonkey.stopTheMonkey();
|
||||
}
|
||||
|
||||
for (StopableIndexingThread indexThread : threads) {
|
||||
for (StopableThread indexThread : threads) {
|
||||
indexThread.safeStop();
|
||||
}
|
||||
|
||||
// wait for stop...
|
||||
for (StopableIndexingThread indexThread : threads) {
|
||||
for (StopableThread indexThread : threads) {
|
||||
indexThread.join();
|
||||
}
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import java.util.HashMap;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
|
@ -1283,7 +1284,14 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
return rsp;
|
||||
}
|
||||
|
||||
class StopableIndexingThread extends Thread {
|
||||
abstract class StopableThread extends Thread {
|
||||
public StopableThread(String name) {
|
||||
super(name);
|
||||
}
|
||||
public abstract void safeStop();
|
||||
}
|
||||
|
||||
class StopableIndexingThread extends StopableThread {
|
||||
private volatile boolean stop = false;
|
||||
protected final int startI;
|
||||
protected final List<Integer> deletes = new ArrayList<Integer>();
|
||||
|
@ -1357,6 +1365,55 @@ public class FullSolrCloudTest extends AbstractDistributedZkTestCase {
|
|||
|
||||
};
|
||||
|
||||
class StopableSearchThread extends StopableThread {
|
||||
private volatile boolean stop = false;
|
||||
protected final AtomicInteger fails = new AtomicInteger();
|
||||
private String[] QUERIES = new String[] {"to come","their country","aid","co*"};
|
||||
|
||||
public StopableSearchThread() {
|
||||
super("StopableSearchThread");
|
||||
setDaemon(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
Random random = random();
|
||||
int numSearches = 0;
|
||||
|
||||
while (true && !stop) {
|
||||
numSearches++;
|
||||
try {
|
||||
//to come to the aid of their country.
|
||||
cloudClient.query(new SolrQuery(QUERIES[random.nextInt(QUERIES.length)]));
|
||||
} catch (Exception e) {
|
||||
System.err.println("QUERY REQUEST FAILED:");
|
||||
e.printStackTrace();
|
||||
if (e instanceof SolrServerException) {
|
||||
System.err.println("ROOT CAUSE:");
|
||||
((SolrServerException) e).getRootCause().printStackTrace();
|
||||
}
|
||||
fails.incrementAndGet();
|
||||
}
|
||||
try {
|
||||
Thread.sleep(random.nextInt(4000) + 300);
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
System.err.println("num searches done:" + numSearches + " with " + fails + " fails");
|
||||
}
|
||||
|
||||
public void safeStop() {
|
||||
stop = true;
|
||||
}
|
||||
|
||||
public int getFails() {
|
||||
return fails.get();
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
protected void waitForThingsToLevelOut(int waitForRecTimeSeconds) throws Exception {
|
||||
log.info("Wait for recoveries to finish - wait " + waitForRecTimeSeconds + " for each attempt");
|
||||
int cnt = 0;
|
||||
|
|
Loading…
Reference in New Issue