You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2018/05/31 14:51:08 UTC

lucene-solr:master: SOLR-12387: cluster-wide defaults for numShards, nrtReplicas, tlogReplicas, pullReplicas

Repository: lucene-solr
Updated Branches:
  refs/heads/master 76263087b -> 12269abe3


SOLR-12387: cluster-wide defaults for numShards, nrtReplicas, tlogReplicas, pullReplicas

SOLR-12389: support deeply nested json objects in clusterprops.json


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

Branch: refs/heads/master
Commit: 12269abe349eff6655f2c3ba96bbad7667b1c641
Parents: 7626308
Author: Noble Paul <no...@apache.org>
Authored: Fri Jun 1 00:50:52 2018 +1000
Committer: Noble Paul <no...@apache.org>
Committed: Fri Jun 1 00:50:52 2018 +1000

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 ++
 .../handler/admin/CollectionHandlerApi.java     | 56 ++++++++++++++++----
 .../solr/handler/admin/CollectionsHandler.java  | 11 ++++
 .../solr/cloud/CollectionsAPISolrJTest.java     | 51 ++++++++++++++++++
 .../solr/handler/admin/TestCollectionAPIs.java  |  5 ++
 .../test/org/apache/solr/util/TestUtils.java    | 28 +++++++++-
 .../solrj/request/CollectionAdminRequest.java   |  2 +-
 .../solrj/request/CollectionApiMapping.java     |  4 ++
 .../solr/common/cloud/ClusterProperties.java    | 18 ++++++-
 .../apache/solr/common/cloud/SolrZkClient.java  | 33 ++++++++++++
 .../apache/solr/common/cloud/ZkStateReader.java | 10 +++-
 .../solr/common/util/JsonSchemaValidator.java   |  5 +-
 .../java/org/apache/solr/common/util/Utils.java | 42 +++++++++++++++
 .../src/resources/apispec/cluster.Commands.json | 43 +++++++++++++++
 .../solr/common/util/JsonValidatorTest.java     |  8 +++
 15 files changed, 303 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 9122b6d..af12aae 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -150,6 +150,10 @@ New Features
   collection. New /admin/metrics/history API allows retrieval of this data in numeric
   or graph formats. (ab)
 
+* SOLR-12387: cluster-wide defaults for numShards, nrtReplicas, tlogReplicas, pullReplicas (noble)
+
+* SOLR-12389: support deeply nested json objects in clusterprops.json (noble)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
index b2da158..d7d179a 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionHandlerApi.java
@@ -17,6 +17,7 @@
 
 package org.apache.solr.handler.admin;
 
+import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumMap;
@@ -27,11 +28,19 @@ import org.apache.solr.client.solrj.request.CollectionApiMapping;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.CommandMeta;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.Meta;
 import org.apache.solr.client.solrj.request.CollectionApiMapping.V2EndPoint;
+import org.apache.solr.common.Callable;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterProperties;
+import org.apache.solr.common.util.CommandOperation;
 import org.apache.solr.handler.admin.CollectionsHandler.CollectionOperation;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 public class CollectionHandlerApi extends BaseHandlerApiSupport {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
   final CollectionsHandler handler;
   static Collection<ApiCommand> apiCommands = createCollMapping();
 
@@ -55,24 +64,53 @@ public class CollectionHandlerApi extends BaseHandlerApiSupport {
         }
       }
     }
