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 2019/02/01 11:49:33 UTC

[lucene-solr] branch jira/solr-11127 updated: Add a read-only mode for SolrCloud collections.

This is an automated email from the ASF dual-hosted git repository.

ab pushed a commit to branch jira/solr-11127
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/jira/solr-11127 by this push:
     new 800cd70  Add a read-only mode for SolrCloud collections.
800cd70 is described below

commit 800cd7080782e3eb0729923bec62c89e8c944b7c
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Fri Feb 1 12:49:11 2019 +0100

    Add a read-only mode for SolrCloud collections.
---
 .../api/collections/ReindexCollectionCmd.java      | 27 +++++++++----
 .../solr/handler/admin/CollectionsHandler.java     |  3 +-
 .../processor/DistributedUpdateProcessor.java      | 47 ++++++++++++++++++++--
 .../apache/solr/common/cloud/ZkStateReader.java    |  2 +
 4 files changed, 68 insertions(+), 11 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
index 6f0b091..0b9efd6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ReindexCollectionCmd.java
@@ -26,7 +26,10 @@ import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
 
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.cloud.Overseer;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.cloud.ClusterState;
@@ -38,6 +41,8 @@ import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
@@ -54,6 +59,7 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
   public static final String COL_PREFIX = ".reindex_";
   public static final String REINDEX_PROP = CollectionAdminRequest.PROPERTY_PREFIX + "reindex";
   public static final String REINDEX_PHASE_PROP = CollectionAdminRequest.PROPERTY_PREFIX + "reindex_phase";
+  public static final String READONLY_PROP = CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP;
 
   private final OverseerCollectionMessageHandler ocmh;
 
@@ -171,15 +177,22 @@ public class ReindexCollectionCmd implements OverseerCollectionMessageHandler.Cm
         return;
       }
 
-      // 1. copy existing docs
+      // 1. put the collection in read-only mode
+      cmd = new ZkNodeProps(Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toLower(),
+          ZkStateReader.COLLECTION_PROP, collection,
+          READONLY_PROP, "true");
+      ocmh.overseer.offerStateUpdate(Utils.toJSON(props));
 
+      // 2. copy the documents to tmp
+      ModifiableSolrParams q = new ModifiableSolrParams();
+      q.set(CommonParams.QT, "/stream");
+      q.set("expr", "daemon(id=\"" + tmpCollection + "\")");
+      SolrResponse rsp = ocmh.cloudManager.request(new QueryRequest(q));
+
+      // 3. delete source collection
+
+      // 4. copy from tmp to source collection
 
