SOLR-15087: Allow restoration to existing collections (#2380)

The recent addition of support for a "readonly" mode for collections
opens the door to restoring to already-existing collections.

This commit adds a codepath to allow this.  Any compatible existing
collection may be used for restoration, including the collection that
was the original source of the backup.
This commit is contained in:
Jason Gerlowski 2021-02-16 21:59:24 -05:00 committed by GitHub
parent 58acfed0d2
commit 15bd858d34
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
6 changed files with 151 additions and 51 deletions

View File

@ -563,7 +563,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
}
private void modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
void modifyCollection(ClusterState clusterState, ZkNodeProps message, @SuppressWarnings({"rawtypes"})NamedList results)
throws Exception {
final String collectionName = message.getStr(ZkStateReader.COLLECTION_PROP);

View File

@ -17,25 +17,6 @@
package org.apache.solr.cloud.api.collections;
import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.apache.solr.cloud.DistributedClusterStateUpdater;
import org.apache.solr.cloud.Overseer;
import org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.ShardRequestTracker;
@ -51,6 +32,7 @@ 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.CollectionAdminParams;
import org.apache.solr.common.params.CollectionParams;
import org.apache.solr.common.params.CoreAdminParams;
import org.apache.solr.common.params.ModifiableSolrParams;
import org.apache.solr.common.util.NamedList;
@ -67,6 +49,26 @@ import org.apache.zookeeper.KeeperException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.net.URI;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import static org.apache.solr.common.cloud.ZkStateReader.*;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATE;
import static org.apache.solr.common.params.CollectionParams.CollectionAction.CREATESHARD;
@ -87,8 +89,8 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
try (RestoreContext restoreContext = new RestoreContext(message, ocmh)) {
if (state.hasCollection(restoreContext.restoreCollectionName)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Restoration collection [" + restoreContext.restoreCollectionName +
"] must be created by the backup process and cannot exist");
RestoreOnExistingCollection restoreOnExistingCollection = new RestoreOnExistingCollection(restoreContext);
restoreOnExistingCollection.process(restoreContext, results);
} else {
RestoreOnANewCollection restoreOnANewCollection = new RestoreOnANewCollection(message, restoreContext.backupCollectionState);
restoreOnANewCollection.validate(restoreContext.backupCollectionState, restoreContext.nodeList.size());
@ -194,8 +196,7 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
/**
* Restoration 'strategy' that takes responsibility for creating the collection to restore to.
*
* This is currently the only supported 'strategy' for backup restoration. Though in-place restoration has been
* proposed and may be added soon (see SOLR-15087)
* @see RestoreOnExistingCollection
*/
private class RestoreOnANewCollection {
private int numNrtReplicas;
@ -551,4 +552,54 @@ public class RestoreCmd implements OverseerCollectionMessageHandler.Cmd {
}
}
}
/**
* Restoration 'strategy' that ensures the collection being restored to already exists.
*
* @see RestoreOnANewCollection
*/
private class RestoreOnExistingCollection {
private RestoreOnExistingCollection(RestoreContext rc) {
int numShardsOfBackup = rc.backupCollectionState.getSlices().size();
int numShards = rc.zkStateReader.getClusterState().getCollection(rc.restoreCollectionName).getSlices().size();
if (numShardsOfBackup != numShards) {
String msg = String.format(Locale.ROOT, "Unable to restoring since number of shards in backup " +
"and specified collection does not match, numShardsOfBackup:%d numShardsOfCollection:%d", numShardsOfBackup, numShards);
throw new SolrException(ErrorCode.BAD_REQUEST, msg);
}
}
public void process(RestoreContext rc, @SuppressWarnings({"rawtypes"}) NamedList results) throws Exception {
ClusterState clusterState = rc.zkStateReader.getClusterState();
DocCollection restoreCollection = clusterState.getCollection(rc.restoreCollectionName);
enableReadOnly(clusterState, restoreCollection);
try {
requestReplicasToRestore(results, restoreCollection, clusterState, rc.backupProperties,
rc.backupPath, rc.repo, rc.shardHandler, rc.asyncId);
} finally {
disableReadOnly(clusterState, restoreCollection);
}
}
private void disableReadOnly(ClusterState clusterState, DocCollection restoreCollection) throws Exception {
ZkNodeProps params = new ZkNodeProps(
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toString(),
ZkStateReader.COLLECTION_PROP, restoreCollection.getName(),
ZkStateReader.READ_ONLY, null
);
ocmh.modifyCollection(clusterState, params, new NamedList<>());
}
private void enableReadOnly(ClusterState clusterState, DocCollection restoreCollection) throws Exception {
ZkNodeProps params = new ZkNodeProps(
Overseer.QUEUE_OPERATION, CollectionParams.CollectionAction.MODIFYCOLLECTION.toString(),
ZkStateReader.COLLECTION_PROP, restoreCollection.getName(),
ZkStateReader.READ_ONLY, "true"
);
ocmh.modifyCollection(clusterState, params, new NamedList<>());
}
}
}

View File

@ -1084,11 +1084,6 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
req.getParams().required().check(NAME, COLLECTION_PROP);
final String collectionName = SolrIdentifierValidator.validateCollectionName(req.getParams().get(COLLECTION_PROP));
final ClusterState clusterState = h.coreContainer.getZkController().getClusterState();
//We always want to restore into an collection name which doesn't exist yet.
if (clusterState.hasCollection(collectionName)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' exists, no action taken.");
}
if (h.coreContainer.getZkController().getZkStateReader().getAliases().hasAlias(collectionName)) {
throw new SolrException(ErrorCode.BAD_REQUEST, "Collection '" + collectionName + "' is an existing alias, no action taken.");
}

View File

@ -65,9 +65,9 @@ class RestoreCoreOp implements CoreAdminHandler.CoreAdminOp {
// this core must be the only replica in its shard otherwise
// we cannot guarantee consistency between replicas because when we add data (or restore index) to this replica
Slice slice = zkController.getClusterState().getCollection(cd.getCollectionName()).getSlice(cd.getShardId());
if (slice.getReplicas().size() != 1) {
if (slice.getReplicas().size() != 1 && !core.readOnly) {
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
"Failed to restore core=" + core.getName() + ", the core must be the only replica in its shard");
"Failed to restore core=" + core.getName() + ", the core must be the only replica in its shard or it must be read only");
}
RestoreCore restoreCore;

View File

@ -1360,12 +1360,14 @@ POST http://localhost:8983/v2/collections/backups
[[restore]]
== RESTORE: Restore Collection
Restores Solr indexes and associated configurations.
Restores Solr indexes and associated configurations to a specified collection.
`/admin/collections?action=RESTORE&name=myBackupName&location=/path/to/my/shared/drive&collection=myRestoredCollectionName`
The RESTORE operation will create a collection with the specified name in the collection parameter. You cannot restore into the same collection the backup was taken from. Also the target collection should not be present at the time the API is called as Solr will create it for you.
The RESTORE operation will replace the content of a collection with files from the specified backup.
If the provided `collection` value matches an existing collection, Solr will use it for restoration, assuming it is compatible (same number of shards, etc.) with the stored backup files.
If the provided `collection` value doesn't exist, a new collection with that name is created in a way compatible with the stored backup files.
The collection created will be have the same number of shards and replicas as the original collection, preserving routing information, etc. Optionally, you can override some parameters documented below.
While restoring, if a configset with the same name exists in ZooKeeper then Solr will reuse that, or else it will upload the backed up configset in ZooKeeper and use that.

View File

@ -23,12 +23,12 @@ import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.util.TestUtil;
import org.apache.solr.client.solrj.SolrQuery;
import org.apache.solr.client.solrj.SolrServerException;
import org.apache.solr.client.solrj.embedded.JettySolrRunner;
import org.apache.solr.client.solrj.impl.CloudSolrClient;
import org.apache.solr.client.solrj.request.CollectionAdminRequest;
import org.apache.solr.client.solrj.request.QueryRequest;
import org.apache.solr.client.solrj.request.UpdateRequest;
import org.apache.solr.client.solrj.response.CollectionAdminResponse;
import org.apache.solr.client.solrj.response.RequestStatusState;
@ -48,6 +48,7 @@ import org.apache.solr.core.backup.Checksum;
import org.apache.solr.core.backup.ShardBackupId;
import org.apache.solr.core.backup.ShardBackupMetadata;
import org.apache.solr.core.backup.repository.BackupRepository;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Test;
import org.slf4j.Logger;
@ -87,13 +88,11 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
private static long docsSeed; // see indexDocs()
protected static final int NUM_SHARDS = 2;//granted we sometimes shard split to get more
protected static final int REPL_FACTOR = 2;
protected static final String BACKUPNAME_PREFIX = "mytestbackup";
protected static final String BACKUP_REPO_NAME = "trackingBackupRepository";
protected String testSuffix = "test1";
protected int replFactor;
protected int numTlogReplicas;
protected int numPullReplicas;
@BeforeClass
public static void createCluster() throws Exception {
@ -101,6 +100,11 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
System.setProperty("solr.directoryFactory", "solr.StandardDirectoryFactory");
}
@Before
public void setUpTrackingRepo() {
TrackingBackupRepository.clear();
}
/**
* @return The name of the collection to use.
*/
@ -114,12 +118,6 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
this.testSuffix = testSuffix;
}
private void randomizeReplicaTypes() {
replFactor = TestUtil.nextInt(random(), 1, 2);
// numTlogReplicas = TestUtil.nextInt(random(), 0, 1);
// numPullReplicas = TestUtil.nextInt(random(), 0, 1);
}
/**
* @return The absolute path for the backup location.
* Could return null.
@ -128,11 +126,10 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
@Test
public void testSimple() throws Exception {
setTestSuffix("testbackupincsimple");
final String backupCollectionName = getCollectionName();
final String restoreCollectionName = backupCollectionName + "_restore";
TrackingBackupRepository.clear();
setTestSuffix("testbackupincsimple");
CloudSolrClient solrClient = cluster.getSolrClient();
CollectionAdminRequest
@ -166,9 +163,12 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
log.info("Created backup with {} docs, took {}ms", numFound, timeTaken);
t = System.nanoTime();
randomlyPrecreateRestoreCollection(restoreCollectionName, "conf1", NUM_SHARDS, 1);
CollectionAdminRequest.restoreCollection(restoreCollectionName, backupName)
.setBackupId(0)
.setLocation(backupLocation).setRepositoryName(BACKUP_REPO_NAME).processAndWait(solrClient, 500);
.setLocation(backupLocation)
.setRepositoryName(BACKUP_REPO_NAME)
.processAndWait(solrClient, 500);
timeTaken = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - t);
log.info("Restored from backup, took {}ms", timeTaken);
numFound = cluster.getSolrClient().query(restoreCollectionName,
@ -177,18 +177,55 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
}
}
@Test
public void testRestoreToOriginalCollection() throws Exception {
setTestSuffix("testbackuprestoretooriginal");
final String backupCollectionName = getCollectionName();
final String backupName = BACKUPNAME_PREFIX + testSuffix;
// Bootstrap the backup collection with seed docs
CollectionAdminRequest
.createCollection(backupCollectionName, "conf1", NUM_SHARDS, REPL_FACTOR)
.process(cluster.getSolrClient());
final int firstBatchNumDocs = indexDocs(backupCollectionName, true);
// Backup and immediately add more docs to the collection
try (BackupRepository repository = cluster.getJettySolrRunner(0).getCoreContainer()
.newBackupRepository(BACKUP_REPO_NAME)) {
final String backupLocation = repository.getBackupLocation(getBackupLocation());
final RequestStatusState result = CollectionAdminRequest.backupCollection(backupCollectionName, backupName)
.setLocation(backupLocation)
.setRepositoryName(BACKUP_REPO_NAME)
.processAndWait(cluster.getSolrClient(), 10 * 1000);
assertEquals(RequestStatusState.COMPLETED, result);
}
final int secondBatchNumDocs = indexDocs(backupCollectionName, true);
final int maxDocs = secondBatchNumDocs + firstBatchNumDocs;
assertEquals(maxDocs, getNumDocsInCollection(backupCollectionName));
// Restore original docs and validate that doc count is correct
try (BackupRepository repository = cluster.getJettySolrRunner(0).getCoreContainer()
.newBackupRepository(BACKUP_REPO_NAME)) {
final String backupLocation = repository.getBackupLocation(getBackupLocation());
final RequestStatusState result = CollectionAdminRequest.restoreCollection(backupCollectionName, backupName)
.setLocation(backupLocation)
.setRepositoryName(BACKUP_REPO_NAME)
.processAndWait(cluster.getSolrClient(), 20 * 1000);
assertEquals(RequestStatusState.COMPLETED, result);
}
assertEquals(firstBatchNumDocs, getNumDocsInCollection(backupCollectionName));
}
@Test
@Slow
@SuppressWarnings("rawtypes")
public void testBackupIncremental() throws Exception {
TrackingBackupRepository.clear();
setTestSuffix("testbackupinc");
randomizeReplicaTypes();
CloudSolrClient solrClient = cluster.getSolrClient();
CollectionAdminRequest
.createCollection(getCollectionName(), "conf1", NUM_SHARDS, replFactor, numTlogReplicas, numPullReplicas)
.createCollection(getCollectionName(), "conf1", NUM_SHARDS, REPL_FACTOR)
.process(solrClient);
indexDocs(getCollectionName(), false);
@ -341,8 +378,11 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
String restoreCollectionName = getCollectionName() + "_restored";
randomlyPrecreateRestoreCollection(restoreCollectionName, "conf1", NUM_SHARDS, REPL_FACTOR);
CollectionAdminRequest.restoreCollection(restoreCollectionName, backupName)
.setLocation(backupLocation).setRepositoryName(BACKUP_REPO_NAME).process(solrClient);
.setLocation(backupLocation)
.setRepositoryName(BACKUP_REPO_NAME)
.process(solrClient);
AbstractDistribZkTestBase.waitForRecoveriesToFinish(
restoreCollectionName, cluster.getSolrClient().getZkStateReader(), log.isDebugEnabled(), true, 30);
@ -380,6 +420,18 @@ public abstract class AbstractIncrementalBackupTest extends SolrCloudTestCase {
return numDocs;
}
private void randomlyPrecreateRestoreCollection(String restoreCollectionName, String configName, int numShards, int numReplicas) throws Exception {
if (random().nextBoolean()) {
CollectionAdminRequest.createCollection(restoreCollectionName, configName, numShards, numReplicas)
.process(cluster.getSolrClient());
cluster.waitForActiveCollection(restoreCollectionName, numShards, numShards*numReplicas);
}
}
private long getNumDocsInCollection(String collectionName) throws Exception {
return new QueryRequest(new SolrQuery("*:*")).process(cluster.getSolrClient(), collectionName).getResults().getNumFound();
}
private class IncrementalBackupVerifier {
private BackupRepository repository;
private URI backupURI;