-    result.put(Meta.GET_NODES, new ApiCommand() {
+    //The following APIs have only V2 implementations
+    addApi(result, Meta.GET_NODES, params -> params.rsp.add("nodes", ((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getClusterState().getLiveNodes()));
+    addApi(result, Meta.SET_CLUSTER_PROPERTY_OBJ, params -> {
+      List<CommandOperation> commands = params.req.getCommands(true);
+      if (commands == null || commands.isEmpty()) throw new RuntimeException("Empty commands");
+      ClusterProperties clusterProperties = new ClusterProperties(((CollectionHandlerApi) params.apiHandler).handler.coreContainer.getZkController().getZkClient());
+
+      try {
+        clusterProperties.setClusterProperties(commands.get(0).getDataMap());
+      } catch (Exception e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Error in API", e);
+      }
+    });
+
+    for (Meta meta : Meta.values()) {
+      if (result.get(meta) == null) {
+        log.error("ERROR_INIT. No corresponding API implementation for : " + meta.commandName);
+      }
+    }
+
+    return result.values();
+  }
+
+  private static void addApi(Map<Meta, ApiCommand> result, Meta metaInfo, Callable<ApiParams> fun) {
+    result.put(metaInfo, new ApiCommand() {
       @Override
       public CommandMeta meta() {
-        return Meta.GET_NODES;
+        return metaInfo;
       }
 
       @Override
       public void invoke(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) throws Exception {
-        rsp.add("nodes", ((CollectionHandlerApi) apiHandler).handler.coreContainer.getZkController().getClusterState().getLiveNodes());
+        fun.call(new ApiParams(req, rsp, apiHandler));
       }
     });
-    for (Meta meta : Meta.values()) {
-      if(result.get(meta) == null){
-        throw new RuntimeException("No implementation for "+ meta.name());
-      }
-    }
+  }
 
-    return result.values();
+  static class ApiParams {
+    final SolrQueryRequest req;
+    final SolrQueryResponse rsp;
+    final BaseHandlerApiSupport apiHandler;
+
+    ApiParams(SolrQueryRequest req, SolrQueryResponse rsp, BaseHandlerApiSupport apiHandler) {
+      this.req = req;
+      this.rsp = rsp;
+      this.apiHandler = apiHandler;
+    }
   }
 
   public CollectionHandlerApi(CollectionsHandler handler) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 f7f6172..01d2fe8 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
@@ -33,6 +33,7 @@ import java.util.Set;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.lang.StringUtils;
@@ -122,6 +123,7 @@ 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;
@@ -204,6 +206,12 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     return this.coreContainer;
   }
 
+  protected void copyFromClusterProp(Map<String, Object> props, String prop) {
+    if (props.get(prop) != null) return;//if it's already specified , return
+    Object defVal = coreContainer.getZkController().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, prop), null);
+    if (defVal != null) props.put(prop, String.valueOf(defVal));
+  }
+
   @Override
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     // Make sure the cores is enabled
@@ -490,6 +498,9 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
         createSysConfigSet(h.coreContainer);
 
       }
+      if (shardsParam == null) h.copyFromClusterProp(props, NUM_SLICES);
+      for (String prop : ImmutableSet.of(NRT_REPLICAS, PULL_REPLICAS, TLOG_REPLICAS))
+        h.copyFromClusterProp(props, prop);
       copyPropertiesWithPrefix(req.getParams(), props, COLL_PROP_PREFIX);
       return copyPropertiesWithPrefix(req.getParams(), props, "router.");
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 1a13f6c..d9e153e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -27,15 +27,19 @@ import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
+import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
 import org.apache.solr.client.solrj.request.CoreStatus;
