You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2018/10/04 10:43:08 UTC

lucene-solr:master: SOLR-12827: Migrate cluster wide defaults syntax in cluster properties to a nested structure

Repository: lucene-solr
Updated Branches:
  refs/heads/master 05949a32c -> 152fd966a


SOLR-12827: Migrate cluster wide defaults syntax in cluster properties to a nested structure

The cluster wide defaults structure has changed from {collectionDefaults: {nrtReplicas : 2}} to {defaults : {collection : {nrtReplicas : 2}}}.  The old format continues to be supported and can be read from ZK as well as written using the V2 set-obj-property syntax but it is deprecated and will be removed in Solr 9. We recommend that users change their API calls to use the new format going forward.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/152fd966
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/152fd966
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/152fd966

Branch: refs/heads/master
Commit: 152fd966a7a23b4a5379d9b24ae731ef4fe58766
Parents: 05949a3
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Thu Oct 4 16:12:55 2018 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Thu Oct 4 16:12:55 2018 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   9 ++
 .../solr/handler/admin/CollectionsHandler.java  |  10 +-
 .../solr/cloud/CollectionsAPISolrJTest.java     | 127 ++++++++++++++++++-
 .../test/org/apache/solr/util/TestUtils.java    |   6 +-
 solr/solr-ref-guide/src/collections-api.adoc    |  40 +++---
 .../solr/common/cloud/ClusterProperties.java    |  41 +++++-
 .../apache/solr/common/cloud/ZkStateReader.java |   7 +-
 .../common/params/CollectionAdminParams.java    |   5 +
 .../src/resources/apispec/cluster.Commands.json |  34 ++++-
 9 files changed, 240 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 7337a23..9bb39e9 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -106,6 +106,11 @@ Upgrade Notes
 * SOLR-12767: The min_rf parameter is no longer needed, Solr will always return the achieved replication factor (rf)
   in the response header.
 
+* SOLR-12827: The cluster wide defaults structure has changed from {collectionDefaults: {nrtReplicas : 2}} to
+  {defaults : {collection : {nrtReplicas : 2}}}.  The old format continues to be supported and can be read from
+  ZK as well as written using the V2 set-obj-property syntax but it is deprecated and will be removed in Solr 9.
+  We recommend that users change their API calls to use the new format going forward.
+
 New Features
 ----------------------
 
@@ -132,6 +137,10 @@ Other Changes
   of a builder class instead of calling a method with large number of arguments. The number of special cases that had
   to be handled have been cut down as well. (shalin)
 
+* SOLR-12827: Migrate cluster wide defaults syntax in cluster properties to a nested structure. The structure has
+  changed from {collectionDefaults: {nrtReplicas : 2}} to {defaults : {collection : {nrtReplicas : 2}}}.
+  (ab, shalin)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index aef2448..93181a4 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -123,7 +123,6 @@ import static org.apache.solr.common.cloud.DocCollection.RULE;
 import static org.apache.solr.common.cloud.DocCollection.SNITCH;
 import static org.apache.solr.common.cloud.DocCollection.STATE_FORMAT;
 import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
-import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
@@ -213,7 +212,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   protected void copyFromClusterProp(Map<String, Object> props, String prop) throws IOException {
     if (props.get(prop) != null) return;//if it's already specified , return
     Object defVal = new ClusterProperties(coreContainer.getZkController().getZkStateReader().getZkClient())
-        .getClusterProperty(ImmutableList.of(COLLECTION_DEF, prop), null);
+        .getClusterProperty(ImmutableList.of(CollectionAdminParams.DEFAULTS, CollectionAdminParams.COLLECTION, prop), null);
     if (defVal != null) props.put(prop, String.valueOf(defVal));
   }
 