-      // ?. set up alias - new docs will go to the tmpCollection
-      cmd = new ZkNodeProps(
-          CommonParams.NAME, collection,
-          "collections", tmpCollection
-      );
-      ocmh.commandMap.get(CollectionParams.CollectionAction.CREATEALIAS).call(clusterState, cmd, cmdResults);
       // nocommit error checking
     } finally {
       if (aborted) {
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 405ad3d..1316fa0 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
@@ -937,6 +937,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     }),
     MODIFYCOLLECTION_OP(MODIFYCOLLECTION, (req, rsp, h) -> {
       Map<String, Object> m = copy(req.getParams(), null, CollectionAdminRequest.MODIFIABLE_COLLECTION_PROPERTIES);
+      copyPropertiesWithPrefix(req.getParams(), m, COLL_PROP_PREFIX);
       if (m.isEmpty())  {
         throw new SolrException(ErrorCode.BAD_REQUEST,
             formatString("no supported values provided {0}", CollectionAdminRequest.MODIFIABLE_COLLECTION_PROPERTIES.toString()));
@@ -944,7 +945,7 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       copy(req.getParams().required(), m, COLLECTION_PROP);
       addMapObject(m, RULE);
       addMapObject(m, SNITCH);
-      for (String prop : CollectionAdminRequest.MODIFIABLE_COLLECTION_PROPERTIES) {
+      for (String prop : m.keySet()) {
         if ("".equals(m.get(prop)))  {
           // set to an empty string is equivalent to removing the property, see SOLR-12507
           m.put(prop, null);
diff --git a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
index 0b125a1..94f3cb0 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/DistributedUpdateProcessor.java
@@ -43,6 +43,7 @@ import org.apache.lucene.util.CharsRefBuilder;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrRequest.METHOD;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.GenericSolrRequest;
 import org.apache.solr.client.solrj.request.UpdateRequest;
 import org.apache.solr.client.solrj.response.SimpleSolrResponse;
@@ -67,6 +68,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.cloud.ZooKeeperException;
+import org.apache.solr.common.params.CommonAdminParams;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.ShardParams;
@@ -84,6 +86,8 @@ import org.apache.solr.schema.SchemaField;
 import org.apache.solr.update.AddUpdateCommand;
 import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.MergeIndexesCommand;
+import org.apache.solr.update.RollbackUpdateCommand;
 import org.apache.solr.update.SolrCmdDistributor;
 import org.apache.solr.update.SolrCmdDistributor.Error;
 import org.apache.solr.update.SolrCmdDistributor.Node;
@@ -186,6 +190,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   private Set<String> skippedCoreNodeNames;
   private boolean isIndexChanged = false;
 
+  private boolean readOnly = false;
+
   /**
    * Number of times requests forwarded to some other shard's leader can be retried
    */
@@ -251,6 +257,13 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     if (cloudDesc != null) {
       collection = cloudDesc.getCollectionName();
       replicaType = cloudDesc.getReplicaType();
+      DocCollection coll = zkController.getClusterState().getCollectionOrNull(collection);
+      if (coll != null) {
+        // check readOnly property in coll state, unless overriden by params
+        if (!req.getParams().getBool(ZkStateReader.READ_ONLY_IGNORE, false)) {
+          readOnly = coll.getBool(CollectionAdminRequest.PROPERTY_PREFIX + ZkStateReader.READ_ONLY_PROP, false);
+        }
+      }
     } else {
       collection = null;
       replicaType = Replica.Type.NRT;
@@ -674,6 +687,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     assert TestInjection.injectFailUpdateRequests();
 
+    if (readOnly) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
 
     if (zkEnabled) {
@@ -1421,7 +1438,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   public void processDelete(DeleteUpdateCommand cmd) throws IOException {
     
     assert TestInjection.injectFailUpdateRequests();
-    
+
+    if (readOnly) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
 
     if (!cmd.isDeleteById()) {
@@ -1930,7 +1951,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   public void processCommit(CommitUpdateCommand cmd) throws IOException {
     
     assert TestInjection.injectFailUpdateRequests();
-    
+
+    if (readOnly) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
     List<Node> nodes = null;
     Replica leaderReplica = null;
@@ -2042,7 +2067,23 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       }
     }
   }
-  
+
+  @Override
+  public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+    if (readOnly) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+    super.processMergeIndexes(cmd);
+  }
+
+  @Override
+  public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+    if (readOnly) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+    super.processRollback(cmd);
+  }
+
   @Override
   public void finish() throws IOException {
     assert ! finished : "lifecycle sanity check";
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 1a26451..8b12fa7 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
@@ -102,6 +102,8 @@ public class ZkStateReader implements Closeable {
   public static final String MAX_AT_ONCE_PROP = "maxAtOnce";
   public static final String MAX_WAIT_SECONDS_PROP = "maxWaitSeconds";
   public static final String STATE_TIMESTAMP_PROP = "stateTimestamp";
+  public static final String READ_ONLY_PROP = "readOnly";
+  public static final String READ_ONLY_IGNORE = "readOnlyIgnore";
   public static final String COLLECTIONS_ZKNODE = "/collections";
   public static final String LIVE_NODES_ZKNODE = "/live_nodes";
   public static final String ALIASES = "/aliases.json";