mirror of https://github.com/apache/lucene.git
wait for leaders on startup for slow/busy comps
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1296659 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
e9c9011737
commit
6a6c1ed04e
|
@ -27,9 +27,9 @@ import org.apache.solr.common.cloud.Slice;
|
|||
import org.apache.solr.common.cloud.SolrZkClient;
|
||||
import org.apache.solr.common.cloud.ZkNodeProps;
|
||||
import org.apache.solr.common.cloud.ZkStateReader;
|
||||
import org.apache.solr.servlet.SolrDispatchFilter;
|
||||
import org.apache.zookeeper.KeeperException;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
|
||||
public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearchTestCase {
|
||||
|
@ -80,6 +80,14 @@ public abstract class AbstractDistributedZkTestCase extends BaseDistributedSearc
|
|||
}
|
||||
|
||||
shards = sb.toString();
|
||||
|
||||
// now wait till we see the leader for each shard
|
||||
for (int i = 1; i <= numShards; i++) {
|
||||
ZkStateReader zkStateReader = ((SolrDispatchFilter) jettys.get(0)
|
||||
.getDispatchFilter().getFilter()).getCores().getZkController()
|
||||
.getZkStateReader();
|
||||
zkStateReader.getLeaderProps("collection1", "shard" + (i + 2), 15000);
|
||||
}
|
||||
}
|
||||
|
||||
protected void waitForRecoveriesToFinish(String collection, ZkStateReader zkStateReader, boolean verbose)
|
||||
|
|
|
@ -314,7 +314,7 @@ public class BasicDistributedZkTest extends AbstractDistributedZkTestCase {
|
|||
NamedList uhandler = (NamedList) uhandlerCat.get("updateHandler");
|
||||
NamedList stats = (NamedList) uhandler.get("stats");
|
||||
Long commits = (Long) stats.get("commits");
|
||||
System.out.println("resp:" + resp);
|
||||
|
||||
return commits;
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue