SOLR-13271: Read-only mode for SolrCloud collections.

This commit is contained in:
Andrzej Bialecki 2019-03-05 14:22:31 +01:00
parent 34382407ef
commit 0bc26d07ad
13 changed files with 280 additions and 9 deletions

View File

@ -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
----------------------

View File

@ -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 {

View File

@ -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;

View File

@ -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?

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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) {
@ -1417,6 +1434,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
assert TestInjection.injectFailUpdateRequests();
if (isReadOnly()) {
throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
}
updateCommand = cmd;
if (!cmd.isDeleteById()) {
@ -1926,6 +1947,10 @@ public class DistributedUpdateProcessor extends UpdateRequestProcessor {
assert TestInjection.injectFailUpdateRequests();
if (isReadOnly()) {
throw new SolrException(ErrorCode.FORBIDDEN, "Collection " + collection + " is read-only.");
}
updateCommand = cmd;
List<Node> nodes = null;
Replica leaderReplica = null;
@ -2036,6 +2061,22 @@ 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";

View File

@ -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 {

View File

@ -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

View File

@ -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);

View File

@ -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);

View File

@ -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";