@@ -457,13 +456,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
   }
 
   public enum CollectionOperation implements CollectionOp {
-    /**
-     * very simple currently, you can pass a template collection, and the new collection is created on
-     * every node the template collection is on
-     * there is a lot more to add - you should also be able to create with an explicit server list
-     * we might also want to think about error handling (add the request to a zk queue and involve overseer?)
-     * as well as specific replicas= options
-     */
     CREATE_OP(CREATE, (req, rsp, h) -> {
       Map<String, Object> props = copy(req.getParams().required(), null, NAME);
       props.put("fromApi", "true");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index cd9087d..d9826eb 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -56,6 +56,8 @@ import static java.util.Arrays.asList;
 import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.NUM_SHARDS_PROP;
+import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
+import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
 
 @LuceneTestCase.Slow
 public class CollectionsAPISolrJTest extends SolrCloudTestCase {
@@ -99,7 +101,7 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   }
 
   @Test
-  public void testCreateCollWithDefaultClusterProperties() throws Exception {
+  public void testCreateCollWithDefaultClusterPropertiesOldFormat() throws Exception {
     String COLL_NAME = "CollWithDefaultClusterProperties";
     try {
       V2Response rsp = new V2Request.Builder("/cluster")
@@ -113,9 +115,9 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
         if (m != null) break;
         Thread.sleep(10);
       }
-      Object clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, NUM_SHARDS_PROP), null);
+      Object clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NUM_SHARDS_PROP), null);
       assertEquals("2", String.valueOf(clusterProperty));
-      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, NRT_REPLICAS), null);
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NRT_REPLICAS), null);
       assertEquals("2", String.valueOf(clusterProperty));
       CollectionAdminResponse response = CollectionAdminRequest
           .createCollection(COLL_NAME, "conf", null, null, null, null)
@@ -130,12 +132,131 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
         assertEquals(2, slice.getReplicas().size());
       }
       CollectionAdminRequest.deleteCollection(COLL_NAME).process(cluster.getSolrClient());
+
+      // unset only a single value using old format
+      rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{\n" +
+              "  \"set-obj-property\": {\n" +
+              "    \"collectionDefaults\": {\n" +
+              "      \"nrtReplicas\": null\n" +
+              "    }\n" +
+              "  }\n" +
+              "}")
+          .build()
+          .process(cluster.getSolrClient());
+      // assert that it is really gone in both old and new paths
+      // we use a timeout so that the change made in ZK is reflected in the watched copy inside ZkStateReader
+      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, new TimeSource.NanoTimeSource());
+      while (!timeOut.hasTimedOut())  {
+        clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NRT_REPLICAS), null);
+        if (clusterProperty == null)  break;
+      }
+      assertNull(clusterProperty);
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, NRT_REPLICAS), null);
+      assertNull(clusterProperty);
+
+      // delete all defaults the old way
+      rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{collectionDefaults:null}}")
+          .build()
+          .process(cluster.getSolrClient());
+      // assert that it is really gone in both old and new paths
+      timeOut = new TimeOut(5, TimeUnit.SECONDS, new TimeSource.NanoTimeSource());
+      while (!timeOut.hasTimedOut()) {
+        clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NUM_SHARDS_PROP), null);
+        if (clusterProperty == null)  break;
+      }
+      assertNull(clusterProperty);
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, NUM_SHARDS_PROP), null);
+      assertNull(clusterProperty);
     } finally {
+      // clean up in case there was an exception during the test
       V2Response rsp = new V2Request.Builder("/cluster")
           .withMethod(SolrRequest.METHOD.POST)
           .withPayload("{set-obj-property:{collectionDefaults: null}}")
           .build()
           .process(cluster.getSolrClient());
+    }
+
+  }
+
+  @Test
+  public void testCreateCollWithDefaultClusterPropertiesNewFormat() throws Exception {
+    String COLL_NAME = "CollWithDefaultClusterProperties";
+    try {
+      V2Response rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{defaults : {collection:{numShards : 2 , nrtReplicas : 2}}}}")
+          .build()
+          .process(cluster.getSolrClient());
+
+      for (int i = 0; i < 10; i++) {
+        Map m = cluster.getSolrClient().getZkStateReader().getClusterProperty(COLLECTION_DEF, null);
+        if (m != null) break;
+        Thread.sleep(10);
+      }
+      Object clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NUM_SHARDS_PROP), null);
+      assertEquals("2", String.valueOf(clusterProperty));
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NRT_REPLICAS), null);
+      assertEquals("2", String.valueOf(clusterProperty));
+      CollectionAdminResponse response = CollectionAdminRequest
+          .createCollection(COLL_NAME, "conf", null, null, null, null)
+          .process(cluster.getSolrClient());
+      assertEquals(0, response.getStatus());
+      assertTrue(response.isSuccess());
+
+      DocCollection coll = cluster.getSolrClient().getClusterStateProvider().getClusterState().getCollection(COLL_NAME);
+      Map<String, Slice> slices = coll.getSlicesMap();
+      assertEquals(2, slices.size());
+      for (Slice slice : slices.values()) {
+        assertEquals(2, slice.getReplicas().size());
+      }
+      CollectionAdminRequest.deleteCollection(COLL_NAME).process(cluster.getSolrClient());
+
+      // unset only a single value
+      rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{\n" +
+              "  \"set-obj-property\": {\n" +
+              "    \"defaults\" : {\n" +
+              "      \"collection\": {\n" +
+              "        \"nrtReplicas\": null\n" +
+              "      }\n" +
+              "    }\n" +
+              "  }\n" +
+              "}")
+          .build()
+          .process(cluster.getSolrClient());
+      // we use a timeout so that the change made in ZK is reflected in the watched copy inside ZkStateReader
+      TimeOut timeOut = new TimeOut(5, TimeUnit.SECONDS, new TimeSource.NanoTimeSource());
+      while (!timeOut.hasTimedOut())  {
+        clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NRT_REPLICAS), null);
+        if (clusterProperty == null)  break;
+      }
+      assertNull(clusterProperty);
+
+      rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{defaults: {collection:null}}}")
+          .build()
+          .process(cluster.getSolrClient());
+      // assert that it is really gone in both old and new paths
+      timeOut = new TimeOut(5, TimeUnit.SECONDS, new TimeSource.NanoTimeSource());
+      while (!timeOut.hasTimedOut()) {
+        clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(DEFAULTS, COLLECTION, NUM_SHARDS_PROP), null);
+        if (clusterProperty == null)  break;
+      }
+      assertNull(clusterProperty);
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, NUM_SHARDS_PROP), null);
+      assertNull(clusterProperty);
+    } finally {
+      V2Response rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{defaults: null}}")
+          .build()
+          .process(cluster.getSolrClient());
 
     }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/core/src/test/org/apache/solr/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/TestUtils.java b/solr/core/src/test/org/apache/solr/util/TestUtils.java
