You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by tf...@apache.org on 2018/03/06 01:44:30 UTC

[2/2] lucene-solr:branch_7x: SOLR-11960: Add collection-level properties

SOLR-11960: Add collection-level properties

Similar to cluster properties, but apply to a single collection.


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

Branch: refs/heads/branch_7x
Commit: cfafc47e9c9229fe94b0d367249db66ec6b54132
Parents: 8775f3f
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Mon Mar 5 16:59:56 2018 -0800
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Mon Mar 5 17:43:50 2018 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../solr/cloud/api/collections/BackupCmd.java   |   2 +
 .../solr/cloud/api/collections/RestoreCmd.java  |   3 +
 .../apache/solr/core/backup/BackupManager.java  |  43 +++-
 .../solr/handler/admin/CollectionsHandler.java  |  11 +
 .../apache/solr/cloud/CollectionPropsTest.java  | 256 +++++++++++++++++++
 .../solr/cloud/CollectionsAPISolrJTest.java     |  36 +++
 .../core/snapshots/TestSolrCloudSnapshots.java  |  15 ++
 .../solr/handler/admin/TestCollectionAPIs.java  |  19 +-
 solr/solr-ref-guide/src/collections-api.adoc    |  43 ++++
 .../solrj/request/CollectionAdminRequest.java   |  39 +++
 .../solrj/request/CollectionApiMapping.java     |   7 +
 .../solr/common/cloud/CollectionProperties.java | 117 +++++++++
 .../common/cloud/CollectionPropsWatcher.java    |  40 +++
 .../apache/solr/common/cloud/ZkStateReader.java | 209 ++++++++++++++-
 .../common/params/CollectionAdminParams.java    |  10 +
 .../solr/common/params/CollectionParams.java    |   1 +
 .../collections.collection.Commands.json        |  20 +-
 ...ollectionAdminRequestRequiredParamsTest.java |   9 +-
 .../solrj/request/TestV1toV2ApiMapper.java      |   7 +
 20 files changed, 881 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index e1526c1..86d8484 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -162,6 +162,10 @@ New Features
 
 * SOLR-11795: Add Solr metrics exporter for Prometheus (Minoru Osuka via koji)
 
+* SOLR-11267: Add support for "add-distinct" atomic update operation (Amrit Sarkar via noble )
+
+* SOLR-11960: Add collection level properties similar to cluster properties (Peter Rusko, Tomás Fernández Löbbe)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
index c411fbc..7589c55 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/BackupCmd.java
@@ -124,6 +124,8 @@ public class BackupCmd implements OverseerCollectionMessageHandler.Cmd {
 
     backupMgr.writeBackupProperties(location, backupName, properties);
 
+    backupMgr.downloadCollectionProperties(location, backupName, collectionName);
+
     log.info("Completed backing up ZK data for backupName={}", backupName);
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
index 09ceb55..1823fe3 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/RestoreCmd.java
@@ -191,6 +191,9 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
       // note: when createCollection() returns, the collection exists (no race)
     }
 
+    // Restore collection properties
+    backupMgr.uploadCollectionProperties(location, backupName, restoreCollectionName);
+
     DocCollection restoreCollection = zkStateReader.getClusterState().getCollection(restoreCollectionName);
 
     DistributedQueue inQueue = Overseer.getStateUpdateQueue(zkStateReader.getZkClient());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java b/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
index d2094d6..afba4b1 100644
--- a/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
+++ b/solr/core/src/java/org/apache/solr/core/backup/BackupManager.java
@@ -44,6 +44,7 @@ import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.backup.repository.BackupRepository;
 import org.apache.solr.core.backup.repository.BackupRepository.PathType;
 import org.apache.solr.util.PropertiesInputStream;
+import org.apache.zookeeper.CreateMode;
 import org.apache.zookeeper.KeeperException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -192,6 +193,46 @@ public class BackupManager {
     downloadFromZK(zkStateReader.getZkClient(), ZkConfigManager.CONFIGS_ZKNODE + "/" + configName, dest);
   }
 
