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/03/05 16:57:10 UTC

[lucene-solr] branch branch_8x updated: SOLR-13271: Read-only mode for SolrCloud collections.

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 5f605da  SOLR-13271: Read-only mode for SolrCloud collections.
5f605da is described below

commit 5f605daf9221ea082ca41c73a9dc216f8851e03f
Author: Andrzej Bialecki <ab...@apache.org>
AuthorDate: Tue Mar 5 14:22:31 2019 +0100

    SOLR-13271: Read-only mode for SolrCloud collections.
---
 solr/CHANGES.txt                                   |   2 +
 .../OverseerCollectionMessageHandler.java          |   5 +
 .../solr/cloud/overseer/CollectionMutator.java     |  11 ++
 .../java/org/apache/solr/core/CoreContainer.java   |  33 ++++-
 .../src/java/org/apache/solr/core/SolrCore.java    |   1 +
 .../solr/handler/admin/CollectionsHandler.java     |   3 +-
 .../apache/solr/update/DefaultSolrCoreState.java   |   2 +-
 .../processor/DistributedUpdateProcessor.java      |  47 ++++++-
 .../apache/solr/cloud/CollectionsAPISolrJTest.java | 145 +++++++++++++++++++++
 solr/solr-ref-guide/src/collections-api.adoc       |  24 ++++
 .../solrj/request/CollectionAdminRequest.java      |   6 +-
 .../apache/solr/common/cloud/DocCollection.java    |   9 ++
 .../apache/solr/common/cloud/ZkStateReader.java    |   1 +
 13 files changed, 280 insertions(+), 9 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 5b4e47f..030a03d 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -48,6 +48,8 @@ New Features
 
 * SOLR-13287: Allow zplot to visualize probability distributions in Apache Zeppelin (Joel Bernstein)
 
+* SOLR-13271: Read-only mode for SolrCloud collections (ab, shalin)
+
 Bug Fixes
 ----------------------
 
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index db117a3..c3e53c6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -702,6 +702,11 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
 
     if (!areChangesVisible)
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Could not modify collection " + message);
+
+    // if switching to/from read-only mode reload the collection
+    if (message.keySet().contains(ZkStateReader.READ_ONLY)) {
+      reloadCollection(null, new ZkNodeProps(NAME, collectionName), results);
+    }
   }
 
   void cleanupCollection(String collectionName, NamedList results) throws Exception {
diff --git a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
index 88e18e2..bb549f4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
+++ b/solr/core/src/java/org/apache/solr/cloud/overseer/CollectionMutator.java
@@ -119,6 +119,17 @@ public class CollectionMutator {
         }
       }
     }