index 27db2d6..beb0bf6 100644
--- a/solr/core/src/test/org/apache/solr/util/TestUtils.java
+++ b/solr/core/src/test/org/apache/solr/util/TestUtils.java
@@ -318,9 +318,7 @@ public class TestUtils extends SolrTestCaseJ4 {
     sink = new HashMap<>();
     sink.put("legacyCloud", "false");
     assertTrue(Utils.mergeJson(sink, (Map<String, Object>) Utils.fromJSONString("collectionDefaults:{numShards:3 , nrtReplicas:2}")));
-    assertEquals(3l, Utils.getObjectByPath(sink, true, ImmutableList.of(COLLECTION_DEF, NUM_SHARDS_PROP)));
-    assertEquals(2l, Utils.getObjectByPath(sink, true, ImmutableList.of(COLLECTION_DEF, NRT_REPLICAS)));
-
-
+    assertEquals(3L, Utils.getObjectByPath(sink, true, ImmutableList.of(COLLECTION_DEF, NUM_SHARDS_PROP)));
+    assertEquals(2L, Utils.getObjectByPath(sink, true, ImmutableList.of(COLLECTION_DEF, NRT_REPLICAS)));
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/solr-ref-guide/src/collections-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index d601069..a5b4e56 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -1201,7 +1201,7 @@ http://localhost:8983/solr/admin/collections?action=CLUSTERPROP&name=urlScheme&v
 
 === Deeply Nested Cluster Properties ===
 
-==== `collectionDefaults` ====
+==== `defaults` ====
 It is possible to set cluster-wide default values for certain attributes of a collection.
 
 
@@ -1209,14 +1209,17 @@ It is possible to set cluster-wide default values for certain attributes of a co
 [source]
 ----
 curl -X POST -H 'Content-type:application/json' --data-binary '
-{ "set-obj-property" : {
-    "collectionDefaults" : {
-        "numShards" : 2,
-        "nrtReplicas" : 1,
-        "tlogReplicas" : 1,
-        "pullReplicas" : 1,
-
-   }
+{
+  "set-obj-property": {
+    "defaults" : {
+      "collection": {
+        "numShards": 2,
+        "nrtReplicas": 1,
+        "tlogReplicas": 1,
+        "pullReplicas": 1
+      }
+    }
+  }
 }' http://localhost:8983/api/cluster
 ----
 
@@ -1224,22 +1227,29 @@ curl -X POST -H 'Content-type:application/json' --data-binary '
 [source]
 ----
 curl -X POST -H 'Content-type:application/json' --data-binary '
-{ "set-obj-property" : {
-    "collectionDefaults" : {
-        "nrtReplicas" : null,
-   }
+{
+  "set-obj-property": {
+    "defaults" : {
+      "collection": {
+        "nrtReplicas": null
+      }
+    }
+  }
 }' http://localhost:8983/api/cluster
 ----
 
-*Example 2: Unset all values in `collectionDefaults`*
+*Example 2: Unset all values in `defaults`*
 [source]
 ----
 curl -X POST -H 'Content-type:application/json' --data-binary '
 { "set-obj-property" : {
-    "collectionDefaults" : null
+    "defaults" : null
 }' http://localhost:8983/api/cluster
 ----
 
+NOTE: Until Solr 7.5, cluster properties supported a "collectionDefaults" key which is no longer supported. Using the API
+structure for Solr 7.4 or Solr 7.5 will continue to work but the format of the properties will automatically be converted
+to the new nested structure. The old "collectionDefaults" key is deprecated and will be removed in Solr 9.
 
 [[collectionprop]]
 == COLLECTIONPROP: Collection Properties

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
index 2452540..21d1298 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ClusterProperties.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
@@ -32,6 +33,8 @@ import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import static org.apache.solr.common.cloud.ZkStateReader.COLLECTION_DEF;
+
 /**
  * Interact with solr cluster properties
  *
@@ -93,7 +96,8 @@ public class ClusterProperties {
   @SuppressWarnings("unchecked")
   public Map<String, Object> getClusterProperties() throws IOException {
     try {
-      return (Map<String, Object>) Utils.fromJSON(client.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
+      Map<String, Object> properties = (Map<String, Object>) Utils.fromJSON(client.getData(ZkStateReader.CLUSTER_PROPS, null, new Stat(), true));
+      return convertCollectionDefaultsToNestedFormat(properties);
     } catch (KeeperException.NoNodeException e) {
       return Collections.emptyMap();
     } catch (KeeperException | InterruptedException e) {
@@ -103,14 +107,45 @@ public class ClusterProperties {
 
   public void setClusterProperties(Map<String, Object> properties) throws IOException, KeeperException, InterruptedException {
     client.atomicUpdate(ZkStateReader.CLUSTER_PROPS, zkData -> {
-      if (zkData == null) return Utils.toJSON(properties);
+      if (zkData == null) return Utils.toJSON(convertCollectionDefaultsToNestedFormat(properties));
       Map<String, Object> zkJson = (Map<String, Object>) Utils.fromJSON(zkData);
-      boolean modified = Utils.mergeJson(zkJson, properties);
+      zkJson = convertCollectionDefaultsToNestedFormat(zkJson);
+      boolean modified = Utils.mergeJson(zkJson, convertCollectionDefaultsToNestedFormat(properties));
       return modified ? Utils.toJSON(zkJson) : null;
     });
   }
 
   /**
+   * See SOLR-12827 for background. We auto convert any "collectionDefaults" keys to "defaults/collection" format.
+   * This method will modify the given map and return the same object. Remove this method in Solr 9.
+   *
+   * @param properties the properties to be converted
+   * @return the converted map
+   */
+  static Map<String, Object> convertCollectionDefaultsToNestedFormat(Map<String, Object> properties) {
+    if (properties.containsKey(COLLECTION_DEF)) {
+      Map<String, Object> values = (Map<String, Object>) properties.remove(COLLECTION_DEF);
+      if (values != null) {
+        properties.putIfAbsent(CollectionAdminParams.DEFAULTS, new LinkedHashMap<>());
+        Map<String, Object> defaults = (Map<String, Object>) properties.get(CollectionAdminParams.DEFAULTS);
+        defaults.compute(CollectionAdminParams.COLLECTION, (k, v) -> {
+          if (v == null) return values;
+          else {
+            ((Map) v).putAll(values);
+            return v;
+          }
+        });
+      } else {
+        // explicitly set to null, so set null in the nested format as well
+        properties.putIfAbsent(CollectionAdminParams.DEFAULTS, new LinkedHashMap<>());
+        Map<String, Object> defaults = (Map<String, Object>) properties.get(CollectionAdminParams.DEFAULTS);
+        defaults.put(CollectionAdminParams.COLLECTION, null);
+      }
+    }
+    return properties;
+  }
+
+  /**
    * This method sets a cluster property.
    *
    * @param propertyName  The property name to be set.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
index 6abfba8..6011f8a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
@@ -127,6 +127,11 @@ public class ZkStateReader implements Closeable {
   public final static String CONFIGNAME_PROP="configName";
 
   public static final String LEGACY_CLOUD = "legacyCloud";
+
+  /**
+   * @deprecated use {@link org.apache.solr.common.params.CollectionAdminParams#DEFAULTS} instead.
+   */
+  @Deprecated
   public static final String COLLECTION_DEF = "collectionDefaults";
 
   public static final String URL_SCHEME = "urlScheme";
@@ -1001,7 +1006,7 @@ public class ZkStateReader implements Closeable {
       while (true) {
         try {
           byte[] data = zkClient.getData(ZkStateReader.CLUSTER_PROPS, clusterPropertiesWatcher, new Stat(), true);
-          this.clusterProperties = (Map<String, Object>) Utils.fromJSON(data);
+          this.clusterProperties = ClusterProperties.convertCollectionDefaultsToNestedFormat((Map<String, Object>) Utils.fromJSON(data));
           log.debug("Loaded cluster properties: {}", this.clusterProperties);
           return;
         } catch (KeeperException.NoNodeException e) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
index a0ef11f..cb70fb8 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionAdminParams.java
@@ -90,4 +90,9 @@ public interface CollectionAdminParams {
    * and points to the collection on which the `withCollection` was specified.
    */
   String COLOCATED_WITH = "COLOCATED_WITH";
+
+  /**
+   * Used by cluster properties API to provide defaults for collection, cluster etc.
+   */
+  String DEFAULTS = "defaults";
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/152fd966/solr/solrj/src/resources/apispec/cluster.Commands.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/resources/apispec/cluster.Commands.json b/solr/solrj/src/resources/apispec/cluster.Commands.json
index 7adc671..d1f5738 100644
--- a/solr/solrj/src/resources/apispec/cluster.Commands.json
+++ b/solr/solrj/src/resources/apispec/cluster.Commands.json
@@ -90,24 +90,50 @@
         "location": {
           "type": "string"
         },
+        "defaults" : {
+          "type" : "object",
+          "properties": {
+            "collection": {
+              "type": "object",
+              "properties": {
+                "numShards": {
+                  "type": "integer",
+                  "description": "Default number of shards for a collection"
+                },
+                "tlogReplicas": {
+                  "type": "integer",
+                  "description": "Default number of TLOG replicas"
+                },
+                "pullReplicas": {
+                  "type": "integer",
+                  "description": "Default number of PULL replicas"
+                },
+                "nrtReplicas": {
+                  "type": "integer",
+                  "description": "Default number of NRT replicas"
+                }
+              }
+            }
+          }
+        },
         "collectionDefaults": {
           "type": "object",
           "properties": {
             "numShards": {
               "type": "integer",
-              "description": "Default no:of shards for a collection"
+              "description": "Default number of shards for a collection"
             },
             "tlogReplicas": {
               "type": "integer",
-              "description": "Default no:of TLOG replicas"
+              "description": "Default number of TLOG replicas"
             },
             "pullReplicas": {
               "type": "integer",
-              "description": "Default no:of PULL replicas"
+              "description": "Default number of PULL replicas"
             },
             "nrtReplicas": {
               "type": "integer",
-              "description": "Default no:of NRT replicas"
+              "description": "Default number of NRT replicas"
             }
           }
         }