mirror of https://github.com/apache/lucene.git
SOLR-6988 Make stateformat=2 as default
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1653867 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
02dfdd3202
commit
ec01b5aeb5
|
@ -853,6 +853,9 @@ public class CollectionsHandler extends RequestHandlerBase {
|
|||
DocCollection.STATE_FORMAT,
|
||||
AUTO_ADD_REPLICAS,
|
||||
"router.");
|
||||
if(props.get(DocCollection.STATE_FORMAT) == null){
|
||||
props.put(DocCollection.STATE_FORMAT,"2");
|
||||
}
|
||||
|
||||
if(SYSTEM_COLL.equals(name)){
|
||||
//We must always create asystem collection with only a single shard
|
||||
|
|
|
@ -77,8 +77,8 @@ public class ExternalCollectionsTest extends AbstractFullDistribZkTestBase {
|
|||
|
||||
|
||||
@Override
|
||||
protected int getStateFormat() {
|
||||
return 2;
|
||||
protected String getStateFormat() {
|
||||
return "2";
|
||||
}
|
||||
|
||||
private void testConfNameAndCollectionNameSame() throws Exception{
|
||||
|
|
|
@ -353,16 +353,12 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
return createJettys(numJettys, false);
|
||||
}
|
||||
|
||||
protected int defaultStateFormat = 1 + random().nextInt(2);
|
||||
protected String defaultStateFormat = String.valueOf( 1 + random().nextInt(2));
|
||||
|
||||
protected int getStateFormat() {
|
||||
protected String getStateFormat() {
|
||||
String stateFormat = System.getProperty("tests.solr.stateFormat", null);
|
||||
if (stateFormat != null) {
|
||||
if ("2".equals(stateFormat)) {
|
||||
return defaultStateFormat = 2;
|
||||
} else if ("1".equals(stateFormat)) {
|
||||
return defaultStateFormat = 1;
|
||||
}
|
||||
defaultStateFormat = stateFormat;
|
||||
}
|
||||
return defaultStateFormat; // random
|
||||
}
|
||||
|
@ -378,7 +374,7 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
List<SolrClient> clients = new ArrayList<>();
|
||||
StringBuilder sb = new StringBuilder();
|
||||
|
||||
if (getStateFormat() == 2) {
|
||||
if ("2".equals(getStateFormat())) {
|
||||
log.info("Creating collection1 with stateFormat=2");
|
||||
SolrZkClient zkClient = new SolrZkClient(zkServer.getZkAddress(),
|
||||
AbstractZkTestCase.TIMEOUT, AbstractZkTestCase.TIMEOUT);
|
||||
|
@ -1623,9 +1619,9 @@ public abstract class AbstractFullDistribZkTestBase extends AbstractDistribZkTes
|
|||
collectionInfos.put(collectionName, list);
|
||||
}
|
||||
params.set("name", collectionName);
|
||||
if (getStateFormat() == 2) {
|
||||
log.info("Creating collection with stateFormat=2: " + collectionName);
|
||||
params.set(DocCollection.STATE_FORMAT, "2");
|
||||
if ("1".equals(getStateFormat()) ) {
|
||||
log.info("Creating collection with stateFormat=1: " + collectionName);
|
||||
params.set(DocCollection.STATE_FORMAT, "1");
|
||||
}
|
||||
SolrRequest request = new QueryRequest(params);
|
||||
request.setPath("/admin/collections");
|
||||
|
|
Loading…
Reference in New Issue