+import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.response.V2Response;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
@@ -50,6 +54,10 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
+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;
+
 @LuceneTestCase.Slow
 public class CollectionsAPISolrJTest extends SolrCloudTestCase {
 
@@ -92,6 +100,49 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testCreateCollWithDefaultClusterProperties() throws Exception {
+    String COLL_NAME = "CollWithDefaultClusterProperties";
+    try {
+      V2Response rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{collectionDefaults:{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(COLLECTION_DEF, NUM_SHARDS_PROP), null);
+      assertEquals("2", String.valueOf(clusterProperty));
+      clusterProperty = cluster.getSolrClient().getZkStateReader().getClusterProperty(ImmutableList.of(COLLECTION_DEF, 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());
+    } finally {
+      V2Response rsp = new V2Request.Builder("/cluster")
+          .withMethod(SolrRequest.METHOD.POST)
+          .withPayload("{set-obj-property:{collectionDefaults: null}}")
+          .build()
+          .process(cluster.getSolrClient());
+
+    }
+
+  }
+
+  @Test
   public void testCreateAndDeleteCollection() throws Exception {
     String collectionName = "solrj_test";
     CollectionAdminResponse response = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
index 3601347..0ac81a8 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -279,6 +279,11 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
     }
 
     @Override
+    protected void copyFromClusterProp(Map<String, Object> props, String prop) {
+
+    }
+
+    @Override
     void invokeAction(SolrQueryRequest req, SolrQueryResponse rsp,
                       CoreContainer cores,
                       CollectionParams.CollectionAction action,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 2f4cd53..edf8135 100644
--- a/solr/core/src/test/org/apache/solr/util/TestUtils.java
+++ b/solr/core/src/test/org/apache/solr/util/TestUtils.java
@@ -26,6 +26,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import com.google.common.collect.ImmutableList;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.CommandOperation;
@@ -39,6 +40,9 @@ import org.apache.solr.common.util.Utils;
 import org.junit.Assert;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+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.util.Utils.fromJSONString;
 
 /**
@@ -228,7 +232,7 @@ public class TestUtils extends SolrTestCaseJ4 {
   }
 
   public void testUtilsJSPath(){
-    
+
     String json = "{\n" +
         "  'authorization':{\n" +
         "    'class':'solr.RuleBasedAuthorizationPlugin',\n" +
@@ -246,6 +250,26 @@ public class TestUtils extends SolrTestCaseJ4 {
         "    '':{'v':4}}}";
     Map m = (Map) fromJSONString(json);
     assertEquals("x-update", Utils.getObjectByPath(m,false, "authorization/permissions[1]/name"));
-    
+
+  }
+
+  public void testMergeJson() {
+    Map<String, Object> sink = (Map<String, Object>) Utils.fromJSONString("{k2:v2, k1: {a:b, p:r, k21:{xx:yy}}}");
+    assertTrue(Utils.mergeJson(sink, (Map<String, Object>) Utils.fromJSONString("k1:{a:c, e:f, p :null, k11:{a1:b1}, k21:{pp : qq}}")));
+
+    assertEquals("v2", Utils.getObjectByPath(sink, true, "k2"));
+    assertEquals("c", Utils.getObjectByPath(sink, true, "k1/a"));
+    assertEquals("yy", Utils.getObjectByPath(sink, true, "k1/k21/xx"));
+    assertEquals("qq", Utils.getObjectByPath(sink, true, "k1/k21/pp"));
+    assertEquals("f", Utils.getObjectByPath(sink, true, "k1/e"));
+    assertEquals("b1", Utils.getObjectByPath(sink, true, "k1/k11/a1"));
+
+    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)));
+
+
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
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 4c74a58..4566033 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
@@ -320,7 +320,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
    * @param numTlogReplicas the number of {@link org.apache.solr.common.cloud.Replica.Type#TLOG} replicas
    * @param numPullReplicas the number of {@link org.apache.solr.common.cloud.Replica.Type#PULL} replicas
    */
-  public static Create createCollection(String collection, String config, int numShards, int numNrtReplicas, int numTlogReplicas, int numPullReplicas) {
+  public static Create createCollection(String collection, String config, Integer numShards, Integer numNrtReplicas, Integer numTlogReplicas, Integer numPullReplicas) {
     return new Create(collection, config, numShards, numNrtReplicas, numTlogReplicas, numPullReplicas);
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index 7fd4efe..74d0bbc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -203,6 +203,10 @@ public class CollectionApiMapping {
         POST,
         CLUSTERPROP,
         "set-property",null),
+    SET_CLUSTER_PROPERTY_OBJ(CLUSTER_CMD,
+        POST,
+        null,
+        "set-obj-property", null),
     UTILIZE_NODE(CLUSTER_CMD,
         POST,
         UTILIZENODE,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 e6df845..87896da 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
@@ -18,6 +18,7 @@
 package org.apache.solr.common.cloud;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.util.Collections;
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -27,6 +28,8 @@ import org.apache.solr.common.util.Utils;
 import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Interact with solr cluster properties
@@ -36,6 +39,8 @@ import org.apache.zookeeper.data.Stat;
  * {@link ZkStateReader#getClusterProperty(String, Object)}
  */
 public class ClusterProperties {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
 
   private final SolrZkClient client;
 
@@ -48,7 +53,7 @@ public class ClusterProperties {
 
   /**
    * Read the value of a cluster property, returning a default if it is not set
-   * @param key           the property name
+   * @param key           the property name or the full path to the property.
    * @param defaultValue  the default value
    * @param <T>           the type of the property
    * @return the property value
@@ -56,7 +61,7 @@ public class ClusterProperties {
    */
   @SuppressWarnings("unchecked")
   public <T> T getClusterProperty(String key, T defaultValue) throws IOException {
-    T value = (T) getClusterProperties().get(key);
+    T value = (T) Utils.getObjectByPath(getClusterProperties(), false, key);
     if (value == null)
       return defaultValue;
     return value;
@@ -77,6 +82,15 @@ 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);
+      Map<String, Object> zkJson = (Map<String, Object>) Utils.fromJSON(zkData);
+      boolean modified = Utils.mergeJson(zkJson, properties);
+      return modified ? Utils.toJSON(zkJson) : null;
+    });
+  }
+
   /**
    * This method sets a cluster property.
    *

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
index c646258..1875073 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
@@ -34,6 +34,7 @@ import java.nio.file.Path;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.function.Function;
 import java.util.function.Predicate;
 import java.util.regex.Pattern;
 
@@ -355,6 +356,38 @@ public class SolrZkClient implements Closeable {
     }
   }
 
+  public void atomicUpdate(String path, Function<byte[], byte[]> editor) throws KeeperException, InterruptedException {
+    for (; ; ) {
+      byte[] modified = null;
+      byte[] zkData = null;
+      Stat s = new Stat();
+      try {
+        if (exists(path, true)) {
+          zkData = getData(path, null, s, true);
+          modified = editor.apply(zkData);
+          if (modified == null) {
+            //no change , no need to persist
+            return;
+          }
+          setData(path, modified, s.getVersion(), true);
+          break;
+        } else {
+          modified = editor.apply(null);
+          if (modified == null) {
+            //no change , no need to persist
+            return;
+          }
+          create(path, modified, CreateMode.PERSISTENT, true);
+          break;
+        }
+      } catch (KeeperException.BadVersionException | KeeperException.NodeExistsException e) {
+        continue;
+      }
+    }
+
+
+  }
+
   /**
    * Returns path of created node
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 fa52678..6b65c34 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,7 @@ public class ZkStateReader implements Closeable {
   public final static String CONFIGNAME_PROP="configName";
 
   public static final String LEGACY_CLOUD = "legacyCloud";
+  public static final String COLLECTION_DEF = "collectionDefaults";
 
   public static final String URL_SCHEME = "urlScheme";
   
@@ -954,7 +955,14 @@ public class ZkStateReader implements Closeable {
    */
   @SuppressWarnings("unchecked")
   public <T> T getClusterProperty(String key, T defaultValue) {
-    T value = (T) clusterProperties.get(key);
+    T value = (T) Utils.getObjectByPath( clusterProperties, false, key);
+    if (value == null)
+      return defaultValue;
+    return value;
+  }
+
+  public <T> T getClusterProperty(List<String> keyPath, T defaultValue) {
+    T value = (T) Utils.getObjectByPath( clusterProperties, false, keyPath);
     if (value == null)
       return defaultValue;
     return value;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
index 2a8d2d1..b1fcc91 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
@@ -74,9 +74,10 @@ public class JsonSchemaValidator {
     return errs.isEmpty() ? null : errs;
   }
 
-  boolean validate(Object data, List<String> errs){
+  boolean validate(Object data, List<String> errs) {
+    if (data == null) return true;
     for (Validator validator : validators) {
-      if(!validator.validate(data, errs)) {
+      if (!validator.validate(data, errs)) {
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index 995d38a..70c9d81 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -456,6 +456,48 @@ public class Utils {
     }
   }
 
+  /**Applies one json over other. The 'input' is applied over the sink
+   * The values in input isapplied over the values in 'sink' . If a value is 'null'
+   * that value is removed from sink
+   *
+   * @param sink the original json object to start with. Ensure that this Map is mutable
+   * @param input the json with new values
+   * @return whether there was any change made to sink or not.
+   */
+
+  public static boolean mergeJson(Map<String, Object> sink, Map<String, Object> input) {
+    boolean isModified = false;
+    for (Map.Entry<String, Object> e : input.entrySet()) {
+      if (sink.get(e.getKey()) != null) {
+        Object sinkVal = sink.get(e.getKey());
+        if (e.getValue() == null) {
+          sink.remove(e.getKey());
+          isModified = true;
+        } else {
+          if (e.getValue() instanceof Map) {
+            Map<String, Object> mapInputVal = (Map<String, Object>) e.getValue();
+            if (sinkVal instanceof Map) {
+              if (mergeJson((Map<String, Object>) sinkVal, mapInputVal)) isModified = true;
+            } else {
+              sink.put(e.getKey(), mapInputVal);
+              isModified = true;
+            }
+          } else {
+            sink.put(e.getKey(), e.getValue());
+            isModified = true;
+          }
+
+        }
+      } else if (e.getValue() != null) {
+        sink.put(e.getKey(), e.getValue());
+        isModified = true;
+      }
+
+    }
+
+    return isModified;
+  }
+
   public static String getBaseUrlForNodeName(final String nodeName, String urlScheme) {
     final int _offset = nodeName.indexOf("_");
     if (_offset < 0) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/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 d871818..7adc671 100644
--- a/solr/solrj/src/resources/apispec/cluster.Commands.json
+++ b/solr/solrj/src/resources/apispec/cluster.Commands.json
@@ -70,6 +70,49 @@
         "val"
       ]
     },
+    "set-obj-property": {
+      "type": "object",
+      "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#clusterprop",
+      "description": "Add, edit, or delete a cluster-wide property.",
+      "properties": {
+        "legacyCloud": {
+          "type": "boolean"
+        },
+        "urlScheme": {
+          "type": "string"
+        },
+        "autoAddReplicas": {
+          "type": "boolean"
+        },
+        "maxCoresPerNode": {
+          "type": "boolean"
+        },
+        "location": {
+          "type": "string"
+        },
+        "collectionDefaults": {
+          "type": "object",
+          "properties": {
+            "numShards": {
+              "type": "integer",
+              "description": "Default no:of shards for a collection"
+            },
+            "tlogReplicas": {
+              "type": "integer",
+              "description": "Default no:of TLOG replicas"
+            },
+            "pullReplicas": {
+              "type": "integer",
+              "description": "Default no:of PULL replicas"
+            },
+            "nrtReplicas": {
+              "type": "integer",
+              "description": "Default no:of NRT replicas"
+            }
+          }
+        }
+      }
+    },
     "utilize-node": {
       "type": "object",
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#utilizenode",

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/12269abe/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java b/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
index fa6d080..b39c497 100644
--- a/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
+++ b/solr/solrj/src/test/org/apache/solr/common/util/JsonValidatorTest.java
@@ -40,6 +40,7 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
     checkSchema("core.config.Commands");
     checkSchema("core.SchemaEdit");
     checkSchema("cluster.configs.Commands");
+    checkSchema("cluster.Commands");
   }
 
 
@@ -176,6 +177,13 @@ public class JsonValidatorTest extends SolrTestCaseJ4  {
 
   }
 
+  public void testNullObjectValue() {
+    ValidatingJsonMap spec = Utils.getSpec("cluster.Commands").getSpec();
+    JsonSchemaValidator validator = new JsonSchemaValidator((Map) Utils.getObjectByPath(spec, false, "/commands/set-obj-property"));
+    List<String> object = validator.validateJson(Utils.fromJSONString("{collectionDefaults: null}"));
+    assertNull(object);
+  }
+
   private void checkSchema(String name) {
     ValidatingJsonMap spec = Utils.getSpec(name).getSpec();
     Map commands = (Map) spec.get("commands");