SOLR-7291: Test indexing on ZK disconnect with ChaosMonkey tests

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1669026 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Ramkumar Aiyengar 2015-03-25 00:03:33 +00:00
parent f678ac5d9d
commit 83c0c952b6
10 changed files with 164 additions and 121 deletions

View File

@ -387,12 +387,14 @@ Other Changes
* SOLR-6673: MDC based logging of collection, shard, replica, core * SOLR-6673: MDC based logging of collection, shard, replica, core
(Ishan Chattopadhyaya , Noble Paul) (Ishan Chattopadhyaya , Noble Paul)
* SOLR-7291: Test indexing on ZK disconnect with ChaosMonkey tests (Ramkumar Aiyengar)
================== 5.0.0 ================== ================== 5.0.0 ==================
Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release. Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.
NOTE: Solr 5.0 only supports creating and removing SolrCloud collections through NOTE: Solr 5.0 only supports creating and removing SolrCloud collections through
the collections API, unlike previous versions. While not using the the collections API, unlike previous versions. While not using the
collections API may still work in 5.0, it is unsupported, not recommended, collections API may still work in 5.0, it is unsupported, not recommended,
and the behavior will change in a 5.x release. and the behavior will change in a 5.x release.

View File

@ -115,11 +115,15 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
numShards = sliceCount + random().nextInt(TEST_NIGHTLY ? 12 : 2) + 1; numShards = sliceCount + random().nextInt(TEST_NIGHTLY ? 12 : 2) + 1;
} }
fixShardCount(numShards); fixShardCount(numShards);
// None of the operations used here are particularly costly, so this should work.
// Using this low timeout will also help us catch index stalling.
clientSoTimeout = 5000;
} }
@Test @Test
public void test() throws Exception { public void test() throws Exception {
boolean testsSuccesful = false; boolean testSuccessful = false;
try { try {
handle.clear(); handle.clear();
handle.put("timestamp", SKIPVAL); handle.put("timestamp", SKIPVAL);
@ -133,14 +137,14 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
// we cannot do delete by query // we cannot do delete by query
// as it's not supported for recovery // as it's not supported for recovery
del("*:*"); del("*:*");
List<StopableThread> threads = new ArrayList<>(); List<StoppableThread> threads = new ArrayList<>();
List<StopableIndexingThread> indexTreads = new ArrayList<>(); List<StoppableIndexingThread> indexTreads = new ArrayList<>();
int threadCount = TEST_NIGHTLY ? 3 : 1; int threadCount = TEST_NIGHTLY ? 3 : 1;
int i = 0; int i = 0;
for (i = 0; i < threadCount; i++) { for (i = 0; i < threadCount; i++) {
StopableIndexingThread indexThread = new StopableIndexingThread(controlClient, cloudClient, Integer.toString(i), true); StoppableIndexingThread indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true);
threads.add(indexThread); threads.add(indexThread);
indexTreads.add(indexThread); indexTreads.add(indexThread);
indexThread.start(); indexThread.start();
@ -149,7 +153,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
threadCount = 1; threadCount = 1;
i = 0; i = 0;
for (i = 0; i < threadCount; i++) { for (i = 0; i < threadCount; i++) {
StopableSearchThread searchThread = new StopableSearchThread(); StoppableSearchThread searchThread = new StoppableSearchThread(cloudClient);
threads.add(searchThread); threads.add(searchThread);
searchThread.start(); searchThread.start();
} }
@ -158,7 +162,7 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
// it's currently hard to know what requests failed when using ConcurrentSolrUpdateServer // it's currently hard to know what requests failed when using ConcurrentSolrUpdateServer
boolean runFullThrottle = random().nextBoolean(); boolean runFullThrottle = random().nextBoolean();
if (runFullThrottle) { if (runFullThrottle) {
FullThrottleStopableIndexingThread ftIndexThread = new FullThrottleStopableIndexingThread( FullThrottleStoppableIndexingThread ftIndexThread = new FullThrottleStoppableIndexingThread(
clients, "ft1", true); clients, "ft1", true);
threads.add(ftIndexThread); threads.add(ftIndexThread);
ftIndexThread.start(); ftIndexThread.start();
@ -184,15 +188,18 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
} finally { } finally {
chaosMonkey.stopTheMonkey(); chaosMonkey.stopTheMonkey();
} }
for (StopableThread indexThread : threads) { // ideally this should go into chaosMonkey
restartZk(1000 * (5 + random().nextInt(4)));
for (StoppableThread indexThread : threads) {
indexThread.safeStop(); indexThread.safeStop();
} }
// start any downed jetties to be sure we still will end up with a leader per shard... // start any downed jetties to be sure we still will end up with a leader per shard...
// wait for stop... // wait for stop...
for (StopableThread indexThread : threads) { for (StoppableThread indexThread : threads) {
indexThread.join(); indexThread.join();
} }
@ -217,9 +224,11 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
// we expect full throttle fails, but cloud client should not easily fail // we expect full throttle fails, but cloud client should not easily fail
for (StopableThread indexThread : threads) { for (StoppableThread indexThread : threads) {
if (indexThread instanceof StopableIndexingThread && !(indexThread instanceof FullThrottleStopableIndexingThread)) { if (indexThread instanceof StoppableIndexingThread && !(indexThread instanceof FullThrottleStoppableIndexingThread)) {
assertFalse("There were too many update fails - we expect it can happen, but shouldn't easily", ((StopableIndexingThread) indexThread).getFailCount() > FAIL_TOLERANCE); int failCount = ((StoppableIndexingThread) indexThread).getFailCount();
assertFalse("There were too many update fails (" + failCount + " > " + FAIL_TOLERANCE
+ ") - we expect it can happen, but shouldn't easily", failCount > FAIL_TOLERANCE);
} }
} }
@ -247,11 +256,8 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
// sometimes we restart zookeeper as well // sometimes we restart zookeeper as well
if (random().nextBoolean()) { if (random().nextBoolean()) {
zkServer.shutdown(); restartZk(1000 * (5 + random().nextInt(4)));
zkServer = new ZkTestServer(zkServer.getZkDir(), zkServer.getPort());
zkServer.run();
} }
try (CloudSolrClient client = createCloudClient("collection1")) { try (CloudSolrClient client = createCloudClient("collection1")) {
createCollection(null, "testcollection", createCollection(null, "testcollection",
@ -263,31 +269,31 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
numShardsNumReplicas.add(1); numShardsNumReplicas.add(1);
checkForCollection("testcollection", numShardsNumReplicas, null); checkForCollection("testcollection", numShardsNumReplicas, null);
testsSuccesful = true; testSuccessful = true;
} finally { } finally {
if (!testsSuccesful) { if (!testSuccessful) {
printLayout(); printLayout();
} }
} }
} }
private Set<String> getAddFails(List<StopableIndexingThread> threads) { private Set<String> getAddFails(List<StoppableIndexingThread> threads) {
Set<String> addFails = new HashSet<String>(); Set<String> addFails = new HashSet<String>();
for (StopableIndexingThread thread : threads) { for (StoppableIndexingThread thread : threads) {
addFails.addAll(thread.getAddFails()); addFails.addAll(thread.getAddFails());
} }
return addFails; return addFails;
} }
private Set<String> getDeleteFails(List<StopableIndexingThread> threads) { private Set<String> getDeleteFails(List<StoppableIndexingThread> threads) {
Set<String> deleteFails = new HashSet<String>(); Set<String> deleteFails = new HashSet<String>();
for (StopableIndexingThread thread : threads) { for (StoppableIndexingThread thread : threads) {
deleteFails.addAll(thread.getDeleteFails()); deleteFails.addAll(thread.getDeleteFails());
} }
return deleteFails; return deleteFails;
} }
class FullThrottleStopableIndexingThread extends StopableIndexingThread { class FullThrottleStoppableIndexingThread extends StoppableIndexingThread {
private CloseableHttpClient httpClient = HttpClientUtil.createClient(null); private CloseableHttpClient httpClient = HttpClientUtil.createClient(null);
private volatile boolean stop = false; private volatile boolean stop = false;
int clientIndex = 0; int clientIndex = 0;
@ -295,14 +301,14 @@ public class ChaosMonkeyNothingIsSafeTest extends AbstractFullDistribZkTestBase
private List<SolrClient> clients; private List<SolrClient> clients;
private AtomicInteger fails = new AtomicInteger(); private AtomicInteger fails = new AtomicInteger();
public FullThrottleStopableIndexingThread(List<SolrClient> clients, public FullThrottleStoppableIndexingThread(List<SolrClient> clients,
String id, boolean doDeletes) { String id, boolean doDeletes) {
super(controlClient, cloudClient, id, doDeletes); super(controlClient, cloudClient, id, doDeletes);
setName("FullThrottleStopableIndexingThread"); setName("FullThrottleStopableIndexingThread");
setDaemon(true); setDaemon(true);
this.clients = clients; this.clients = clients;
HttpClientUtil.setConnectionTimeout(httpClient, 15000); HttpClientUtil.setConnectionTimeout(httpClient, clientConnectionTimeout);
HttpClientUtil.setSoTimeout(httpClient, 15000); HttpClientUtil.setSoTimeout(httpClient, clientSoTimeout);
cusc = new ConcurrentUpdateSolrClient( cusc = new ConcurrentUpdateSolrClient(
((HttpSolrClient) clients.get(0)).getBaseURL(), httpClient, 8, ((HttpSolrClient) clients.get(0)).getBaseURL(), httpClient, 8,
2) { 2) {

View File

@ -110,7 +110,7 @@ public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
tryDelete(); tryDelete();
List<StopableIndexingThread> threads = new ArrayList<>(); List<StoppableIndexingThread> threads = new ArrayList<>();
int threadCount = 2; int threadCount = 2;
int batchSize = 1; int batchSize = 1;
if (random().nextBoolean()) { if (random().nextBoolean()) {
@ -126,7 +126,7 @@ public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
} }
for (int i = 0; i < threadCount; i++) { for (int i = 0; i < threadCount; i++) {
StopableIndexingThread indexThread = new StopableIndexingThread(controlClient, cloudClient, Integer.toString(i), true, maxUpdates, batchSize, pauseBetweenUpdates); // random().nextInt(999) + 1 StoppableIndexingThread indexThread = new StoppableIndexingThread(controlClient, cloudClient, Integer.toString(i), true, maxUpdates, batchSize, pauseBetweenUpdates); // random().nextInt(999) + 1
threads.add(indexThread); threads.add(indexThread);
indexThread.start(); indexThread.start();
} }
@ -152,16 +152,16 @@ public class ChaosMonkeySafeLeaderTest extends AbstractFullDistribZkTestBase {
chaosMonkey.stopTheMonkey(); chaosMonkey.stopTheMonkey();
} }
for (StopableIndexingThread indexThread : threads) { for (StoppableIndexingThread indexThread : threads) {
indexThread.safeStop(); indexThread.safeStop();
} }
// wait for stop... // wait for stop...
for (StopableIndexingThread indexThread : threads) { for (StoppableIndexingThread indexThread : threads) {
indexThread.join(); indexThread.join();
} }
for (StopableIndexingThread indexThread : threads) { for (StoppableIndexingThread indexThread : threads) {
assertEquals(0, indexThread.getFailCount()); assertEquals(0, indexThread.getFailCount());
} }

View File

@ -34,8 +34,8 @@ public class RecoveryZkTest extends AbstractFullDistribZkTestBase {
//private static final String DISTRIB_UPDATE_CHAIN = "distrib-update-chain"; //private static final String DISTRIB_UPDATE_CHAIN = "distrib-update-chain";
private static Logger log = LoggerFactory.getLogger(RecoveryZkTest.class); private static Logger log = LoggerFactory.getLogger(RecoveryZkTest.class);
private StopableIndexingThread indexThread; private StoppableIndexingThread indexThread;
private StopableIndexingThread indexThread2; private StoppableIndexingThread indexThread2;
public RecoveryZkTest() { public RecoveryZkTest() {
super(); super();
@ -72,11 +72,11 @@ public class RecoveryZkTest extends AbstractFullDistribZkTestBase {
maxDoc = maxDocNightlyList[random().nextInt(maxDocList.length - 1)]; maxDoc = maxDocNightlyList[random().nextInt(maxDocList.length - 1)];
} }
indexThread = new StopableIndexingThread(controlClient, cloudClient, "1", true, maxDoc, 1, true); indexThread = new StoppableIndexingThread(controlClient, cloudClient, "1", true, maxDoc, 1, true);
indexThread.start(); indexThread.start();
indexThread2 = new StopableIndexingThread(controlClient, cloudClient, "2", true, maxDoc, 1, true); indexThread2 = new StoppableIndexingThread(controlClient, cloudClient, "2", true, maxDoc, 1, true);
indexThread2.start(); indexThread2.start();
// give some time to index... // give some time to index...

View File

@ -29,7 +29,7 @@ import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.embedded.JettySolrRunner; import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient; import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.cloud.BasicDistributedZkTest; import org.apache.solr.cloud.BasicDistributedZkTest;
import org.apache.solr.cloud.StopableIndexingThread; import org.apache.solr.cloud.StoppableIndexingThread;
import org.apache.solr.core.CoreContainer; import org.apache.solr.core.CoreContainer;
import org.apache.solr.core.HdfsDirectoryFactory; import org.apache.solr.core.HdfsDirectoryFactory;
import org.apache.solr.core.SolrCore; import org.apache.solr.core.SolrCore;
@ -100,18 +100,18 @@ public class HdfsWriteToMultipleCollectionsTest extends BasicDistributedZkTest {
waitForRecoveriesToFinish(ACOLLECTION + i, false); waitForRecoveriesToFinish(ACOLLECTION + i, false);
} }
List<CloudSolrClient> cloudClients = new ArrayList<>(); List<CloudSolrClient> cloudClients = new ArrayList<>();
List<StopableIndexingThread> threads = new ArrayList<>(); List<StoppableIndexingThread> threads = new ArrayList<>();
for (int i = 0; i < cnt; i++) { for (int i = 0; i < cnt; i++) {
CloudSolrClient client = new CloudSolrClient(zkServer.getZkAddress()); CloudSolrClient client = new CloudSolrClient(zkServer.getZkAddress());
client.setDefaultCollection(ACOLLECTION + i); client.setDefaultCollection(ACOLLECTION + i);
cloudClients.add(client); cloudClients.add(client);
StopableIndexingThread indexThread = new StopableIndexingThread(null, client, "1", true, docCount, 1, true); StoppableIndexingThread indexThread = new StoppableIndexingThread(null, client, "1", true, docCount, 1, true);
threads.add(indexThread); threads.add(indexThread);
indexThread.start(); indexThread.start();
} }
int addCnt = 0; int addCnt = 0;
for (StopableIndexingThread thread : threads) { for (StoppableIndexingThread thread : threads) {
thread.join(); thread.join();
addCnt += thread.getNumAdds() - thread.getNumDeletes(); addCnt += thread.getNumAdds() - thread.getNumDeletes();
} }

View File

@ -230,6 +230,8 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
protected boolean verifyStress = true; protected boolean verifyStress = true;
protected int nThreads = 3; protected int nThreads = 3;
protected int clientConnectionTimeout = DEFAULT_CONNECTION_TIMEOUT;
protected int clientSoTimeout = 90000;
public static int ORDERED = 1; public static int ORDERED = 1;
public static int SKIP = 2; public static int SKIP = 2;
@ -435,8 +437,8 @@ public abstract class BaseDistributedSearchTestCase extends SolrTestCaseJ4 {
try { try {
// setup the client... // setup the client...
HttpSolrClient client = new HttpSolrClient(buildUrl(port) + "/" + DEFAULT_TEST_CORENAME); HttpSolrClient client = new HttpSolrClient(buildUrl(port) + "/" + DEFAULT_TEST_CORENAME);
client.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT); client.setConnectionTimeout(clientConnectionTimeout);
client.setSoTimeout(90000); client.setSoTimeout(clientSoTimeout);
client.setDefaultMaxConnectionsPerHost(100); client.setDefaultMaxConnectionsPerHost(100);
client.setMaxTotalConnections(100); client.setMaxTotalConnections(100);
return client; return client;

View File

@ -232,4 +232,13 @@ public abstract class AbstractDistribZkTestBase extends BaseDistributedSearchTes
zkClient.printLayoutToStdOut(); zkClient.printLayoutToStdOut();
zkClient.close(); zkClient.close();
} }
protected void restartZk(int pauseMillis) throws Exception {
log.info("Restarting ZK with a pause of {}ms in between", pauseMillis);
zkServer.shutdown();
// disconnect enough to test stalling, if things stall, then clientSoTimeout w""ill be hit
Thread.sleep(pauseMillis);
zkServer = new ZkTestServer(zkServer.getZkDir(), zkServer.getPort());
zkServer.run();
}
} }

View File

@ -77,7 +77,6 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Properties; import java.util.Properties;
import java.util.Random;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -128,7 +127,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
protected boolean checkCreatedVsState; protected boolean checkCreatedVsState;
protected boolean useJettyDataDir = true; protected boolean useJettyDataDir = true;
protected Map<URI,SocketProxy> proxies = new HashMap<URI,SocketProxy>(); protected Map<URI,SocketProxy> proxies = new HashMap<>();
public static class CloudJettyRunner { public static class CloudJettyRunner {
public JettySolrRunner jetty; public JettySolrRunner jetty;
@ -207,14 +206,11 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
if (isSSLMode()) { if (isSSLMode()) {
System.clearProperty("urlScheme"); System.clearProperty("urlScheme");
ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(), try (ZkStateReader zkStateReader = new ZkStateReader(zkServer.getZkAddress(),
AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT); AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT)) {
try {
zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS, zkStateReader.getZkClient().create(ZkStateReader.CLUSTER_PROPS,
ZkStateReader.toJSON(Collections.singletonMap("urlScheme","https")), ZkStateReader.toJSON(Collections.singletonMap("urlScheme", "https")),
CreateMode.PERSISTENT, true); CreateMode.PERSISTENT, true);
} finally {
zkStateReader.close();
} }
} }
} }
@ -1386,16 +1382,13 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
.getNumFound(); .getNumFound();
// do some really inefficient mapping... // do some really inefficient mapping...
ZkStateReader zk = new ZkStateReader(zkServer.getZkAddress(), 10000,
AbstractZkTestCase.TIMEOUT);
Map<String,Slice> slices = null; Map<String,Slice> slices = null;
ClusterState clusterState; ClusterState clusterState;
try { try (ZkStateReader zk = new ZkStateReader(zkServer.getZkAddress(), 10000,
AbstractZkTestCase.TIMEOUT)) {
zk.createClusterStateWatchersAndUpdate(); zk.createClusterStateWatchersAndUpdate();
clusterState = zk.getClusterState(); clusterState = zk.getClusterState();
slices = clusterState.getSlicesMap(DEFAULT_COLLECTION); slices = clusterState.getSlicesMap(DEFAULT_COLLECTION);
} finally {
zk.close();
} }
if (slices == null) { if (slices == null) {
@ -1445,67 +1438,16 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
if (r.nextBoolean()) params.set("collection", DEFAULT_COLLECTION); if (r.nextBoolean()) params.set("collection", DEFAULT_COLLECTION);
QueryResponse rsp = cloudClient.query(params); return cloudClient.query(params);
return rsp;
} }
static abstract class StopableThread extends Thread { static abstract class StoppableThread extends Thread {
public StopableThread(String name) { public StoppableThread(String name) {
super(name); super(name);
} }
public abstract void safeStop(); public abstract void safeStop();
} }
class StopableSearchThread extends StopableThread {
private volatile boolean stop = false;
protected final AtomicInteger queryFails = 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();
}
queryFails.incrementAndGet();
}
try {
Thread.sleep(random.nextInt(4000) + 300);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
log.info("num searches done:" + numSearches + " with " + queryFails + " fails");
}
@Override
public void safeStop() {
stop = true;
}
public int getFails() {
return queryFails.get();
}
};
public void waitForThingsToLevelOut(int waitForRecTimeSeconds) throws Exception { public void waitForThingsToLevelOut(int waitForRecTimeSeconds) throws Exception {
log.info("Wait for recoveries to finish - wait " + waitForRecTimeSeconds + " for each attempt"); log.info("Wait for recoveries to finish - wait " + waitForRecTimeSeconds + " for each attempt");
int cnt = 0; int cnt = 0;
@ -1934,14 +1876,14 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
REPLICATION_FACTOR, replicationFactor, REPLICATION_FACTOR, replicationFactor,
MAX_SHARDS_PER_NODE, maxShardsPerNode, MAX_SHARDS_PER_NODE, maxShardsPerNode,
NUM_SLICES, numShards); NUM_SLICES, numShards);
Map<String,List<Integer>> collectionInfos = new HashMap<String,List<Integer>>(); Map<String,List<Integer>> collectionInfos = new HashMap<>();
createCollection(collectionInfos, collName, props, client); createCollection(collectionInfos, collName, props, client);
} }
protected List<Replica> ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception { protected List<Replica> ensureAllReplicasAreActive(String testCollectionName, String shardId, int shards, int rf, int maxWaitSecs) throws Exception {
long startMs = System.currentTimeMillis(); long startMs = System.currentTimeMillis();
Map<String,Replica> notLeaders = new HashMap<String,Replica>(); Map<String,Replica> notLeaders = new HashMap<>();
ZkStateReader zkr = cloudClient.getZkStateReader(); ZkStateReader zkr = cloudClient.getZkStateReader();
zkr.updateClusterState(true); // force the state to be fresh zkr.updateClusterState(true); // force the state to be fresh
@ -2001,7 +1943,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
long diffMs = (System.currentTimeMillis() - startMs); long diffMs = (System.currentTimeMillis() - startMs);
log.info("Took " + diffMs + " ms to see all replicas become active."); log.info("Took " + diffMs + " ms to see all replicas become active.");
List<Replica> replicas = new ArrayList<Replica>(); List<Replica> replicas = new ArrayList<>();
replicas.addAll(notLeaders.values()); replicas.addAll(notLeaders.values());
return replicas; return replicas;
} }
@ -2017,7 +1959,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
if (collection != null) { if (collection != null) {
cs = clusterState.getCollection(collection).toString(); cs = clusterState.getCollection(collection).toString();
} else { } else {
Map<String,DocCollection> map = new HashMap<String,DocCollection>(); Map<String,DocCollection> map = new HashMap<>();
for (String coll : clusterState.getCollections()) for (String coll : clusterState.getCollections())
map.put(coll, clusterState.getCollection(coll)); map.put(coll, clusterState.getCollection(coll));
CharArr out = new CharArr(); CharArr out = new CharArr();

View File

@ -28,7 +28,7 @@ import org.apache.solr.common.SolrInputDocument;
* limitations under the License. * limitations under the License.
*/ */
public class StopableIndexingThread extends AbstractFullDistribZkTestBase.StopableThread { public class StoppableIndexingThread extends AbstractFullDistribZkTestBase.StoppableThread {
static String t1 = "a_t"; static String t1 = "a_t";
static String i1 = "a_i"; static String i1 = "a_i";
private volatile boolean stop = false; private volatile boolean stop = false;
@ -46,12 +46,12 @@ public class StopableIndexingThread extends AbstractFullDistribZkTestBase.Stopab
private int batchSize; private int batchSize;
private boolean pauseBetweenUpdates; private boolean pauseBetweenUpdates;
public StopableIndexingThread(SolrClient controlClient, SolrClient cloudClient, String id, boolean doDeletes) { public StoppableIndexingThread(SolrClient controlClient, SolrClient cloudClient, String id, boolean doDeletes) {
this(controlClient, cloudClient, id, doDeletes, -1, 1, true); this(controlClient, cloudClient, id, doDeletes, -1, 1, true);
} }
public StopableIndexingThread(SolrClient controlClient, SolrClient cloudClient, String id, boolean doDeletes, int numCycles, int batchSize, boolean pauseBetweenUpdates) { public StoppableIndexingThread(SolrClient controlClient, SolrClient cloudClient, String id, boolean doDeletes, int numCycles, int batchSize, boolean pauseBetweenUpdates) {
super("StopableIndexingThread"); super("StoppableIndexingThread");
this.controlClient = controlClient; this.controlClient = controlClient;
this.cloudClient = cloudClient; this.cloudClient = cloudClient;
this.id = id; this.id = id;

View File

@ -0,0 +1,82 @@
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.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
class StoppableSearchThread extends AbstractFullDistribZkTestBase.StoppableThread {
static Logger log = LoggerFactory.getLogger(AbstractFullDistribZkTestBase.class);
private final CloudSolrClient cloudClient;
private volatile boolean stop = false;
protected final AtomicInteger queryFails = new AtomicInteger();
private String[] QUERIES = new String[] {"to come","their country","aid","co*"};
public StoppableSearchThread(CloudSolrClient cloudClient) {
super("StoppableSearchThread");
this.cloudClient = cloudClient;
setDaemon(true);
}
@Override
public void run() {
Random random = LuceneTestCase.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();
}
queryFails.incrementAndGet();
}
try {
Thread.sleep(random.nextInt(4000) + 300);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
log.info("num searches done:" + numSearches + " with " + queryFails + " fails");
}
@Override
public void safeStop() {
stop = true;
}
public int getFails() {
return queryFails.get();
}
}