From bacef57576ba3d25a70f96006879919e4091d0be Mon Sep 17 00:00:00 2001 From: Anshum Gupta Date: Thu, 18 Dec 2014 22:27:38 +0000 Subject: [PATCH] SOLR-6523: Provide SolrJ support for specifying stateFormat while creating Collections git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1646558 13f79535-47bb-0310-9956-ffa450edef68 --- solr/CHANGES.txt | 3 +++ .../CollectionsAPIDistributedZkTest.java | 26 +++++++++++++++++++ .../solrj/request/CollectionAdminRequest.java | 23 +++++++++------- 3 files changed, 43 insertions(+), 9 deletions(-) diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index fb37f7a4b6d..b4797c5dc4e 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -510,6 +510,9 @@ Other Changes * SOLR-5922: Add support for adding core properties to SolrJ Collection Admin Request calls. (Varun Thacker via Anshum Gupta). +* SOLR-6523: Provide SolrJ support for specifying stateFormat while creating Collections. + (Anshum Gupta) + ================== 4.10.3 ================== Bug Fixes diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java index db2e24aa9d2..cabb9faba66 100644 --- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java +++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPIDistributedZkTest.java @@ -85,6 +85,7 @@ import org.apache.solr.core.SolrResourceLoader; import org.apache.solr.servlet.SolrDispatchFilter; import org.apache.solr.update.DirectUpdateHandler2; import org.apache.solr.util.DefaultSolrThreadFactory; +import org.apache.zookeeper.data.Stat; import org.junit.Before; import org.junit.BeforeClass; @@ -335,6 +336,31 @@ public class CollectionsAPIDistributedZkTest extends AbstractFullDistribZkTestBa coresStatus = response.getCollectionCoresStatus(); assertEquals(2, coresStatus.size()); + // Test Creating a collection with new stateformat. + String collectionName = "solrj_newstateformat"; + createCollectionRequest = new CollectionAdminRequest.Create(); + createCollectionRequest.setCollectionName(collectionName); + createCollectionRequest.setNumShards(2); + createCollectionRequest.setConfigName("conf1"); + createCollectionRequest.setStateFormat(2); + + response = createCollectionRequest.process(server); + assertEquals(0, response.getStatus()); + assertTrue(response.isSuccess()); + CollectionAdminRequest requestStatusRequest = new CollectionAdminRequest.RequestStatus(); + requestStatusRequest.setCollectionName(collectionName); + + waitForRecoveriesToFinish(collectionName, false); + assertTrue("Collection state does not exist", + cloudClient.getZkStateReader().getZkClient() + .exists(ZkStateReader.getCollectionPath(collectionName), true)); + Stat stat = new Stat(); + ClusterState clusterState = cloudClient.getZkStateReader().getClusterState(); + assertEquals("The zkversion of the nodes must be same zkver:" + stat.getVersion() , + stat.getVersion(),clusterState.getCollection(collectionName).getZNodeVersion() ); + assertTrue("DocCollection#getStateFormat() must be > 1", + cloudClient.getZkStateReader().getClusterState().getCollection(collectionName).getStateFormat() > 1); + CollectionAdminRequest.CreateShard createShardRequest = new CollectionAdminRequest .CreateShard(); createShardRequest.setCollectionName("solrj_implicit"); diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java index 7403bd249d4..e57e451d29a 100644 --- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java +++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java @@ -17,18 +17,12 @@ package org.apache.solr.client.solrj.request; -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; -import java.util.Properties; -import java.util.concurrent.TimeUnit; - import org.apache.solr.client.solrj.SolrRequest; import org.apache.solr.client.solrj.SolrServer; import org.apache.solr.client.solrj.SolrServerException; import org.apache.solr.client.solrj.response.CollectionAdminResponse; import org.apache.solr.common.SolrException; +import org.apache.solr.common.cloud.DocCollection; import org.apache.solr.common.cloud.ZkStateReader; import org.apache.solr.common.params.CollectionParams.CollectionAction; import org.apache.solr.common.params.CoreAdminParams; @@ -37,6 +31,13 @@ import org.apache.solr.common.params.ShardParams; import org.apache.solr.common.params.SolrParams; import org.apache.solr.common.util.ContentStream; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.TimeUnit; + /** * This class is experimental and subject to change. * @@ -94,6 +95,7 @@ public class CollectionAdminRequest extends SolrRequest private Properties properties; protected Boolean autoAddReplicas; + protected Integer stateFormat; public Create() { @@ -109,6 +111,7 @@ public class CollectionAdminRequest extends SolrRequest public void setMaxShardsPerNode(Integer numShards) { this.maxShardsPerNode = numShards; } public void setAutoAddReplicas(boolean autoAddReplicas) { this.autoAddReplicas = autoAddReplicas; } public void setReplicationFactor(Integer repl) { this.replicationFactor = repl; } + public void setStateFormat(Integer stateFormat) { this.stateFormat = stateFormat; } public String getConfigName() { return configName; } public String getCreateNodeSet() { return createNodeSet; } @@ -118,6 +121,7 @@ public class CollectionAdminRequest extends SolrRequest public Integer getMaxShardsPerNode() { return maxShardsPerNode; } public Integer getReplicationFactor() { return replicationFactor; } public Boolean getAutoAddReplicas() { return autoAddReplicas; } + public Integer getStateFormat() { return stateFormat; } public Properties getProperties() { return properties; @@ -153,7 +157,6 @@ public class CollectionAdminRequest extends SolrRequest params.set("router.field", routerField); } if (replicationFactor != null) { - // OverseerCollectionProcessor.REPLICATION_FACTOR params.set( "replicationFactor", replicationFactor); } if (asyncId != null) { @@ -166,7 +169,9 @@ public class CollectionAdminRequest extends SolrRequest if(properties != null) { addProperties(params, properties); } - + if (stateFormat != null) { + params.set(DocCollection.STATE_FORMAT, stateFormat); + } return params; } }