mirror of https://github.com/apache/lucene.git
SOLR-13271: Read-only mode for SolrCloud collections.
This commit is contained in:
parent
34382407ef
commit
0bc26d07ad
|
@ -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
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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?
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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";
|
||||
|
||||
|
|
Loading…
Reference in New Issue