+    // other aux properties are also modifiable
+    for (String prop : message.keySet()) {
+      if (prop.startsWith(CollectionAdminRequest.PROPERTY_PREFIX)) {
+        hasAnyOps = true;
+        if (message.get(prop) == null) {
+          m.remove(prop);
+        } else {
+          m.put(prop, message.get(prop));
+        }
+      }
+    }
 
     if (!hasAnyOps) {
       return ZkStateWriter.NO_OP;
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 3dc1bb5..8c5e227 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -57,6 +57,7 @@ import org.apache.http.auth.AuthSchemeProvider;
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.config.Lookup;
 import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.cloud.SolrCloudManager;
@@ -78,6 +79,7 @@ import org.apache.solr.common.SolrException.ErrorCode;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Replica.State;
+import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.CollectionAdminParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.common.util.IOUtils;
@@ -121,6 +123,7 @@ import org.apache.solr.update.SolrCoreState;
 import org.apache.solr.update.UpdateShardHandler;
 import org.apache.solr.util.DefaultSolrThreadFactory;
 import org.apache.solr.util.OrderedExecutor;
+import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.stats.MetricUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
@@ -1449,10 +1452,36 @@ public class CoreContainer {
         log.info("Reloading SolrCore '{}' using configuration from {}", cd.getName(), coreConfig.getName());
         newCore = core.reload(coreConfig);
 
+        DocCollection docCollection = null;
+        if (getZkController() != null) {
+          docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+          // turn off indexing now, before the new core is registered
+          if (docCollection.getBool(ZkStateReader.READ_ONLY, false)) {
+            newCore.readOnly = true;
+          }
+        }
+
         registerCore(cd, newCore, false, false);
 
-        if (getZkController() != null) {
-          DocCollection docCollection = getZkController().getClusterState().getCollection(cd.getCollectionName());
+        // force commit on old core if the new one is readOnly and prevent any new updates
+        if (newCore.readOnly) {
+          RefCounted<IndexWriter> iwRef = core.getSolrCoreState().getIndexWriter(null);
+          if (iwRef != null) {
+            IndexWriter iw = iwRef.get();
+            // switch old core to readOnly
+            core.readOnly = true;
+            try {
+              if (iw != null) {
+                iw.commit();
+              }
+            } finally {
+              iwRef.decref();
+            }
+          }
+        }
+
+
+        if (docCollection != null) {
           Replica replica = docCollection.getReplica(cd.getCloudDescriptor().getCoreNodeName());
           assert replica != null;
           if (replica.getType() == Replica.Type.TLOG) { // TODO: needed here?
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index eb9d40d..93bd690 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -237,6 +237,7 @@ public final class SolrCore implements SolrInfoBean, SolrMetricProducer, Closeab
 
   public boolean searchEnabled = true;
   public boolean indexEnabled = true;
+  public volatile boolean readOnly = false;
 
   public Set<String> getMetricNames() {
     return metricNames;
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 725d2bd..3e026af 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
@@ -935,6 +935,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()));
@@ -942,7 +943,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/DefaultSolrCoreState.java b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
index 8b87631..87a885f 100644
--- a/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
+++ b/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
@@ -116,7 +116,7 @@ public final class DefaultSolrCoreState extends SolrCoreState implements Recover
   @Override
   public RefCounted<IndexWriter> getIndexWriter(SolrCore core)
       throws IOException {
-    if (core != null && !core.indexEnabled) {
+    if (core != null && (!core.indexEnabled || core.readOnly)) {
       throw new SolrException(SolrException.ErrorCode.FORBIDDEN, "Indexing is temporarily disabled");
     }
     boolean succeeded = false;
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 2a76ab9..b01fd8a 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
@@ -80,6 +80,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;
@@ -182,6 +184,8 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   private Set<String> skippedCoreNodeNames;
   private boolean isIndexChanged = false;
 
+  private boolean readOnlyCollection = false;
+
   /**
    * Number of times requests forwarded to some other shard's leader can be retried
    */
@@ -247,6 +251,11 @@ 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
+        readOnlyCollection = coll.isReadOnly();
+      }
     } else {
       collection = null;
       replicaType = Replica.Type.NRT;
@@ -267,6 +276,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
     cloneRequiredOnLeader = shouldClone;
   }
 
+  private boolean isReadOnly() {
+    return readOnlyCollection || req.getCore().readOnly;
+  }
+
   private List<Node> setupRequest(String id, SolrInputDocument doc) {
     return setupRequest(id, doc, null);
   }
@@ -669,6 +682,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
 
     assert TestInjection.injectFailUpdateRequests();
 
+    if (isReadOnly()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
 
     if (zkEnabled) {
@@ -1416,7 +1433,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   public void processDelete(DeleteUpdateCommand cmd) throws IOException {
     
     assert TestInjection.injectFailUpdateRequests();
-    
+
+    if (isReadOnly()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
 
     if (!cmd.isDeleteById()) {
@@ -1925,7 +1946,11 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
   public void processCommit(CommitUpdateCommand cmd) throws IOException {
     
     assert TestInjection.injectFailUpdateRequests();
-    
+
+    if (isReadOnly()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+
     updateCommand = cmd;
     List<Node> nodes = null;
     Replica leaderReplica = null;
@@ -2035,7 +2060,23 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
       }
     }
   }
-  
+
+  @Override
+  public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+    if (isReadOnly()) {
+      throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
+    }
+    super.processMergeIndexes(cmd);
+  }
+
+  @Override
+  public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+    if (isReadOnly()) {
+      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/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
index b685aa8..2f61c6c 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CollectionsAPISolrJTest.java
@@ -25,19 +25,23 @@ import static org.apache.solr.common.params.CollectionAdminParams.COLLECTION;
 import static org.apache.solr.common.params.CollectionAdminParams.DEFAULTS;
 
 import java.io.IOException;
+import java.lang.invoke.MethodHandles;
 import java.nio.file.Path;
 import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.CoreAdminRequest;
@@ -45,15 +49,19 @@ import org.apache.solr.client.solrj.request.CoreStatus;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.CoreAdminResponse;
+import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.client.solrj.response.V2Response;
+import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.cloud.ClusterProperties;
 import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.Slice;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.RetryUtil;
 import org.apache.solr.common.util.TimeSource;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.util.TimeOut;
@@ -63,9 +71,12 @@ import org.junit.Before;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 @LuceneTestCase.Slow
 public class CollectionsAPISolrJTest extends SolrCloudTestCase {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
 
   @Before
   public void beforeTest() throws Exception {
@@ -590,6 +601,140 @@ public class CollectionsAPISolrJTest extends SolrCloudTestCase {
     fail("Timed out waiting for cluster property value");
   }
 
+  private static final int NUM_DOCS = 10;
+
+  @Test
+  public void testReadOnlyCollection() throws Exception {
+    final String collectionName = "readOnlyTest";
+    CloudSolrClient solrClient = cluster.getSolrClient();
+
+    CollectionAdminRequest.createCollection(collectionName, "conf", 2, 2)
+        .process(solrClient);
+
+    solrClient.setDefaultCollection(collectionName);
+
+    cluster.waitForActiveCollection(collectionName, 2, 4);
+
+    // verify that indexing works
+    List<SolrInputDocument> docs = new ArrayList<>();
+    for (int i = 0; i < NUM_DOCS; i++) {
+      docs.add(new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
+    }
+    solrClient.add(docs);
+    solrClient.commit();
+    // verify the docs exist
+    QueryResponse rsp = solrClient.query(params(CommonParams.Q, "*:*"));
+    assertEquals("initial num docs", NUM_DOCS, rsp.getResults().getNumFound());
+
+    // index more but don't commit
+    docs.clear();
+    for (int i = NUM_DOCS; i < NUM_DOCS * 2; i++) {
+      docs.add(new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
+    }
+    solrClient.add(docs);
+
+    Replica leader
+        = solrClient.getZkStateReader().getLeaderRetry(collectionName, "shard1", DEFAULT_TIMEOUT);
+
+    final AtomicReference<Long> coreStartTime = new AtomicReference<>(getCoreStatus(leader).getCoreStartTime().getTime());
+
+    // Check for value change
+    CollectionAdminRequest.modifyCollection(collectionName,
+        Collections.singletonMap(ZkStateReader.READ_ONLY, "true"))
+        .process(solrClient);
+
+    DocCollection coll = solrClient.getZkStateReader().getClusterState().getCollection(collectionName);
+    assertNotNull(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY));
+    assertEquals(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY).toString(), "true");
+
+    // wait for the expected collection reload
+    RetryUtil.retryUntil("Timed out waiting for core to reload", 30, 1000, TimeUnit.MILLISECONDS, () -> {
+      long restartTime = 0;
+      try {
+        restartTime = getCoreStatus(leader).getCoreStartTime().getTime();
+      } catch (Exception e) {
+        log.warn("Exception getting core start time: {}", e.getMessage());
+        return false;
+      }
+      return restartTime > coreStartTime.get();
+    });
+
+    coreStartTime.set(getCoreStatus(leader).getCoreStartTime().getTime());
+
+    // check for docs - reloading should have committed the new docs
+    // this also verifies that searching works in read-only mode
+    rsp = solrClient.query(params(CommonParams.Q, "*:*"));
+    assertEquals("num docs after turning on read-only", NUM_DOCS * 2, rsp.getResults().getNumFound());
+
+    // try sending updates
+    try {
+      solrClient.add(new SolrInputDocument("id", "shouldFail"));
+      fail("add() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+    try {
+      solrClient.deleteById("shouldFail");
+      fail("deleteById() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+    try {
+      solrClient.deleteByQuery("id:shouldFail");
+      fail("deleteByQuery() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+    try {
+      solrClient.commit();
+      fail("commit() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+    try {
+      solrClient.optimize();
+      fail("optimize() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+    try {
+      solrClient.rollback();
+      fail("rollback() should fail in read-only mode");
+    } catch (Exception e) {
+      // expected - ignore
+    }
+
+    // Check for removing value
+    // setting to empty string is equivalent to removing the property, see SOLR-12507
+    CollectionAdminRequest.modifyCollection(collectionName,
+        Collections.singletonMap(ZkStateReader.READ_ONLY, ""))
+        .process(cluster.getSolrClient());
+    coll = cluster.getSolrClient().getZkStateReader().getClusterState().getCollection(collectionName);
+    assertNull(coll.toString(), coll.getProperties().get(ZkStateReader.READ_ONLY));
+
+    // wait for the expected collection reload
+    RetryUtil.retryUntil("Timed out waiting for core to reload", 30, 1000, TimeUnit.MILLISECONDS, () -> {
+      long restartTime = 0;
+      try {
+        restartTime = getCoreStatus(leader).getCoreStartTime().getTime();
+      } catch (Exception e) {
+        log.warn("Exception getting core start time: {}", e.getMessage());
+        return false;
+      }
+      return restartTime > coreStartTime.get();
+    });
+
+    // check that updates are working now
+    docs.clear();
+    for (int i = NUM_DOCS * 2; i < NUM_DOCS * 3; i++) {
+      docs.add(new SolrInputDocument("id", String.valueOf(i), "string_s", String.valueOf(i)));
+    }
+    solrClient.add(docs);
+    solrClient.commit();
+    rsp = solrClient.query(params(CommonParams.Q, "*:*"));
+    assertEquals("num docs after turning off read-only", NUM_DOCS * 3, rsp.getResults().getNumFound());
+  }
+
 
   @Test
   public void testOverseerStatus() throws IOException, SolrServerException {
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index e44c6ca..36c639b 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -120,6 +120,8 @@ If `true`, the request will complete only when all affected replicas become acti
 The name of the collection with which all replicas of this collection must be co-located. The collection must already exist and must have a single shard named `shard1`.
 See <<colocating-collections.adoc#colocating-collections, Colocating collections>> for more details.
 
+Collections are first created in read-write mode but can be put in `readOnly`
+mode using the <<modifycollection, MODIFYCOLLECTION>> action.
 
 === CREATE Response
 
@@ -191,9 +193,31 @@ The attributes that can be modified are:
 * snitch
 * policy
 * withCollection
+* readOnly
+* other custom properties that use a `property.` prefix
 
 See the <<create,CREATE action>> section above for details on these attributes.
 
+==== Read-only mode
+Setting the `readOnly` attribute to `true` puts the collection in read-only mode,
+in which any index update requests are rejected. Other collection-level actions (eg. adding /
+removing / moving replicas) are still available in this mode.
+
+The transition from the (default) read-write to read-only mode consists of the following steps:
+
+* the `readOnly` flag is changed in collection state,
+* any new update requests are rejected with 403 FORBIDDEN error code (ongoing
+  long-running requests are aborted, too),
+* a forced commit is performed to flush and commit any in-flight updates.
+
+NOTE: This may potentially take a long time if there are still major segment merges running
+ in the background.
+
+* a collection <<reload, RELOAD action>> is executed.
+
+Removing the `readOnly` property or setting it to false enables the
+processing of updates and reloads the collection.
+
 [[reload]]
 == RELOAD: Reload a Collection
 
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 6f7af617..a5dad2a 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
@@ -59,6 +59,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 import static org.apache.solr.common.params.CollectionAdminParams.COLL_CONF;
@@ -87,11 +88,12 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
       POLICY,
       COLL_CONF,
       WITH_COLLECTION,
-      COLOCATED_WITH);
+      COLOCATED_WITH,
+      READ_ONLY);
 
   protected final CollectionAction action;
 
-  private static String PROPERTY_PREFIX = "property.";
+  public static String PROPERTY_PREFIX = "property.";
 
   public CollectionAdminRequest(CollectionAction action) {
     this("/admin/collections", action);
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
index e896272..5e3ff4a 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/DocCollection.java
@@ -39,6 +39,7 @@ import static org.apache.solr.common.cloud.ZkStateReader.AUTO_ADD_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.MAX_SHARDS_PER_NODE;
 import static org.apache.solr.common.cloud.ZkStateReader.NRT_REPLICAS;
 import static org.apache.solr.common.cloud.ZkStateReader.PULL_REPLICAS;
+import static org.apache.solr.common.cloud.ZkStateReader.READ_ONLY;
 import static org.apache.solr.common.cloud.ZkStateReader.REPLICATION_FACTOR;
 import static org.apache.solr.common.cloud.ZkStateReader.TLOG_REPLICAS;
 
@@ -71,6 +72,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
   private final Integer maxShardsPerNode;
   private final Boolean autoAddReplicas;
   private final String policy;
+  private final Boolean readOnly;
 
   public DocCollection(String name, Map<String, Slice> slices, Map<String, Object> props, DocRouter router) {
     this(name, slices, props, router, Integer.MAX_VALUE, ZkStateReader.CLUSTER_STATE);
@@ -99,6 +101,8 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     Boolean autoAddReplicas = (Boolean) verifyProp(props, AUTO_ADD_REPLICAS);
     this.policy = (String) props.get(Policy.POLICY);
     this.autoAddReplicas = autoAddReplicas == null ? Boolean.FALSE : autoAddReplicas;
+    Boolean readOnly = (Boolean) verifyProp(props, READ_ONLY);
+    this.readOnly = readOnly == null ? Boolean.FALSE : readOnly;
     
     verifyProp(props, RULE);
     verifyProp(props, SNITCH);
@@ -152,6 +156,7 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
       case TLOG_REPLICAS:
         return Integer.parseInt(o.toString());
       case AUTO_ADD_REPLICAS:
+      case READ_ONLY:
         return Boolean.parseBoolean(o.toString());
       case "snitch":
       case "rule":
@@ -274,6 +279,10 @@ public class DocCollection extends ZkNodeProps implements Iterable<Slice> {
     return router;
   }
 
+  public boolean isReadOnly() {
+    return readOnly;
+  }
+
   @Override
   public String toString() {
     return "DocCollection("+name+"/" + znode + "/" + znodeVersion + ")=" + JSONUtil.toJSON(this);
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 7a3f7d2..37d6515 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
@@ -124,6 +124,7 @@ public class ZkStateReader implements Closeable {
   public static final String PULL_REPLICAS = "pullReplicas";
   public static final String NRT_REPLICAS = "nrtReplicas";
   public static final String TLOG_REPLICAS = "tlogReplicas";
+  public static final String READ_ONLY = "readOnly";
 
   public static final String ROLES = "/roles.json";