diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt index 060cea5d350..433c5a70850 100644 --- a/solr/CHANGES.txt +++ b/solr/CHANGES.txt @@ -70,6 +70,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 db117a3a0cb..c3e53c6a851 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 88e18e20267..bb549f43363 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 3dc1bb547ce..8c5e227b189 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 eb9d40db0ba..93bd69001f6 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 725d2bd2bbf..3e026af7962 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 8b87631cfab..87a885f4364 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 2a76ab90635..b01fd8a9c46 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 b685aa8cdb7..2f61c6cde81 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 e44c6ca4d0d..36c639bdfed 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 6f7af617e03..a5dad2ac00c 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 e8962726e65..5e3ff4a803c 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 7a3f7d2546e..37d65155c3a 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";