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

[1/8] lucene-solr:jira/solr-11670-2: LUCENE-8126: Add "s2" to SpatialPrefixTreeFactory lookup table

Repository: lucene-solr
Updated Branches:
  refs/heads/jira/solr-11670-2 f9ad3fb68 -> 9f5103a3c


LUCENE-8126: Add "s2" to SpatialPrefixTreeFactory lookup table


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

Branch: refs/heads/jira/solr-11670-2
Commit: e0d6465af94b6c6f7b8d570dee97c98de572c876
Parents: d789516
Author: David Smiley <ds...@apache.org>
Authored: Mon Mar 5 10:52:02 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Mon Mar 5 10:52:02 2018 -0500

----------------------------------------------------------------------
 .../lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java      | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e0d6465a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
index 67480d5..d88e41a 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/tree/SpatialPrefixTreeFactory.java
@@ -43,6 +43,7 @@ public abstract class SpatialPrefixTreeFactory {
    * If it's neither of these, then "geohash" is chosen for a geo context, otherwise "quad" is chosen.
    */
   public static SpatialPrefixTree makeSPT(Map<String,String> args, ClassLoader classLoader, SpatialContext ctx) {
+    //TODO refactor to use Java SPI like how Lucene already does for codecs/postingsFormats, etc
     SpatialPrefixTreeFactory instance;
     String cname = args.get(PREFIX_TREE);
     if (cname == null)
@@ -53,6 +54,8 @@ public abstract class SpatialPrefixTreeFactory {
       instance = new QuadPrefixTree.Factory();
     else if ("packedQuad".equalsIgnoreCase(cname))
       instance = new PackedQuadPrefixTree.Factory();
+    else if ("s2".equalsIgnoreCase(cname))
+      instance = new S2PrefixTree.Factory();
     else {
       try {
         Class<?> c = classLoader.loadClass(cname);


[2/8] lucene-solr:jira/solr-11670-2: SOLR-12020: fix refinement of terms facet on date field (don't use Date.toString)

Posted by ab...@apache.org.
SOLR-12020: fix refinement of terms facet on date field (don't use Date.toString)


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

Branch: refs/heads/jira/solr-11670-2
Commit: fc2fd1dc25dbfda1764e044090dce60c22f724c6
Parents: e0d6465
Author: yonik <yo...@apache.org>
Authored: Mon Mar 5 11:03:48 2018 -0500
Committer: yonik <yo...@apache.org>
Committed: Mon Mar 5 11:03:48 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../solr/search/facet/FacetFieldProcessor.java  |  7 +++++--
 .../apache/solr/search/facet/FacetModule.java   | 22 +++++++++++++++++++-
 .../solr/search/facet/TestJsonFacets.java       | 11 ++++++++++
 4 files changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fc2fd1dc/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index d531311..1639e14 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -247,6 +247,9 @@ Bug Fixes
 
 * SOLR-12011: Consistence problem when in-sync replicas are DOWN. (Cao Manh Dat)
 
+* SOLR-12020: JSON Facet API: terms facet on date field fails in refinement phase with
+  "Invalid Date String" error. (yonik)
+
 Optimizations
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fc2fd1dc/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
index 143cfd6..50f4676 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetFieldProcessor.java
@@ -18,8 +18,10 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.time.Instant;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
@@ -682,9 +684,10 @@ abstract class FacetFieldProcessor extends FacetProcessor<FacetField> {
     FieldType ft = sf.getType();
     bucketVal = ft.toNativeType(bucketVal);  // refinement info passed in as JSON will cause int->long and float->double
     bucket.add("val", bucketVal);
-    // String internal = ft.toInternal( tobj.toString() );  // TODO - we need a better way to get from object to query...
 
-    Query domainQ = ft.getFieldQuery(null, sf, bucketVal.toString());
+    // fieldQuery currently relies on a string input of the value...
+    String bucketStr = bucketVal instanceof Date ? Instant.ofEpochMilli(((Date)bucketVal).getTime()).toString() : bucketVal.toString();
+    Query domainQ = ft.getFieldQuery(null, sf, bucketStr);
 
     fillBucket(bucket, domainQ, null, skip, facetInfo);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fc2fd1dc/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index 7fc016b..759ece7 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -17,7 +17,9 @@
 package org.apache.solr.search.facet;
 
 import java.io.IOException;
+import java.time.Instant;
 import java.util.Collection;
+import java.util.Date;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -38,7 +40,9 @@ import org.apache.solr.handler.component.ShardResponse;
 import org.apache.solr.search.QueryContext;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.RTimer;
+import org.noggit.CharArr;
 import org.noggit.JSONUtil;
+import org.noggit.JSONWriter;
 import org.noggit.ObjectBuilder;
 
 public class FacetModule extends SearchComponent {
@@ -234,7 +238,23 @@ public class FacetModule extends SearchComponent {
 
       Map<String,Object> finfo = new HashMap<>(1);
       finfo.put(FACET_REFINE, refinement);
-      String finfoStr = JSONUtil.toJSON(finfo, -1);
+
+      // String finfoStr = JSONUtil.toJSON(finfo, -1);  // this doesn't handle formatting of Date objects the way we want
+      CharArr out = new CharArr();
+      JSONWriter jsonWriter = new JSONWriter(out, -1) {
+        @Override
+        public void handleUnknownClass(Object o) {
+          // handle date formatting correctly
+          if (o instanceof Date) {
+            String s = Instant.ofEpochMilli(((Date)o).getTime()).toString();
+            writeString(s);
+            return;
+          }
+          super.handleUnknownClass(o);
+        }
+      };
+      jsonWriter.write(finfo);
+      String finfoStr = out.toString();
       // System.err.println("##################### REFINE=" + finfoStr);
       shardsRefineRequest.params.add(FACET_INFO, finfoStr);
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/fc2fd1dc/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
index 753c7dc..632c006 100644
--- a/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
+++ b/solr/core/src/test/org/apache/solr/search/facet/TestJsonFacets.java
@@ -746,6 +746,17 @@ public class TestJsonFacets extends SolrTestCaseHS {
       );
     }
 
+    // test field faceting on date field
+    client.testJQ(params(p, "q", "*:*"
+        , "json.facet", "{" +
+            " f1:{${terms}  type:field, field:${date}}" +
+            "}"
+        )
+        , "facets=={count:6 " +
+            ",f1:{ buckets:[ {val:'2001-01-01T01:01:01Z', count:1},{val:'2001-02-03T01:02:03Z', count:1},{val:'2002-02-02T02:02:02Z', count:1},{val:'2002-03-01T03:02:01Z', count:1},{val:'2003-03-03T03:03:03Z', count:1} ] }" +
+            "}"
+    );
+
 
     // percentiles 0,10,50,90,100
     // catA: 2.0 2.2 3.0 3.8 4.0


[4/8] lucene-solr:jira/solr-11670-2: SOLR-11960: Add collection-level properties

Posted by ab...@apache.org.
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/c1a44251
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/c1a44251
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/c1a44251

Branch: refs/heads/jira/solr-11670-2
Commit: c1a44251fefabb0ed743f1bdaf287ac89ac38758
Parents: c4c391a
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 16:59:56 2018 -0800

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../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, 879 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c1a44251/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 1639e14..f9ca9e6 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -184,6 +184,8 @@ New Features
 
 * 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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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/c1a44251/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"));
 
 
   }


[3/8] lucene-solr:jira/solr-11670-2: SOLR-12020: Remove unused import

Posted by ab...@apache.org.
SOLR-12020: Remove unused import


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

Branch: refs/heads/jira/solr-11670-2
Commit: c4c391ad15098c172941bd29aa41f564a338ae47
Parents: fc2fd1d
Author: Tomas Fernandez Lobbe <tf...@apache.org>
Authored: Mon Mar 5 16:56:33 2018 -0800
Committer: Tomas Fernandez Lobbe <tf...@apache.org>
Committed: Mon Mar 5 16:56:33 2018 -0800

----------------------------------------------------------------------
 solr/core/src/java/org/apache/solr/search/facet/FacetModule.java | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4c391ad/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
index 759ece7..9d94281 100644
--- a/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
+++ b/solr/core/src/java/org/apache/solr/search/facet/FacetModule.java
@@ -41,7 +41,6 @@ import org.apache.solr.search.QueryContext;
 import org.apache.solr.search.SyntaxError;
 import org.apache.solr.util.RTimer;
 import org.noggit.CharArr;
-import org.noggit.JSONUtil;
 import org.noggit.JSONWriter;
 import org.noggit.ObjectBuilder;
 


[8/8] lucene-solr:jira/solr-11670-2: Merge branch 'master' into jira/solr-11670-2

Posted by ab...@apache.org.
Merge branch 'master' into jira/solr-11670-2


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

Branch: refs/heads/jira/solr-11670-2
Commit: 9f5103a3cbd33659bcdcb1247ae841487cd4440e
Parents: f9ad3fb 9cec222
Author: Andrzej Bialecki <ab...@apache.org>
Authored: Tue Mar 6 18:28:26 2018 +0100
Committer: Andrzej Bialecki <ab...@apache.org>
Committed: Tue Mar 6 18:28:26 2018 +0100

----------------------------------------------------------------------
 .../prefix/tree/SpatialPrefixTreeFactory.java   |   3 +
 solr/CHANGES.txt                                |   7 +
 solr/contrib/prometheus-exporter/README.md      |  21 --
 solr/contrib/prometheus-exporter/README.txt     |  22 +-
 .../solr/prometheus/exporter/SolrExporter.java  |  13 +-
 .../org/apache/solr/cloud/ZkController.java     |   2 +-
 .../solr/cloud/api/collections/BackupCmd.java   |   2 +
 .../solr/cloud/api/collections/RestoreCmd.java  |   3 +
 .../cloud/autoscaling/ComputePlanAction.java    |   2 +-
 .../cloud/autoscaling/ScheduledTrigger.java     |  25 +-
 .../apache/solr/core/backup/BackupManager.java  |  43 +++-
 .../solr/handler/admin/CollectionsHandler.java  |  11 +
 .../solr/search/facet/FacetFieldProcessor.java  |   7 +-
 .../apache/solr/search/facet/FacetModule.java   |  23 +-
 .../apache/solr/cloud/CollectionPropsTest.java  | 256 +++++++++++++++++++
 .../solr/cloud/CollectionsAPISolrJTest.java     |  36 +++
 .../core/snapshots/TestSolrCloudSnapshots.java  |  15 ++
 .../solr/handler/admin/TestCollectionAPIs.java  |  19 +-
 .../solr/search/facet/TestJsonFacets.java       |  11 +
 solr/solr-ref-guide/src/collections-api.adoc    |  43 ++++
 ...toring-solr-with-prometheus-and-grafana.adoc | 120 +++++----
 .../src/solrcloud-autoscaling-triggers.adoc     |   5 +-
 .../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 +
 32 files changed, 1049 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9f5103a3/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
----------------------------------------------------------------------
diff --cc solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
index 7c7e4e7,1aa7a0a..f673fad
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
@@@ -34,8 -33,6 +34,7 @@@ import java.util.concurrent.TimeUnit
  import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
  import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
  import org.apache.solr.common.SolrException;
- import org.apache.solr.common.params.CollectionParams;
 +import org.apache.solr.common.util.TimeSource;
  import org.apache.solr.core.SolrResourceLoader;
  import org.apache.solr.util.DateMathParser;
  import org.apache.solr.util.TimeZoneUtils;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9f5103a3/solr/solrj/src/java/org/apache/solr/common/cloud/ZkStateReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9f5103a3/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
----------------------------------------------------------------------


[6/8] lucene-solr:jira/solr-11670-2: SOLR-11795: use README.txt rather than .md and add description for default values

Posted by ab...@apache.org.
SOLR-11795: use README.txt rather than .md and add description for default values


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

Branch: refs/heads/jira/solr-11670-2
Commit: 122271f28deb739e3d7fe84c86c6a359d367abd8
Parents: 4bdc99a
Author: koji <ko...@apache.org>
Authored: Tue Mar 6 17:14:15 2018 +0900
Committer: koji <ko...@apache.org>
Committed: Tue Mar 6 17:14:15 2018 +0900

----------------------------------------------------------------------
 solr/contrib/prometheus-exporter/README.md      |  21 ----
 solr/contrib/prometheus-exporter/README.txt     |  22 +++-
 .../solr/prometheus/exporter/SolrExporter.java  |  13 +-
 ...toring-solr-with-prometheus-and-grafana.adoc | 120 ++++++++++++-------
 4 files changed, 102 insertions(+), 74 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/122271f2/solr/contrib/prometheus-exporter/README.md
----------------------------------------------------------------------
diff --git a/solr/contrib/prometheus-exporter/README.md b/solr/contrib/prometheus-exporter/README.md
deleted file mode 100644
index 888f237..0000000
--- a/solr/contrib/prometheus-exporter/README.md
+++ /dev/null
@@ -1,21 +0,0 @@
-Welcome to Apache Solr Prometheus Exporter
-========
-
-Apache Solr Prometheus Exporter (solr-exporter) provides a way for you to expose metrics for Solr to Prometheus.
-
-# Getting Started With Solr Prometheus Exporter
-
-For information on how to get started with solr-exporter please see:
- * [Solr Reference Guide's section on Monitoring Solr with Prometheus and Grafana](https://lucene.apache.org/solr/guide/monitoring-solr-with-prometheus-and-grafana.html)
-
-# Getting Started With Solr
-
-For information on how to get started with solr please see:
- * [solr/README.txt](../../README.txt)
- * [Solr Tutorial](https://lucene.apache.org/solr/guide/solr-tutorial.html)
-
-# How To Contribute
-
-For information on how to contribute see:
- * http://wiki.apache.org/lucene-java/HowToContribute
- * http://wiki.apache.org/solr/HowToContribute

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/122271f2/solr/contrib/prometheus-exporter/README.txt
----------------------------------------------------------------------
diff --git a/solr/contrib/prometheus-exporter/README.txt b/solr/contrib/prometheus-exporter/README.txt
index 42061c0..888f237 100644
--- a/solr/contrib/prometheus-exporter/README.txt
+++ b/solr/contrib/prometheus-exporter/README.txt
@@ -1 +1,21 @@
-README.md
\ No newline at end of file
+Welcome to Apache Solr Prometheus Exporter
+========
+
+Apache Solr Prometheus Exporter (solr-exporter) provides a way for you to expose metrics for Solr to Prometheus.
+
+# Getting Started With Solr Prometheus Exporter
+
+For information on how to get started with solr-exporter please see:
+ * [Solr Reference Guide's section on Monitoring Solr with Prometheus and Grafana](https://lucene.apache.org/solr/guide/monitoring-solr-with-prometheus-and-grafana.html)
+
+# Getting Started With Solr
+
+For information on how to get started with solr please see:
+ * [solr/README.txt](../../README.txt)
+ * [Solr Tutorial](https://lucene.apache.org/solr/guide/solr-tutorial.html)
+
+# How To Contribute
+
+For information on how to contribute see:
+ * http://wiki.apache.org/lucene-java/HowToContribute
+ * http://wiki.apache.org/solr/HowToContribute

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/122271f2/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java
----------------------------------------------------------------------
diff --git a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java
index 752f176..01ebd74 100644
--- a/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java
+++ b/solr/contrib/prometheus-exporter/src/java/org/apache/solr/prometheus/exporter/SolrExporter.java
@@ -56,31 +56,31 @@ public class SolrExporter {
   private static final String ARG_PORT_METAVAR = "PORT";
   private static final String ARG_PORT_DEST = "port";
   private static final Integer ARG_PORT_DEFAULT = 9983;
-  private static final String ARG_PORT_HELP = "solr-exporter listen port";
+  private static final String ARG_PORT_HELP = "Specify the solr-exporter HTTP listen port; default is " + String.valueOf(ARG_PORT_DEFAULT) + ".";
 
   private static final String[] ARG_BASE_URL_FLAGS = { "-b", "--baseurl" };
   private static final String ARG_BASE_URL_METAVAR = "BASE_URL";
   private static final String ARG_BASE_URL_DEST = "baseUrl";
   private static final String ARG_BASE_URL_DEFAULT = "";
-  private static final String ARG_BASE_URL_HELP = "specify Solr base URL when connecting to Solr in standalone mode (for example 'http://localhost:8983/solr')";
+  private static final String ARG_BASE_URL_HELP = "Specify the Solr base URL when connecting to Solr in standalone mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'http://localhost:8983/solr'.";
 
   private static final String[] ARG_ZK_HOST_FLAGS = { "-z", "--zkhost" };
   private static final String ARG_ZK_HOST_METAVAR = "ZK_HOST";
   private static final String ARG_ZK_HOST_DEST = "zkHost";
   private static final String ARG_ZK_HOST_DEFAULT = "";
-  private static final String ARG_ZK_HOST_HELP = "specify ZooKeeper connection string when connecting to Solr in SolrCloud mode (for example 'localhost:2181/solr')";
+  private static final String ARG_ZK_HOST_HELP = "Specify the ZooKeeper connection string when connecting to Solr in SolrCloud mode. If omitted both the -b parameter and the -z parameter, connect to http://localhost:8983/solr. For example 'localhost:2181/solr'.";
 
   private static final String[] ARG_CONFIG_FLAGS = { "-f", "--config-file" };
   private static final String ARG_CONFIG_METAVAR = "CONFIG";
   private static final String ARG_CONFIG_DEST = "configFile";
   private static final String ARG_CONFIG_DEFAULT = "./conf/solr-exporter-config.xml";
-  private static final String ARG_CONFIG_HELP = "specify configuration file";
+  private static final String ARG_CONFIG_HELP = "Specify the configuration file; default is " + ARG_CONFIG_DEFAULT + ".";
 
-  private static final String[] ARG_NUM_THREADS_FLAGS = { "-n", "--num-thread" };
+  private static final String[] ARG_NUM_THREADS_FLAGS = { "-n", "--num-threads" };
   private static final String ARG_NUM_THREADS_METAVAR = "NUM_THREADS";
   private static final String ARG_NUM_THREADS_DEST = "numThreads";
   private static final Integer ARG_NUM_THREADS_DEFAULT = 1;
-  private static final String ARG_NUM_THREADS_HELP = "specify number of threads";
+  private static final String ARG_NUM_THREADS_HELP = "Specify the number of threads. solr-exporter creates a thread pools for request to Solr. If you need to improve request latency via solr-exporter, you can increase the number of threads; default is " + String.valueOf(ARG_NUM_THREADS_DEFAULT) + ".";
 
   private int port;
   private SolrClient solrClient;
@@ -135,7 +135,6 @@ public class SolrExporter {
   public void stop() {
     this.httpServer.stop();
     this.registry.unregister(this.collector);
-//    this.collector.shutdown();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/122271f2/solr/solr-ref-guide/src/monitoring-solr-with-prometheus-and-grafana.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/monitoring-solr-with-prometheus-and-grafana.adoc b/solr/solr-ref-guide/src/monitoring-solr-with-prometheus-and-grafana.adoc
index 0d5295b..5348cfb 100644
--- a/solr/solr-ref-guide/src/monitoring-solr-with-prometheus-and-grafana.adoc
+++ b/solr/solr-ref-guide/src/monitoring-solr-with-prometheus-and-grafana.adoc
@@ -25,6 +25,7 @@ image::images/monitoring-solr-with-prometheus-and-grafana/solr-exporter-diagram.
 
 This feature is experimental status.
 
+
 == Running solr-exporter
 
 You can start solr-exporter by running `./bin/solr-exporter` from the solr-exporter directory.
@@ -56,26 +57,28 @@ See command help:
 [source,plain]
 ----
 $ ./bin/solr-exporter -h
-usage: SolrCollector [-h] [-v] [-p PORT] [-b BASE_URL] [-z ZK_HOST] [-f CONFIG]
-                     [-n NUM_THREADS]
-
-Prometheus exporter for Apache Solr.
-
-optional arguments:
-  -h, --help             show this help message and exit
-  -p PORT, --port PORT   solr-exporter listen port
-  -b BASE_URL, --baseurl BASE_URL
-                         specify Solr base URL when connecting  to Solr in standalone mode (for
-                         example 'http://localhost:8983/solr')
-  -z ZK_HOST, --zkhost ZK_HOST
-                         specify  ZooKeeper  connection  string  when  connecting  to  Solr  in
-                         SolrCloud mode (for example 'localhost:2181/solr')
-  -f CONFIG, --config-file CONFIG
-                         specify configuration file
-  -n NUM_THREADS, --num-thread NUM_THREADS
-                         specify number of threads
 ----
 
+=== Command Line Parameters
+
+`-p`, `--port`::
+Specify the solr-exporter HTTP listen port; default is `9983`.
+
+`-b`, `--baseurl`::
+Specify the Solr base URL when connecting to Solr in standalone mode. If omitted both the `-b` parameter and the `-z` parameter, connect to `http://localhost:8983/solr`. For example `http://localhost:8983/solr`.
+
+`-z`, `--zkhost`::
+Specify the ZooKeeper connection string when connecting to Solr in SolrCloud mode. If omitted both the `-b` parameter and the `-z` parameter, connect to `http://localhost:8983/solr`. For example `localhost:2181/solr`.
+
+`-f`, `--config-file`::
+Specify the configuration file; default is `./conf/solr-exporter-config.xml`.
+
+`-n`, `--num-threads`::
+Specify the number of threads. solr-exporter creates a thread pools for request to Solr. If you need to improve request latency via solr-exporter, you can increase the number of threads; default is `1`.
+
+
+== Metrics Exposition URL
+
 The Solr's metrics exposed by solr-exporter can see at the following URL.
 
 http://localhost:9983/metrics[http://localhost:9983/metrics]
@@ -217,32 +220,70 @@ The configuration is in `./config/solr-exporter-config.xml`. An example with all
 </config>
 ----
 
-|===
-|Name|Description
+===  solr-exporter Configuration Tags and Elements
+
+`/config/rules/ping`::
+Scrape <<ping.adoc#ping,Ping>> response.
+
+`/config/rules/metrics`::
+Scrape <<metrics-reporting.adoc#metrics-api,Metrics API>> response.
+
+`/config/rules/collections`::
+Scrape <<collections-api.adoc#collections-api,Collections API>> response.
+
+`/config/rules/search`::
+Scrape <<searching.adoc#searching,Search API>> response.
+
+`*[@query]`::
+Query parameter for each features. You can specify `collection`, `core`, `path`, and `params`.
 
-|ping|Scrape <<ping.adoc#ping,Ping>> response.
-|metrics|Scrape <<metrics-reporting.adoc#metrics-api,Metrics API>> response.
-|collections|Scrape <<collections-api.adoc#collections-api,Collections API>> response.
-|search|Scrape <<searching.adoc#searching,Search API>> response.
-|*.query|Query parameter for each features. You can specify `collection`, `core`, `path`, and `params`.
-|*.jsonQueries|JSON Query that is jq syntax. For more details, see https://stedolan.github.io/jq/manual/[https://stedolan.github.io/jq/manual/].
-|===
+`*[@jsonQueries]`::
+JSON Query that is jq syntax. For more details, see https://stedolan.github.io/jq/manual/[https://stedolan.github.io/jq/manual/].
 
 jq query has to output JSON in the following format.
 
 [source,json]
 ----
 {
-  name         : "solr_ping",
-  type         : "GAUGE",
-  help         : "See following URL: https://lucene.apache.org/solr/guide/ping.html",
-  label_names  : ["base_url","core"],
-  label_values : ["http://localhost:8983/solr","collection1"],
-  value        : 1.0
+  "name"         : "solr_ping",
+  "type"         : "GAUGE",
+  "help"         : "See following URL: https://lucene.apache.org/solr/guide/ping.html",
+  "label_names"  : ["base_url","core"],
+  "label_values" : ["http://localhost:8983/solr","collection1"],
+  "value"        : 1.0
 }
 ----
 
-It will be converted to the following exposition format.
+===  solr-exporter Exposition Format
+
+solr-exporter converts the JSON to the following exposition format:
+
+[source,plain]
+----
+# TYPE <name> <type>
+# HELP <name> <help>
+<name>{<label_names[0]>=<label_values[0]>,<label_names[1]>=<labelvalues[1]>,...} <value>
+----
+
+`name`::
+The metric name to set. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
+
+`type`::
+The type of the metric, can be `COUNTER`, `GAUGE`, `SUMMARY`, `HISTOGRAM` or `UNTYPED`. For more detauils, see https://prometheus.io/docs/concepts/metric_types/[https://prometheus.io/docs/concepts/metric_types/].
+
+`help`::
+Help text for the metric.
+
+`label_names`::
+Label names for the metric. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
+
+`label_values`::
+Label values for the metric. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
+
+`value`::
+Value for the metric. Value must be set to Double type.
+
+For example, solr-exporter converts the above JSON to the following:
 
 [source,plain]
 ----
@@ -251,17 +292,6 @@ It will be converted to the following exposition format.
 solr_ping{base_url="http://localhost:8983/solr",core="collection1"} 1.0
 ----
 
-|===
-|Name|Description
-
-|name|The metric name to set. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
-|type|The type of the metric, can be `COUNTER`, `GAUGE`, `SUMMARY`, `HISTOGRAM` or `UNTYPED`. For more detauils, see https://prometheus.io/docs/concepts/metric_types/[https://prometheus.io/docs/concepts/metric_types/].
-|help|Help text for the metric.
-|label_names|Label names for the metric. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
-|label_values|Label values for the metric. For more details, see https://prometheus.io/docs/practices/naming/[https://prometheus.io/docs/practices/naming/].
-|value|Value for the metric. Value must be set to Double type.
-|===
-
 
 == Prometheus Configuration
 


[7/8] lucene-solr:jira/solr-11670-2: SOLR-11066: Added examples of startTime in ref guide. Added null check for processor and code comments to explain skipped events and lastRunAt initialization. Moved default preferredOp value to ComputePlanAction. Extr

Posted by ab...@apache.org.
SOLR-11066: Added examples of startTime in ref guide. Added null check for processor and code comments to explain skipped events and lastRunAt initialization. Moved default preferredOp value to ComputePlanAction. Extracted a constant.


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

Branch: refs/heads/jira/solr-11670-2
Commit: 9cec2221a66ee7c1571531c4ce1342f51288c38d
Parents: 122271f
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue Mar 6 21:27:05 2018 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue Mar 6 21:27:05 2018 +0530

----------------------------------------------------------------------
 .../cloud/autoscaling/ComputePlanAction.java    |  2 +-
 .../cloud/autoscaling/ScheduledTrigger.java     | 25 +++++++++++++-------
 .../src/solrcloud-autoscaling-triggers.adoc     |  5 +++-
 3 files changed, 22 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9cec2221/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
index 9eb6382..aed821a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ComputePlanAction.java
@@ -209,7 +209,7 @@ public class ComputePlanAction extends TriggerActionBase {
         }
         break;
       case SCHEDULED:
-        String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP);
+        String preferredOp = (String) event.getProperty(AutoScalingParams.PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
         CollectionParams.CollectionAction action = CollectionParams.CollectionAction.get(preferredOp);
         suggester = session.getSuggester(action);
         break;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9cec2221/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
index 9c8ee6d..1aa7a0a 100644
--- a/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
+++ b/solr/core/src/java/org/apache/solr/cloud/autoscaling/ScheduledTrigger.java
@@ -33,7 +33,6 @@ import java.util.TimeZone;
 import org.apache.solr.client.solrj.cloud.autoscaling.SolrCloudManager;
 import org.apache.solr.client.solrj.cloud.autoscaling.TriggerEventType;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.util.DateMathParser;
 import org.apache.solr.util.TimeZoneUtils;
@@ -49,6 +48,7 @@ public class ScheduledTrigger extends TriggerBase {
   private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   private static final String DEFAULT_GRACE_DURATION = "+15MINUTES";
+  private static final String LAST_RUN_AT = "lastRunAt";
   static final String ACTUAL_EVENT_TIME = "actualEventTime";
 
   private final String everyStr;
@@ -72,13 +72,18 @@ public class ScheduledTrigger extends TriggerBase {
     this.everyStr = (String) properties.get("every");
     this.graceDurationStr = (String) properties.getOrDefault("graceDuration", DEFAULT_GRACE_DURATION);
 
-    preferredOp = (String) properties.getOrDefault(PREFERRED_OP, CollectionParams.CollectionAction.MOVEREPLICA.toLower());
+    preferredOp = (String) properties.get(PREFERRED_OP);
 
     // attempt parsing to validate date math strings
     Instant startTime = parseStartTime(startTimeStr, timeZoneStr);
     DateMathParser.parseMath(null, startTime + everyStr, timeZone);
     DateMathParser.parseMath(null, startTime + graceDurationStr, timeZone);
 
+    // We set lastRunAt to be the startTime (which could be a date math expression such as 'NOW')
+    // Ordinarily, NOW will always be evaluated in this constructor so it may seem that
+    // the trigger will always fire the first time.
+    // However, the lastRunAt is overwritten with the value from ZK
+    // during restoreState() operation (which is performed before run()) so the trigger works correctly
     this.lastRunAt = startTime;
   }
 
@@ -107,13 +112,13 @@ public class ScheduledTrigger extends TriggerBase {
 
   @Override
   protected Map<String, Object> getState() {
-    return Collections.singletonMap("lastRunAt", lastRunAt.toEpochMilli());
+    return Collections.singletonMap(LAST_RUN_AT, lastRunAt.toEpochMilli());
   }
 
   @Override
   protected void setState(Map<String, Object> state) {
-    if (state.containsKey("lastRunAt")) {
-      this.lastRunAt = Instant.ofEpochMilli((Long) state.get("lastRunAt"));
+    if (state.containsKey(LAST_RUN_AT)) {
+      this.lastRunAt = Instant.ofEpochMilli((Long) state.get(LAST_RUN_AT));
     }
   }
 
@@ -163,8 +168,10 @@ public class ScheduledTrigger extends TriggerBase {
         log.warn("ScheduledTrigger was not able to run event at scheduled time: {}. Now: {}",
             nextRunTime, now);
       }
-      if (processor.process(new ScheduledEvent(getEventType(), getName(), nextRunTime.toEpochMilli(),
-          preferredOp, now.toEpochMilli(), true)))  {
+      // Even though we are skipping the event, we need to notify any listeners of the IGNORED stage
+      // so we create a dummy event with the ignored=true flag and ScheduledTriggers will do the rest
+      if (processor != null && processor.process(new ScheduledEvent(getEventType(), getName(), nextRunTime.toEpochMilli(),
+          preferredOp, now.toEpochMilli(), true))) {
         lastRunAt = nextRunTime;
         return;
       }
@@ -191,7 +198,9 @@ public class ScheduledTrigger extends TriggerBase {
 
     public ScheduledEvent(TriggerEventType eventType, String source, long eventTime, String preferredOp, long actualEventTime, boolean ignored) {
       super(eventType, source, eventTime, null, ignored);
-      properties.put(PREFERRED_OP, preferredOp);
+      if (preferredOp != null)  {
+        properties.put(PREFERRED_OP, preferredOp);
+      }
       properties.put(ACTUAL_EVENT_TIME, actualEventTime);
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/9cec2221/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
index ec09a79..08e0085 100644
--- a/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
+++ b/solr/solr-ref-guide/src/solrcloud-autoscaling-triggers.adoc
@@ -159,7 +159,10 @@ The Scheduled trigger generates events according to a fixed rate schedule.
 
 The trigger supports the following configuration:
 
-* `startTime` - (string, required) the start date/time of the schedule. This should either be an ISO-8601 date time string (the same standard used during search and indexing in Solr, thus defaulting to UTC) or be specified with the `timeZone` parameter.
+* `startTime` - (string, required) the start date/time of the schedule. This should either be a DateMath string e.g. 'NOW' or be an ISO-8601 date time string (the same standard used during search and indexing in Solr, thus defaulting to UTC) or be specified without the trailing 'Z' accompanied with the `timeZone` parameter. For example, each of the following values are acceptable:
+  ** `2018-01-31T15:30:00Z` - ISO-8601 date time string. The trailing `Z` signals that the time is in UTC
+  ** `NOW+5MINUTES` - Solr's date math string
+  ** `2018-01-31T15:30:00` - No trailing 'Z' signals that the `timeZone` parameter must be specified to avoid ambiguity
 * `every` - (string, required) a positive Solr date math string which is added to the `startTime` or the last run time to arrive at the next scheduled time
 * `graceTime` - (string, optional) a positive Solr date math string. This is the additional grace time over the scheduled time within which the trigger is allowed to generate an event.
 * `timeZone` - (string, optional) a time zone string which is used for calculating the scheduled times


[5/8] lucene-solr:jira/solr-11670-2: SOLR-12047: Increase checkStateInZk timeout

Posted by ab...@apache.org.
SOLR-12047: Increase checkStateInZk timeout


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

Branch: refs/heads/jira/solr-11670-2
Commit: 4bdc99a465f830c5fe71abc2833327b9c27b221c
Parents: c1a4425
Author: Cao Manh Dat <da...@apache.org>
Authored: Tue Mar 6 08:51:38 2018 +0700
Committer: Cao Manh Dat <da...@apache.org>
Committed: Tue Mar 6 08:51:38 2018 +0700

----------------------------------------------------------------------
 solr/CHANGES.txt                                           | 2 ++
 solr/core/src/java/org/apache/solr/cloud/ZkController.java | 2 +-
 2 files changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4bdc99a4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f9ca9e6..f398c0b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -359,6 +359,8 @@ Other Changes
 * SOLR-11957: Increase MaxFileSize=32MB and MaxBackupIndex=10 for RollingFileAppender in log4j.properties
   (Varun Thacker, shalin)
 
+* SOLR-12047: Increase checkStateInZk timeout (Cao Manh Dat, Varun Thacker)
+
 ==================  7.2.1 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4bdc99a4/solr/core/src/java/org/apache/solr/cloud/ZkController.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/ZkController.java b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
index a159db5..9d9d2bb 100644
--- a/solr/core/src/java/org/apache/solr/cloud/ZkController.java
+++ b/solr/core/src/java/org/apache/solr/cloud/ZkController.java
@@ -1704,7 +1704,7 @@ public class ZkController {
       AtomicReference<String> errorMessage = new AtomicReference<>();
       AtomicReference<DocCollection> collectionState = new AtomicReference<>();
       try {
-        zkStateReader.waitForState(cd.getCollectionName(), 3, TimeUnit.SECONDS, (n, c) -> {
+        zkStateReader.waitForState(cd.getCollectionName(), 10, TimeUnit.SECONDS, (n, c) -> {
           collectionState.set(c);
           if (c == null)
             return false;