+  public void uploadCollectionProperties(URI backupLoc, String backupId, String collectionName) throws IOException {
+    URI sourceDir = repository.resolve(backupLoc, backupId, ZK_STATE_DIR);
+    URI source = repository.resolve(sourceDir, ZkStateReader.COLLECTION_PROPS_ZKNODE);
+    if (!repository.exists(source)) {
+      // No collection properties to restore
+      return;
+    }
+    String zkPath = ZkStateReader.COLLECTIONS_ZKNODE + '/' + collectionName + '/' + ZkStateReader.COLLECTION_PROPS_ZKNODE;
+
+    try (IndexInput is = repository.openInput(sourceDir, ZkStateReader.COLLECTION_PROPS_ZKNODE, IOContext.DEFAULT)) {
+      byte[] arr = new byte[(int) is.length()];
+      is.readBytes(arr, 0, (int) is.length());
+      zkStateReader.getZkClient().create(zkPath, arr, CreateMode.PERSISTENT, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error uploading file to zookeeper path " + source.toString() + " to " + zkPath,
+          SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  public void downloadCollectionProperties(URI backupLoc, String backupId, String collectionName) throws IOException {
+    URI dest = repository.resolve(backupLoc, backupId, ZK_STATE_DIR, ZkStateReader.COLLECTION_PROPS_ZKNODE);
+    String zkPath = ZkStateReader.COLLECTIONS_ZKNODE + '/' + collectionName + '/' + ZkStateReader.COLLECTION_PROPS_ZKNODE;
+
+
+    try {
+      if (!zkStateReader.getZkClient().exists(zkPath, true)) {
+        // Nothing to back up
+        return;
+      }
+
+      try (OutputStream os = repository.createOutput(dest)) {
+        byte[] data = zkStateReader.getZkClient().getData(zkPath, null, null, true);
+        os.write(data);
+      }
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error downloading file from zookeeper path " + zkPath + " to " + dest.toString(),
+          SolrZkClient.checkInterrupted(e));
+    }
+  }
+
   private void downloadFromZK(SolrZkClient zkClient, String zkPath, URI dir) throws IOException {
     try {
       if (!repository.exists(dir)) {
@@ -232,7 +273,7 @@ public class BackupManager {
             is.readBytes(arr, 0, (int) is.length());
             zkClient.makePath(zkNodePath, arr, true);
           } catch (KeeperException | InterruptedException e) {
-            throw new IOException(e);
+            throw new IOException(SolrZkClient.checkInterrupted(e));
           }
           break;
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 9824970..f4855eb 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
@@ -58,6 +58,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.Aliases;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.CollectionProperties;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.ImplicitDocRouter;
 import org.apache.solr.common.cloud.Replica;
@@ -133,6 +134,8 @@ import static org.apache.solr.common.cloud.ZkStateReader.REPLICA_TYPE;
 import static org.apache.solr.common.cloud.ZkStateReader.SHARD_ID_PROP;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionAdminParams.COUNT_PROP;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_NAME;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_VALUE;
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonAdminParams.ASYNC;
 import static org.apache.solr.common.params.CommonAdminParams.IN_PLACE_MOVE;
@@ -684,6 +687,14 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       cp.setClusterProperty(name, val);
       return null;
     }),
+    COLLECTIONPROP_OP(COLLECTIONPROP, (req, rsp, h) -> {
+      String collection = req.getParams().required().get(NAME);
+      String name = req.getParams().required().get(PROPERTY_NAME);
+      String val = req.getParams().get(PROPERTY_VALUE);
+      CollectionProperties cp = new CollectionProperties(h.coreContainer.getZkController().getZkClient());
+      cp.setCollectionProperty(collection, name, val);
+      return null;
+    }),
     REQUESTSTATUS_OP(REQUESTSTATUS, (req, rsp, h) -> {
       req.getParams().required().check(REQUESTID);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/core/src/test/org/apache/solr/cloud/CollectionPropsTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CollectionPropsTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionPropsTest.java
new file mode 100644
index 0000000..01d8be7
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionPropsTest.java
@@ -0,0 +1,256 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.cloud;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Locale;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.CollectionAdminResponse;
+import org.apache.solr.common.cloud.CollectionProperties;
+import org.apache.solr.common.cloud.CollectionPropsWatcher;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.zookeeper.KeeperException;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@LuceneTestCase.Slow
+@SolrTestCaseJ4.SuppressSSL
+public class CollectionPropsTest extends SolrCloudTestCase {
+  private String collectionName;
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  @BeforeClass
+  public static void setupClass() throws Exception {
+    Boolean useLegacyCloud = rarely();
+    log.info("Using legacyCloud?: {}", useLegacyCloud);
+
+    configureCluster(4)
+        .withProperty(ZkStateReader.LEGACY_CLOUD, String.valueOf(useLegacyCloud))
+        .addConfig("conf", configset("cloud-minimal"))
+        .configure();
+  }
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+
+    collectionName = "CollectionPropsTest" + System.nanoTime();
+
+    CollectionAdminRequest.Create request = CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2);
+    CollectionAdminResponse response = request.process(cluster.getSolrClient());
+    assertTrue("Unable to create collection: " + response.toString(), response.isSuccess());
+  }
+
+  @Test
+  public void testReadWriteNoCache() throws InterruptedException, IOException {
+    CollectionProperties collectionProps = new CollectionProperties(zkClient());
+
+    collectionProps.setCollectionProperty(collectionName, "property1", "value1");
+    collectionProps.setCollectionProperty(collectionName, "property2", "value2");
+    checkValue("property1", "value1");
+    checkValue("property2", "value2");
+    
+    collectionProps.setCollectionProperty(collectionName, "property1", "value1"); // no change
+    checkValue("property1", "value1");
+
+    collectionProps.setCollectionProperty(collectionName, "property1", null);
+    collectionProps.setCollectionProperty(collectionName, "property2", "newValue");
+    checkValue("property1", null);
+    checkValue("property2", "newValue");
+    
+    collectionProps.setCollectionProperty(collectionName, "property2", null);
+    checkValue("property2", null);
+    
+    collectionProps.setCollectionProperty(collectionName, "property2", null); // no change
+    checkValue("property2", null);
+  }
+  
+  @Test
+  public void testReadWriteCached() throws InterruptedException, IOException {
+    CollectionProperties collectionProps = new CollectionProperties(zkClient());
+    final CollectionPropsWatcher w = new CollectionPropsWatcher() {
+      @Override
+      public boolean onStateChanged(Map<String,String> collectionProperties) {
+        return false;
+      }
+    };
+    
+    cluster.getSolrClient().getZkStateReader().registerCollectionPropsWatcher(collectionName, w);
+    
+    collectionProps.setCollectionProperty(collectionName, "property1", "value1");
+    collectionProps.setCollectionProperty(collectionName, "property2", "value2");
+    waitForValue("property1", "value1", 5000);
+    waitForValue("property2", "value2", 5000);
+    
+    collectionProps.setCollectionProperty(collectionName, "property1", "value1"); // no change
+    checkValue("property1", "value1");
+
+    collectionProps.setCollectionProperty(collectionName, "property1", null);
+    collectionProps.setCollectionProperty(collectionName, "property2", "newValue");
+    waitForValue("property1", null, 5000);
+    waitForValue("property2", "newValue", 5000);
+    
+    collectionProps.setCollectionProperty(collectionName, "property2", null);
+    waitForValue("property2", null, 5000);
+    
+    collectionProps.setCollectionProperty(collectionName, "property2", null); // no change
+    checkValue("property2", null);
+    
+    cluster.getSolrClient().getZkStateReader().removeCollectionPropsWatcher(collectionName, w);
+    
+    collectionProps.setCollectionProperty(collectionName, "property1", "value1");
+    checkValue("property1", "value1"); //Should be no cache, so the change should take effect immediately
+  }
+  
+  private void checkValue(String propertyName, String expectedValue) throws InterruptedException {
+    final Object value = cluster.getSolrClient().getZkStateReader().getCollectionProperties(collectionName).get(propertyName);
+    assertEquals("Unexpected value for collection property: " + propertyName, expectedValue, value);
+  }
+
+  private void waitForValue(String propertyName, String expectedValue, int timeout) throws InterruptedException {
+    final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+
+    for (int i = 0; i < timeout; i += 10) {
+      final Object value = zkStateReader.getCollectionProperties(collectionName).get(propertyName);
+      if ((expectedValue == null && value == null) ||
+          (expectedValue != null && expectedValue.equals(value))) {
+        return;
+      }
+      Thread.sleep(10);
+    }
+
+    fail(String.format(Locale.ROOT, "Could not see value change after setting collection property. Name: %s, current value: %s, expected value: %s",
+            propertyName, zkStateReader.getCollectionProperties(collectionName).get(propertyName), expectedValue));
+  }
+
+  @Test
+  public void testWatcher() throws KeeperException, InterruptedException, IOException {
+    final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    CollectionProperties collectionProps = new CollectionProperties(zkClient());
+
+    // Add a watcher to collection props
+    final Watcher watcher = new Watcher();
+    zkStateReader.registerCollectionPropsWatcher(collectionName, watcher);
+    assertEquals(0, watcher.waitForTrigger(TEST_NIGHTLY?2000:200));
+
+    // Trigger a new znode event
+    collectionProps.setCollectionProperty(collectionName, "property", "value1");
+    assertEquals(1, watcher.waitForTrigger());
+    assertEquals("value1", watcher.getProps().get("property"));
+
+    // Trigger a value change event
+    collectionProps.setCollectionProperty(collectionName, "property", "value2");
+    watcher.waitForTrigger();
+    assertEquals("value2", watcher.getProps().get("property"));
+
+    // Delete the properties znode
+    zkStateReader.getZkClient().delete("/collections/" + collectionName + "/collectionprops.json", -1, true);
+    assertEquals(1, watcher.waitForTrigger());
+    assertTrue(watcher.getProps().isEmpty());
+
+    // Remove watcher and make sure that the watcher is not triggered
+    zkStateReader.removeCollectionPropsWatcher(collectionName, watcher);
+    collectionProps.setCollectionProperty(collectionName, "property", "value1");
+    assertEquals("ZK watcher was triggered after it was removed for collection " + collectionName, 0, watcher.waitForTrigger());
+  }
+
+  @Test
+  public void testMultipleWatchers() throws InterruptedException, IOException {
+    final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    CollectionProperties collectionProps = new CollectionProperties(zkClient());
+
+    // Register the core with ZkStateReader
+    zkStateReader.registerCore(collectionName);
+
+    // Subsequent watchers won't be triggered when adding
+    final Watcher watcher1 = new Watcher();
+    zkStateReader.registerCollectionPropsWatcher(collectionName, watcher1);
+    watcher1.waitForTrigger(); // this might still get triggered because of registerCore
+    final Watcher watcher2 = new Watcher();
+    zkStateReader.registerCollectionPropsWatcher(collectionName, watcher2);
+    assertEquals(0, watcher2.waitForTrigger(TEST_NIGHTLY?2000:200));
+
+    // Make sure a value change triggers both watchers
+    collectionProps.setCollectionProperty(collectionName, "property", "value1");
+    assertEquals(1, watcher1.waitForTrigger());
+    assertEquals(1, watcher2.waitForTrigger());
+
+    // The watchers should be triggered when after the core is unregistered
+    zkStateReader.unregisterCore(collectionName);
+    collectionProps.setCollectionProperty(collectionName, "property", "value2");
+    assertEquals(1, watcher1.waitForTrigger());
+
+    // The watcher should be triggered after another watcher is removed
+    zkStateReader.removeCollectionPropsWatcher(collectionName, watcher2);
+    collectionProps.setCollectionProperty(collectionName, "property", "value3");
+    assertEquals(1, watcher1.waitForTrigger());
+
+    // The last watcher shouldn't be triggered after removed, even if the core is registered
+    zkStateReader.registerCore(collectionName);
+    zkStateReader.removeCollectionPropsWatcher(collectionName, watcher1);
+    collectionProps.setCollectionProperty(collectionName, "property", "value4");
+    assertEquals(0, watcher1.waitForTrigger(TEST_NIGHTLY?2000:200));
+  }
+
+  private class Watcher implements CollectionPropsWatcher {
+    private Map<String, String> props = null;
+    private AtomicInteger triggered = new AtomicInteger();
+
+    @Override
+    public boolean onStateChanged(Map<String, String> collectionProperties) {
+      triggered.incrementAndGet();
+      final ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+      props = zkStateReader.getCollectionProperties(collectionName);
+      synchronized (this) {
+        notifyAll();
+      }
+
+
+      return false;
+    }
+
+    private Map<String, String> getProps() {
+      return props;
+    }
+    
+    private int waitForTrigger() throws InterruptedException {
+      return waitForTrigger(1000);
+    }
+
+    private int waitForTrigger(int waitTime) throws InterruptedException {
+      synchronized (this) {
+        if (triggered.get() > 0) {
+          return triggered.getAndSet(0);
+        }
+
+        wait(waitTime);
+      }
+
+      return triggered.getAndSet(0);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 d6d492c..1a13f6c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -23,7 +23,9 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Optional;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
@@ -41,7 +43,9 @@ import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
+import org.apache.solr.util.TimeOut;
 import org.apache.zookeeper.KeeperException;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -342,6 +346,38 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
   }
 
   @Test
+  public void testCollectionProp() throws InterruptedException, IOException, SolrServerException {
+    final String collectionName = "collectionPropTest";
+    final String propName = "testProperty";
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .process(cluster.getSolrClient());
+
+    // Check for value change
+    CollectionAdminRequest.setCollectionProperty(collectionName, propName, "false")
+        .process(cluster.getSolrClient());
+    checkCollectionProperty(collectionName, propName, "false", 3000);
+
+    // Check for removing value
+    CollectionAdminRequest.setCollectionProperty(collectionName, propName, null)
+        .process(cluster.getSolrClient());
+    checkCollectionProperty(collectionName, propName, null, 3000);
+  }
+
+  private void checkCollectionProperty(String collection, String propertyName, String propertyValue, long timeoutMs) throws InterruptedException {
+    TimeOut timeout = new TimeOut(timeoutMs, TimeUnit.MILLISECONDS, TimeSource.NANO_TIME);
+    while (!timeout.hasTimedOut()){
+      Thread.sleep(10);
+      if (Objects.equals(cluster.getSolrClient().getZkStateReader().getCollectionProperties(collection).get(propertyName), propertyValue)) {
+        return;
+      }
+    }
+
+    fail("Timed out waiting for cluster property value");
+  }
+
+
+  @Test
   public void testOverseerStatus() throws IOException, SolrServerException {
     CollectionAdminResponse response = new CollectionAdminRequest.OverseerStatus().process(cluster.getSolrClient());
     assertEquals(0, response.getStatus());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
index b65863e..7e5b980 100644
--- a/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
+++ b/solr/core/src/test/org/apache/solr/core/snapshots/TestSolrCloudSnapshots.java
@@ -88,6 +88,13 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
     int nDocs = BackupRestoreUtils.indexDocs(cluster.getSolrClient(), collectionName, docsSeed);
     BackupRestoreUtils.verifyDocs(nDocs, solrClient, collectionName);
 
+    // Set a collection property
+    final boolean collectionPropertySet = usually();
+    if (collectionPropertySet) {
+      CollectionAdminRequest.CollectionProp setProperty = CollectionAdminRequest.setCollectionProperty(collectionName, "test.property", "test.value");
+      setProperty.process(solrClient);
+    }
+
     String commitName = TestUtil.randomSimpleString(random(), 1, 5);
 
     // Verify if snapshot creation works with replica failures.
@@ -193,6 +200,14 @@ public class TestSolrCloudSnapshots extends SolrCloudTestCase {
       BackupRestoreUtils.verifyDocs(nDocs, solrClient, restoreCollectionName);
     }
 
+    // Check collection property
+    Map<String, String> collectionProperties = solrClient.getZkStateReader().getCollectionProperties(restoreCollectionName);
+    if (collectionPropertySet) {
+      assertEquals("Snapshot restore hasn't restored collection properties", "test.value", collectionProperties.get("test.property"));
+    } else {
+      assertNull("Collection property shouldn't be present", collectionProperties.get("test.property"));
+    }
+
     // Verify if the snapshot deletion works correctly when one or more replicas containing the snapshot are
     // deleted
     boolean replicaDeletion = rarely();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 f72bea1..c08328c 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
@@ -48,6 +48,9 @@ import org.slf4j.LoggerFactory;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
 import static org.apache.solr.cloud.Overseer.QUEUE_OPERATION;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_NAME;
+import static org.apache.solr.common.params.CollectionAdminParams.PROPERTY_VALUE;
+import static org.apache.solr.common.params.CommonParams.NAME;
 import static org.apache.solr.common.util.Utils.fromJSONString;
 
 public class TestCollectionAPIs extends SolrTestCaseJ4 {
@@ -163,6 +166,11 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
         "{migrate-docs : {forwardTimeout: 1800, target: coll2, splitKey: 'a123!'} }", null,
         "{operation : migrate ,collection : coll1, target.collection:coll2, forward.timeout:1800, split.key:'a123!'}"
     );
+    
+    compareOutput(apiBag, "/collections/coll1", POST,
+        "{set-collection-property : {name: 'foo', value:'bar'} }", null,
+        "{operation : collectionprop, name : coll1, propertyName:'foo', propertyValue:'bar'}"
+    );
 
   }
 
@@ -252,7 +260,16 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
                       CoreContainer cores,
                       CollectionParams.CollectionAction action,
                       CollectionOperation operation) throws Exception {
-      Map<String, Object> result = operation.execute(req, rsp, this);
+      Map<String, Object> result = null;
+      if (action == CollectionParams.CollectionAction.COLLECTIONPROP) {
+        //Fake this action, since we don't want to write to ZooKeeper in this test
+        result = new HashMap<>();
+        result.put(NAME, req.getParams().required().get(NAME));
+        result.put(PROPERTY_NAME, req.getParams().required().get(PROPERTY_NAME));
+        result.put(PROPERTY_VALUE, req.getParams().required().get(PROPERTY_VALUE));
+      } else {
+        result = operation.execute(req, rsp, this);
+      }
       if (result != null) {
         result.put(QUEUE_OPERATION, operation.action.toLower());
         rsp.add(ZkNodeProps.class.getName(), new ZkNodeProps(result));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 dee5443..77f054a 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -1073,6 +1073,49 @@ http://localhost:8983/solr/admin/collections?action=CLUSTERPROP&name=urlScheme&v
 </response>
 ----
 
+[[collectionprop]]
+== COLLECTIONPROP: Collection Properties
+
+Add, edit or delete a collection property.
+
+`/admin/collections?action=COLLECTIONPROP&name=_collectionName_&propertyName=_propertyName_&propertyValue=_propertyValue_`
+
+=== COLLECTIONPROP Parameters
+
+`name`::
+The name of the collection for which the property would be set.
+
+`propertyName`::
+The name of the property.
+
+`propertyValue`::
+The value of the property. When not provided, the property is deleted.
+
+=== COLLECTIONPROP Response
+
+The response will include the status of the request and the properties that were updated or removed. If the status is anything other than "0", an error message will explain why the request failed.
+
+=== Examples using COLLECTIONPROP
+
+*Input*
+
+[source,text]
+----
+http://localhost:8983/solr/admin/collections?action=COLLECTIONPROP&name=coll&propertyName=foo&val=bar&wt=xml
+----
+
+*Output*
+
+[source,xml]
+----
+<response>
+  <lst name="responseHeader">
+    <int name="status">0</int>
+    <int name="QTime">0</int>
+  </lst>
+</response>
+----
+
 [[migrate]]
 == MIGRATE: Migrate Documents to Another Collection
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 8826384..b7bbf2f 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
@@ -1823,6 +1823,45 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
   }
 
+  public static CollectionProp setCollectionProperty(String collection, String propertyName, String propertyValue) {
+    return new CollectionProp(collection, propertyName, propertyValue);
+  }
+
+  // COLLECTIONPROP request
+  public static class CollectionProp extends AsyncCollectionSpecificAdminRequest {
+
+    private String propertyName;
+    private String propertyValue;
+
+    private CollectionProp(String collection, String propertyName, String propertyValue) {
+      super(CollectionAction.COLLECTIONPROP, collection);
+      this.propertyName = checkNotNull("propertyName", propertyName);
+      this.propertyValue = propertyValue;
+    }
+
+    public String getPropertyName() {
+      return this.propertyName;
+    }
+
+    public String getPropertyValue() {
+      return this.propertyValue;
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = new ModifiableSolrParams(super.getParams());
+      params.add(CollectionAdminParams.PROPERTY_NAME, propertyName);
+      params.add(CollectionAdminParams.PROPERTY_VALUE, propertyValue);
+
+      return params;
+    }
+
+    @Override
+    protected CollectionAdminResponse createResponse(SolrClient client) {
+      return new CollectionAdminResponse();
+    }
+  }
+
   /**
    * Returns a SolrRequest to migrate data matching a split key to another collection
    */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 56fe930..95e7008 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
@@ -187,6 +187,13 @@ public class CollectionApiMapping {
         DELETEREPLICAPROP,
         "delete-replica-property",
         null),
+    SET_COLLECTION_PROPERTY(PER_COLLECTION,
+        POST,
+        COLLECTIONPROP,
+        "set-collection-property",
+        Utils.makeMap(NAME, "collection",
+            "propertyName", "name",
+            "propertyValue", "value")),
     ADD_ROLE(CLUSTER_CMD,
         POST,
         ADDROLE,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionProperties.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionProperties.java b/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionProperties.java
new file mode 100644
index 0000000..b5ea129
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionProperties.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.common.cloud;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.Map;
+
+import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+
+/**
+ * Interact with solr collection properties
+ *
+ * Note that all methods on this class make calls to ZK on every invocation.  For
+ * read-only eventually-consistent uses, clients should instead call
+ * {@link ZkStateReader#getCollectionProperties(String)}
+ */
+public class CollectionProperties {
+
+  private final SolrZkClient client;
+
+  /**
+   * Creates a CollectionProperties object using a provided SolrZkClient
+   */
+  public CollectionProperties(SolrZkClient client) {
+    this.client = client;
+  }
+
+  /**
+   * Read the value of a collection property, returning a default if it is not set
+   * @param key           the property name
+   * @param defaultValue  the default value
+   * @return the property value
+   * @throws IOException if there is an error reading the value from zookeeper
+   */
+  public String getCollectionProperty(String collection, String key, String defaultValue) throws IOException {
+    String value = getCollectionProperties(collection).get(key);
+    if (value == null)
+      return defaultValue;
+    return value;
+  }
+
+  /**
+   * Return the collection properties
+   * @throws IOException if there is an error reading properties from zookeeper
+   */
+  @SuppressWarnings("unchecked")
+  public Map<String, String> getCollectionProperties(String collection) throws IOException {
+    try {
+      return (Map<String, String>) Utils.fromJSON(client.getData(ZkStateReader.getCollectionPropsPath(collection), null, new Stat(), true));
+    } catch (KeeperException.NoNodeException e) {
+      return Collections.emptyMap();
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading properties for collection " + collection, SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  /**
+   * This method sets a collection property.
+   *
+   * @param collection    The property name to be set.
+   * @param propertyName  The property name to be set.
+   * @param propertyValue The value of the property.
+   * @throws IOException if there is an error writing data to zookeeper
+   */
+  @SuppressWarnings("unchecked")
+  public void setCollectionProperty(String collection, String propertyName, String propertyValue) throws IOException {
+    String znodePath = ZkStateReader.getCollectionPropsPath(collection);
+
+    while (true) {
+      Stat s = new Stat();
+      try {
+        if (client.exists(znodePath, true)) {
+          Map<String, String> properties = (Map<String, String>) Utils.fromJSON(client.getData(znodePath, null, s, true));
+          if (propertyValue == null) {
+            if (properties.remove(propertyName) != null) { // Don't update ZK unless absolutely necessary.
+              client.setData(znodePath, Utils.toJSON(properties), s.getVersion(), true);
+            }
+          } else {
+            if (!propertyValue.equals(properties.put(propertyName, propertyValue))) { // Don't update ZK unless absolutely necessary.
+              client.setData(znodePath, Utils.toJSON(properties), s.getVersion(), true);
+            }
+          }
+        } else {
+          Map<String, String> properties = new LinkedHashMap<>();
+          properties.put(propertyName, propertyValue);
+          client.create(znodePath, Utils.toJSON(properties), CreateMode.PERSISTENT, true);
+        }
+      } catch (KeeperException.BadVersionException | KeeperException.NodeExistsException e) {
+        //race condition
+        continue;
+      } catch (InterruptedException | KeeperException e) {
+        throw new IOException("Error setting property for collection " + collection, SolrZkClient.checkInterrupted(e));
+      }
+      break;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionPropsWatcher.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionPropsWatcher.java b/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionPropsWatcher.java
new file mode 100644
index 0000000..8c9ef48
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/CollectionPropsWatcher.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.common.cloud;
+
+import java.util.Map;
+
+/**
+ * Callback registered with {@link ZkStateReader#registerCollectionPropsWatcher(String, CollectionPropsWatcher)}
+ * and called whenever the collection properties change.
+ */
+public interface CollectionPropsWatcher {
+
+  /**
+   * Called when the collection properties we are registered against has a change of state
+   *
+   * Note that, due to the way Zookeeper watchers are implemented, a single call may be
+   * the result of several state changes
+   *
+   * @param collectionProperties the new collection properties
+   *
+   * @return true if the watcher should be removed
+   */
+  boolean onStateChanged(Map<String, String> collectionProperties);
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 51872c5..cd2de97 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
@@ -100,6 +100,7 @@ public class ZkStateReader implements Closeable {
   public static final String ALIASES = "/aliases.json";
   public static final String CLUSTER_STATE = "/clusterstate.json";
   public static final String CLUSTER_PROPS = "/clusterprops.json";
+  public static final String COLLECTION_PROPS_ZKNODE = "collectionprops.json";
   public static final String REJOIN_AT_HEAD_PROP = "rejoinAtHead";
   public static final String SOLR_SECURITY_CONF_PATH = "/security.json";
   public static final String SOLR_AUTOSCALING_CONF_PATH = "/autoscaling.json";
@@ -150,6 +151,9 @@ public class ZkStateReader implements Closeable {
   /** Collections with format2 state.json, not "interesting" and not actively watched. */
   private final ConcurrentHashMap<String, LazyCollectionRef> lazyCollectionStates = new ConcurrentHashMap<>();
 
+  /** Collection properties being actively watched */
+  private final ConcurrentHashMap<String, Map<String, String>> watchedCollectionProps = new ConcurrentHashMap<>();
+
   private volatile SortedSet<String> liveNodes = emptySortedSet();
 
   private volatile Map<String, Object> clusterProperties = Collections.emptyMap();
@@ -160,7 +164,9 @@ public class ZkStateReader implements Closeable {
 
   private final Runnable securityNodeListener;
 
-  private ConcurrentHashMap<String, CollectionWatch> collectionWatches = new ConcurrentHashMap<>();
+  private ConcurrentHashMap<String, CollectionWatch<CollectionStateWatcher>> collectionWatches = new ConcurrentHashMap<>();
+
+  private ConcurrentHashMap<String, CollectionWatch<CollectionPropsWatcher>> collectionPropsWatches = new ConcurrentHashMap<>();
 
   private final ExecutorService notifications = ExecutorUtil.newMDCAwareCachedThreadPool("watches");
 
@@ -197,10 +203,10 @@ public class ZkStateReader implements Closeable {
     return new AutoScalingConfig(map);
   }
 
-  private static class CollectionWatch {
+  private static class CollectionWatch <T> {
 
     int coreRefCount = 0;
-    Set<CollectionStateWatcher> stateWatchers = ConcurrentHashMap.newKeySet();
+    Set<T> stateWatchers = ConcurrentHashMap.newKeySet();
 
     public boolean canBeRemoved() {
       return coreRefCount + stateWatchers.size() == 0;
@@ -936,6 +942,53 @@ public class ZkStateReader implements Closeable {
   }
 
   /**
+   * Get collection properties for a given collection. If the collection is watched, simply return it from the cache,
+   * otherwise fetch it directly from zookeeper.
+   */
+  public Map<String, String> getCollectionProperties(final String collection) {
+    Map<String, String> properties = watchedCollectionProps.get(collection);
+    if (properties == null) {
+      try {
+        properties = fetchCollectionProperties(collection, null);
+        // Not storing the value in watchedCollectionProps, because it can gat stale, since we have no watcher set.
+      } catch (Exception e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Error reading collection properties", SolrZkClient.checkInterrupted(e));
+      }
+    }
+
+    return properties;
+  }
+
+  static String getCollectionPropsPath(final String collection) {
+    return COLLECTIONS_ZKNODE + '/' + collection + '/' + COLLECTION_PROPS_ZKNODE;
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, String> fetchCollectionProperties(String collection, Watcher watcher) throws KeeperException, InterruptedException {
+    final String znodePath = getCollectionPropsPath(collection);
+    while (true) {
+      try {
+        Stat stat = new Stat();
+        byte[] data = zkClient.getData(znodePath, watcher, stat, true);
+        return (Map<String, String>) Utils.fromJSON(data);
+      } catch (ClassCastException e) {
+        throw new SolrException(ErrorCode.SERVER_ERROR, "Unable to parse collection properties for collection " + collection, e);
+      } catch (KeeperException.NoNodeException e) {
+        if (watcher != null) {
+          // Leave an exists watch in place in case a collectionprops.json is created later.
+          Stat exists = zkClient.exists(znodePath, watcher, true);
+          if (exists != null) {
+            // Rare race condition, we tried to fetch the data and couldn't find it, then we found it exists.
+            // Loop and try again.
+            continue;
+          }
+        }
+        return Collections.emptyMap();
+      }
+    }
+  }
+
+  /**
    * Returns the content of /security.json from ZooKeeper as a Map
    * If the files doesn't exist, it returns null.
    */
@@ -1059,6 +1112,56 @@ public class ZkStateReader implements Closeable {
     }
   }
 
+  /** Watches collection properties */
+  class PropsWatcher implements Watcher {
+    private final String coll;
+
+    PropsWatcher(String coll) {
+      this.coll = coll;
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+      // session events are not change events, and do not remove the watcher
+      if (EventType.None.equals(event.getType())) {
+        return;
+      }
+
+      if (!collectionPropsWatches.containsKey(coll)) {
+        // No one can be notified of the change, we can ignore it and "unset" the watch
+        LOG.debug("Ignoring property change for collection {}", coll);
+        return;
+      }
+
+      LOG.info("A collection property change: [{}] for collection [{}] has occurred - updating...",
+          event, coll);
+
+      refreshAndWatch(true);
+    }
+
+    /**
+     * Refresh collection properties from ZK and leave a watch for future changes. Updates the properties in
+     * watchedCollectionProps with the results of the refresh. Optionally notifies watchers
+     */
+    void refreshAndWatch(boolean notifyWatchers) {
+      try {
+        Map<String, String> properties = fetchCollectionProperties(coll, this);
+        watchedCollectionProps.put(coll, properties);
+        if (notifyWatchers) {
+          notifyPropsWatchers(coll, properties);
+        }
+      } catch (KeeperException.SessionExpiredException | KeeperException.ConnectionLossException e) {
+        LOG.warn("ZooKeeper watch triggered, but Solr cannot talk to ZK: [{}]", e.getMessage());
+      } catch (KeeperException e) {
+        LOG.error("Lost collection property watcher for {} due to ZK error", coll, e);
+        throw new ZooKeeperException(ErrorCode.SERVER_ERROR, "A ZK error has occurred", e);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+        LOG.error("Lost collection property watcher for {} due to the thread being interrupted", coll, e);
+      }
+    }
+  }
+
   /** Watches /collections children . */
   class CollectionsChildWatcher implements Watcher {
 
@@ -1156,7 +1259,7 @@ public class ZkStateReader implements Closeable {
       }
     }
   }
-  
+
   public static String getCollectionPathRoot(String coll) {
     return COLLECTIONS_ZKNODE+"/"+coll;
   }
@@ -1183,7 +1286,7 @@ public class ZkStateReader implements Closeable {
     collectionWatches.compute(collection, (k, v) -> {
       if (v == null) {
         reconstructState.set(true);
-        v = new CollectionWatch();
+        v = new CollectionWatch<>();
       }
       v.coreRefCount++;
       return v;
@@ -1191,6 +1294,20 @@ public class ZkStateReader implements Closeable {
     if (reconstructState.get()) {
       new StateWatcher(collection).refreshAndWatch();
     }
+
+    AtomicBoolean addPropsWatch = new AtomicBoolean(false);
+    collectionPropsWatches.compute(collection, (k, v) -> {
+      if (v == null) {
+        addPropsWatch.set(true);
+        v = new CollectionWatch<>();
+      }
+      v.coreRefCount++;
+      return v;
+    });
+
+    if (addPropsWatch.get()) {
+      new PropsWatcher(collection).refreshAndWatch(false);
+    }
   }
 
   /**
@@ -1223,6 +1340,18 @@ public class ZkStateReader implements Closeable {
         constructState(Collections.emptySet());
       }
     }
+
+    collectionPropsWatches.compute(collection, (k, v) -> {
+      if (v == null)
+        return null;
+      if (v.coreRefCount > 0)
+        v.coreRefCount--;
+      if (v.canBeRemoved()) {
+        watchedCollectionProps.remove(collection);
+        return null;
+      }
+      return v;
+    });
   }
 
   /**
@@ -1232,7 +1361,7 @@ public class ZkStateReader implements Closeable {
     AtomicBoolean watchSet = new AtomicBoolean(false);
     collectionWatches.compute(collection, (k, v) -> {
       if (v == null) {
-        v = new CollectionWatch();
+        v = new CollectionWatch<>();
         watchSet.set(true);
       }
       v.stateWatchers.add(stateWatcher);
@@ -1374,6 +1503,35 @@ public class ZkStateReader implements Closeable {
     return updated;
   }
 
+  public void registerCollectionPropsWatcher(final String collection, CollectionPropsWatcher propsWatcher) {
+    AtomicBoolean watchSet = new AtomicBoolean(false);
+    collectionPropsWatches.compute(collection, (k, v) -> {
+      if (v == null) {
+        v = new CollectionWatch<>();
+        watchSet.set(true);
+      }
+      v.stateWatchers.add(propsWatcher);
+      return v;
+    });
+
+    if (watchSet.get()) {
+      new PropsWatcher(collection).refreshAndWatch(false);
+    }
+  }
+
+  public void removeCollectionPropsWatcher(String collection, CollectionPropsWatcher watcher) {
+    collectionPropsWatches.compute(collection, (k, v) -> {
+      if (v == null)
+        return null;
+      v.stateWatchers.remove(watcher);
+      if (v.canBeRemoved()) {
+        watchedCollectionProps.remove(collection);
+        return null;
+      }
+      return v;
+    });
+  }
+
   public static class ConfigData {
     public Map<String, Object> data;
     public int version;
@@ -1583,4 +1741,43 @@ public class ZkStateReader implements Closeable {
 
   }
 
+  private void notifyPropsWatchers(String collection, Map<String, String> properties) {
+    try {
+      notifications.submit(new PropsNotification(collection, properties));
+    }
+    catch (RejectedExecutionException e) {
+      if (!closed) {
+        LOG.error("Couldn't run collection properties notifications for {}", collection, e);
+      }
+    }
+  }
+
+  private class PropsNotification implements Runnable {
+
+    final String collection;
+    final Map<String, String> collectionProperties;
+
+    private PropsNotification(String collection, Map<String, String> collectionProperties) {
+      this.collection = collection;
+      this.collectionProperties = collectionProperties;
+    }
+
+    @Override
+    public void run() {
+      List<CollectionPropsWatcher> watchers = new ArrayList<>();
+      collectionPropsWatches.compute(collection, (k, v) -> {
+        if (v == null)
+          return null;
+        watchers.addAll(v.stateWatchers);
+        return v;
+      });
+      for (CollectionPropsWatcher watcher : watchers) {
+        if (watcher.onStateChanged(collectionProperties)) {
+          removeCollectionPropsWatcher(collection, watcher);
+        }
+      }
+    }
+
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/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 04acd9f..cdd9458 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
@@ -64,4 +64,14 @@ public interface CollectionAdminParams {
    */
   public static final Collection<String> INDEX_BACKUP_STRATEGIES =
       Arrays.asList(COPY_FILES_STRATEGY, NO_INDEX_BACKUP_STRATEGY);
+
+  /**
+   * Name of collection property to set
+   */
+  public static final String PROPERTY_NAME = "propertyName";
+
+  /**
+   * Value of collection property to set
+   */
+  public static final String PROPERTY_VALUE = "propertyValue";
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
index d404d60..cbf14cc 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
@@ -91,6 +91,7 @@ public interface CollectionParams {
     ADDROLE(true, LockLevel.NONE),
     REMOVEROLE(true, LockLevel.NONE),
     CLUSTERPROP(true, LockLevel.NONE),
+    COLLECTIONPROP(true, LockLevel.COLLECTION),
     REQUESTSTATUS(false, LockLevel.NONE),
     DELETESTATUS(false, LockLevel.NONE),
     ADDREPLICA(true, LockLevel.SHARD),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/resources/apispec/collections.collection.Commands.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/resources/apispec/collections.collection.Commands.json b/solr/solrj/src/resources/apispec/collections.collection.Commands.json
index 0cd3644..b4e87b0 100644
--- a/solr/solrj/src/resources/apispec/collections.collection.Commands.json
+++ b/solr/solrj/src/resources/apispec/collections.collection.Commands.json
@@ -84,7 +84,7 @@
     "balance-shard-unique":{
       "type":"object",
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#balanceshardunique",
-      "description": "Insures a property is distributed equally across all physical nodes of a collection. If the property already exists on a replica, effort is made to leave it there. However, if it does not exist on any repica, a shard will be chosen and the property added.",
+      "description": "Ensures a property is distributed equally across all physical nodes of a collection. If the property already exists on a replica, effort is made to leave it there. However, if it does not exist on any repica, a shard will be chosen and the property added.",
       "properties":{
         "property":{
           "type":"string",
@@ -171,6 +171,24 @@
         }
       },
       "required":["shard","replica","property"]
+    },
+    "set-collection-property": {
+      "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#collectionprop",
+      "description": "Sets a property for the collection",
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The name of the property"
+        },
+        "value": {
+          "type": "string",
+          "description": "The value of the property. When not provided, the property is deleted"
+        }
+      },
+      "required": [
+        "name"
+      ]
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java b/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
index 870aa34..876df2b 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/CollectionAdminRequestRequiredParamsTest.java
@@ -26,6 +26,8 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CollectionAdminParams;
+import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.params.ShardParams;
 import org.apache.solr.common.params.SolrParams;
 
@@ -56,7 +58,12 @@ public class CollectionAdminRequestRequiredParamsTest extends LuceneTestCase {
     CollectionAdminRequest.ClusterProp request = CollectionAdminRequest.setClusterProperty("foo","bar");
     assertContainsParams(request.getParams(), ACTION, NAME, "val");
   }
-  
+
+  public void testCollectionProp() {
+    final CollectionAdminRequest.CollectionProp request = CollectionAdminRequest.setCollectionProperty("foo", "bar", "baz");
+    assertContainsParams(request.getParams(), ACTION, CoreAdminParams.NAME, CollectionAdminParams.PROPERTY_NAME, CollectionAdminParams.PROPERTY_VALUE);
+  }
+
   public void testAddRole() {
     CollectionAdminRequest.AddRole request = CollectionAdminRequest.addRole("node","role");
     assertContainsParams(request.getParams(), ACTION, "node", "role");

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/cfafc47e/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
index 9573b3b..ae563f1 100644
--- a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
@@ -51,6 +51,13 @@ public class TestV1toV2ApiMapper extends LuceneTestCase {
     assertEquals("shard1", Utils.getObjectByPath(m,true,"/add-replica/shard"));
     assertEquals("NRT", Utils.getObjectByPath(m,true,"/add-replica/type"));
 
+    CollectionAdminRequest.CollectionProp collectionProp = CollectionAdminRequest.setCollectionProperty("mycoll", "prop", "value");
+    v2r = V1toV2ApiMapper.convert(collectionProp).build();
+    m = (Map) Utils.fromJSON(ContentStreamBase.create(new BinaryRequestWriter(), v2r).getStream());
+    assertEquals("/c", v2r.getPath());
+    assertEquals("mycoll", Utils.getObjectByPath(m,true,"/set-collection-property/name"));
+    assertEquals("prop", Utils.getObjectByPath(m,true,"/set-collection-property/propertyName"));
+    assertEquals("value", Utils.getObjectByPath(m,true,"/set-collection-property/propertyValue"));
 
 
   }