Restore from Individual Shard Snapshot Files in Parallel (#48110) (#48686)

Make restoring shard snapshots run in parallel on the `SNAPSHOT` thread-pool.
This commit is contained in:
Armin Braun 2019-10-30 14:36:30 +01:00 committed by GitHub
parent dbc05cd808
commit 52e5ceb321
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 373 additions and 278 deletions

View File

@ -42,6 +42,7 @@ import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionRunnable;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
@ -1850,12 +1851,16 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
return storeRecovery.recoverFromStore(this); return storeRecovery.recoverFromStore(this);
} }
public boolean restoreFromRepository(Repository repository) { public void restoreFromRepository(Repository repository, ActionListener<Boolean> listener) {
try {
assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard"; assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard";
assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " + assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " +
recoveryState.getRecoverySource(); recoveryState.getRecoverySource();
StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); StoreRecovery storeRecovery = new StoreRecovery(shardId, logger);
return storeRecovery.recoverFromRepository(this, repository); storeRecovery.recoverFromRepository(this, repository, listener);
} catch (Exception e) {
listener.onFailure(e);
}
} }
/** /**
@ -2540,17 +2545,15 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
case SNAPSHOT: case SNAPSHOT:
markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread markAsRecovering("from snapshot", recoveryState); // mark the shard as recovering on the cluster state thread
SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState.getRecoverySource(); SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState.getRecoverySource();
threadPool.generic().execute(() -> { threadPool.generic().execute(
try { ActionRunnable.<Boolean>wrap(ActionListener.wrap(r -> {
final Repository repository = repositoriesService.repository(recoverySource.snapshot().getRepository()); if (r) {
if (restoreFromRepository(repository)) {
recoveryListener.onRecoveryDone(recoveryState); recoveryListener.onRecoveryDone(recoveryState);
} }
} catch (Exception e) { },
recoveryListener.onRecoveryFailure(recoveryState, e -> recoveryListener.onRecoveryFailure(recoveryState, new RecoveryFailedException(recoveryState, null, e), true)),
new RecoveryFailedException(recoveryState, null, e), true); restoreListener -> restoreFromRepository(
} repositoriesService.repository(recoverySource.snapshot().getRepository()), restoreListener)));
});
break; break;
case LOCAL_SHARDS: case LOCAL_SHARDS:
final IndexMetaData indexMetaData = indexSettings().getIndexMetaData(); final IndexMetaData indexMetaData = indexSettings().getIndexMetaData();

View File

@ -32,6 +32,8 @@ import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexInput;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RecoverySource;
@ -90,10 +92,16 @@ final class StoreRecovery {
RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType();
assert recoveryType == RecoverySource.Type.EMPTY_STORE || recoveryType == RecoverySource.Type.EXISTING_STORE : assert recoveryType == RecoverySource.Type.EMPTY_STORE || recoveryType == RecoverySource.Type.EXISTING_STORE :
"expected store recovery type but was: " + recoveryType; "expected store recovery type but was: " + recoveryType;
return executeRecovery(indexShard, () -> { final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
final ActionListener<Boolean> recoveryListener = recoveryListener(indexShard, future);
try {
logger.debug("starting recovery from store ..."); logger.debug("starting recovery from store ...");
internalRecoverFromStore(indexShard); internalRecoverFromStore(indexShard);
}); recoveryListener.onResponse(true);
} catch (Exception e) {
recoveryListener.onFailure(e);
}
return future.actionGet();
} }
return false; return false;
} }
@ -121,7 +129,8 @@ final class StoreRecovery {
final boolean isSplit = sourceMetaData.getNumberOfShards() < indexShard.indexSettings().getNumberOfShards(); final boolean isSplit = sourceMetaData.getNumberOfShards() < indexShard.indexSettings().getNumberOfShards();
assert isSplit == false || sourceMetaData.getCreationVersion().onOrAfter(Version.V_6_0_0_alpha1) : "for split we require a " + assert isSplit == false || sourceMetaData.getCreationVersion().onOrAfter(Version.V_6_0_0_alpha1) : "for split we require a " +
"single type but the index is created before 6.0.0"; "single type but the index is created before 6.0.0";
return executeRecovery(indexShard, () -> { final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
ActionListener.completeWith(recoveryListener(indexShard, future), () -> {
logger.debug("starting recovery from local shards {}", shards); logger.debug("starting recovery from local shards {}", shards);
try { try {
final Directory directory = indexShard.store().directory(); // don't close this directory!! final Directory directory = indexShard.store().directory(); // don't close this directory!!
@ -136,11 +145,13 @@ final class StoreRecovery {
// copied segments - we will also see them in stats etc. // copied segments - we will also see them in stats etc.
indexShard.getEngine().forceMerge(false, -1, false, indexShard.getEngine().forceMerge(false, -1, false,
false, false); false, false);
return true;
} catch (IOException ex) { } catch (IOException ex) {
throw new IndexShardRecoveryException(indexShard.shardId(), "failed to recover from local shards", ex); throw new IndexShardRecoveryException(indexShard.shardId(), "failed to recover from local shards", ex);
} }
}); });
assert future.isDone();
return future.actionGet();
} }
return false; return false;
} }
@ -272,21 +283,22 @@ final class StoreRecovery {
* previously created index snapshot into an existing initializing shard. * previously created index snapshot into an existing initializing shard.
* @param indexShard the index shard instance to recovery the snapshot from * @param indexShard the index shard instance to recovery the snapshot from
* @param repository the repository holding the physical files the shard should be recovered from * @param repository the repository holding the physical files the shard should be recovered from
* @return <code>true</code> if the shard has been recovered successfully, <code>false</code> if the recovery * @param listener resolves to <code>true</code> if the shard has been recovered successfully, <code>false</code> if the recovery
* has been ignored due to a concurrent modification of if the clusters state has changed due to async updates. * has been ignored due to a concurrent modification of if the clusters state has changed due to async updates.
*/ */
boolean recoverFromRepository(final IndexShard indexShard, Repository repository) { void recoverFromRepository(final IndexShard indexShard, Repository repository, ActionListener<Boolean> listener) {
try {
if (canRecover(indexShard)) { if (canRecover(indexShard)) {
RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType();
assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType; assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType;
SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource(); SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource();
return executeRecovery(indexShard, () -> { restore(indexShard, repository, recoverySource, recoveryListener(indexShard, listener));
logger.debug("restoring from {} ...", indexShard.recoveryState().getRecoverySource()); } else {
restore(indexShard, repository, recoverySource); listener.onResponse(false);
}); }
} catch (Exception e) {
listener.onFailure(e);
} }
return false;
} }
private boolean canRecover(IndexShard indexShard) { private boolean canRecover(IndexShard indexShard) {
@ -300,12 +312,9 @@ final class StoreRecovery {
return true; return true;
} }
/** private ActionListener<Boolean> recoveryListener(IndexShard indexShard, ActionListener<Boolean> listener) {
* Recovers the state of the shard from the store. return ActionListener.wrap(res -> {
*/ if (res) {
private boolean executeRecovery(final IndexShard indexShard, Runnable recoveryRunnable) throws IndexShardRecoveryException {
try {
recoveryRunnable.run();
// Check that the gateway didn't leave the shard in init or recovering stage. it is up to the gateway // Check that the gateway didn't leave the shard in init or recovering stage. it is up to the gateway
// to call post recovery. // to call post recovery.
final IndexShardState shardState = indexShard.state(); final IndexShardState shardState = indexShard.state();
@ -333,26 +342,32 @@ final class StoreRecovery {
} else if (logger.isDebugEnabled()) { } else if (logger.isDebugEnabled()) {
logger.debug("recovery completed from [shard_store], took [{}]", timeValueMillis(recoveryState.getTimer().time())); logger.debug("recovery completed from [shard_store], took [{}]", timeValueMillis(recoveryState.getTimer().time()));
} }
return true; }
} catch (IndexShardRecoveryException e) { listener.onResponse(res);
}, ex -> {
if (ex instanceof IndexShardRecoveryException) {
if (indexShard.state() == IndexShardState.CLOSED) { if (indexShard.state() == IndexShardState.CLOSED) {
// got closed on us, just ignore this recovery // got closed on us, just ignore this recovery
return false; listener.onResponse(false);
return;
} }
if ((e.getCause() instanceof IndexShardClosedException) || (e.getCause() instanceof IndexShardNotStartedException)) { if ((ex.getCause() instanceof IndexShardClosedException) || (ex.getCause() instanceof IndexShardNotStartedException)) {
// got closed on us, just ignore this recovery // got closed on us, just ignore this recovery
return false; listener.onResponse(false);
return;
} }
throw e; listener.onFailure(ex);
} catch (IndexShardClosedException | IndexShardNotStartedException e) { } else if (ex instanceof IndexShardClosedException || ex instanceof IndexShardNotStartedException) {
} catch (Exception e) { listener.onResponse(false);
} else {
if (indexShard.state() == IndexShardState.CLOSED) { if (indexShard.state() == IndexShardState.CLOSED) {
// got closed on us, just ignore this recovery // got closed on us, just ignore this recovery
return false; listener.onResponse(false);
} else {
listener.onFailure(new IndexShardRecoveryException(shardId, "failed recovery", ex));
} }
throw new IndexShardRecoveryException(shardId, "failed recovery", e);
} }
return false; });
} }
/** /**
@ -447,14 +462,30 @@ final class StoreRecovery {
/** /**
* Restores shard from {@link SnapshotRecoverySource} associated with this shard in routing table * Restores shard from {@link SnapshotRecoverySource} associated with this shard in routing table
*/ */
private void restore(final IndexShard indexShard, final Repository repository, final SnapshotRecoverySource restoreSource) { private void restore(IndexShard indexShard, Repository repository, SnapshotRecoverySource restoreSource,
ActionListener<Boolean> listener) {
logger.debug("restoring from {} ...", indexShard.recoveryState().getRecoverySource());
final RecoveryState.Translog translogState = indexShard.recoveryState().getTranslog(); final RecoveryState.Translog translogState = indexShard.recoveryState().getTranslog();
if (restoreSource == null) { if (restoreSource == null) {
throw new IndexShardRestoreFailedException(shardId, "empty restore source"); listener.onFailure(new IndexShardRestoreFailedException(shardId, "empty restore source"));
return;
} }
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("[{}] restoring shard [{}]", restoreSource.snapshot(), shardId); logger.trace("[{}] restoring shard [{}]", restoreSource.snapshot(), shardId);
} }
final ActionListener<Void> restoreListener = ActionListener.wrap(
v -> {
final Store store = indexShard.store();
bootstrap(indexShard, store);
assert indexShard.shardRouting.primary() : "only primary shards can recover from store";
writeEmptyRetentionLeasesFile(indexShard);
indexShard.openEngineAndRecoverFromTranslog();
indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm());
indexShard.finalizeRecovery();
indexShard.postRecovery("restore done");
listener.onResponse(true);
}, e -> listener.onFailure(new IndexShardRestoreFailedException(shardId, "restore failed", e))
);
try { try {
translogState.totalOperations(0); translogState.totalOperations(0);
translogState.totalOperationsOnStart(0); translogState.totalOperationsOnStart(0);
@ -467,17 +498,9 @@ final class StoreRecovery {
final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName); final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName);
assert indexShard.getEngineOrNull() == null; assert indexShard.getEngineOrNull() == null;
repository.restoreShard(indexShard.store(), restoreSource.snapshot().getSnapshotId(), indexId, snapshotShardId, repository.restoreShard(indexShard.store(), restoreSource.snapshot().getSnapshotId(), indexId, snapshotShardId,
indexShard.recoveryState()); indexShard.recoveryState(), restoreListener);
final Store store = indexShard.store();
bootstrap(indexShard, store);
assert indexShard.shardRouting.primary() : "only primary shards can recover from store";
writeEmptyRetentionLeasesFile(indexShard);
indexShard.openEngineAndRecoverFromTranslog();
indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm());
indexShard.finalizeRecovery();
indexShard.postRecovery("restore done");
} catch (Exception e) { } catch (Exception e) {
throw new IndexShardRestoreFailedException(shardId, "restore failed", e); restoreListener.onFailure(e);
} }
} }

View File

@ -128,8 +128,9 @@ public class FilterRepository implements Repository {
in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, writeShardGens, listener); in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, writeShardGens, listener);
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState,
in.restoreShard(store, snapshotId, indexId, snapshotShardId, recoveryState); ActionListener<Void> listener) {
in.restoreShard(store, snapshotId, indexId, snapshotShardId, recoveryState, listener);
} }
@Override @Override

View File

@ -224,9 +224,10 @@ public interface Repository extends LifecycleComponent {
* @param indexId id of the index in the repository from which the restore is occurring * @param indexId id of the index in the repository from which the restore is occurring
* @param snapshotShardId shard id (in the snapshot) * @param snapshotShardId shard id (in the snapshot)
* @param recoveryState recovery state * @param recoveryState recovery state
* @param listener listener to invoke once done
*/ */
void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState); void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState,
ActionListener<Void> listener);
/** /**
* Retrieve shard snapshot status for the stored snapshot * Retrieve shard snapshot status for the stored snapshot
* *

View File

@ -1214,11 +1214,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
// Start as many workers as fit into the snapshot pool at once at the most // Start as many workers as fit into the snapshot pool at once at the most
final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), indexIncrementalFileCount); final int workers = Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), indexIncrementalFileCount);
final ActionListener<Void> filesListener = ActionListener.delegateResponse( final ActionListener<Void> filesListener = fileQueueListener(filesToSnapshot, workers, allFilesUploadedListener);
new GroupedActionListener<>(allFilesUploadedListener, workers), (l, e) -> {
filesToSnapshot.clear(); // Stop uploading the remaining files if we run into any exception
l.onFailure(e);
});
for (int i = 0; i < workers; ++i) { for (int i = 0; i < workers; ++i) {
executor.execute(ActionRunnable.run(filesListener, () -> { executor.execute(ActionRunnable.run(filesListener, () -> {
BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = filesToSnapshot.poll(0L, TimeUnit.MILLISECONDS); BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = filesToSnapshot.poll(0L, TimeUnit.MILLISECONDS);
@ -1242,20 +1238,43 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState, ActionListener<Void> listener) {
ShardId shardId = store.shardId(); final ShardId shardId = store.shardId();
try { final ActionListener<Void> restoreListener = ActionListener.delegateResponse(listener,
(l, e) -> l.onFailure(new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e)));
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
final BlobContainer container = shardContainer(indexId, snapshotShardId); final BlobContainer container = shardContainer(indexId, snapshotShardId);
BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(container, snapshotId); executor.execute(ActionRunnable.wrap(restoreListener, l -> {
SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); final BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(container, snapshotId);
final SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles());
new FileRestoreContext(metadata.name(), shardId, snapshotId, recoveryState) { new FileRestoreContext(metadata.name(), shardId, snapshotId, recoveryState) {
@Override @Override
protected void restoreFiles(List<BlobStoreIndexShardSnapshot.FileInfo> filesToRecover, Store store) throws IOException { protected void restoreFiles(List<BlobStoreIndexShardSnapshot.FileInfo> filesToRecover, Store store,
ActionListener<Void> listener) {
if (filesToRecover.isEmpty()) {
listener.onResponse(null);
} else {
// Start as many workers as fit into the snapshot pool at once at the most
final int workers =
Math.min(threadPool.info(ThreadPool.Names.SNAPSHOT).getMax(), snapshotFiles.indexFiles().size());
final BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> files = new LinkedBlockingQueue<>(filesToRecover);
final ActionListener<Void> allFilesListener =
fileQueueListener(files, workers, ActionListener.map(listener, v -> null));
// restore the files from the snapshot to the Lucene store // restore the files from the snapshot to the Lucene store
for (final BlobStoreIndexShardSnapshot.FileInfo fileToRecover : filesToRecover) { for (int i = 0; i < workers; ++i) {
logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name()); executor.execute(ActionRunnable.run(allFilesListener, () -> {
store.incRef();
try {
BlobStoreIndexShardSnapshot.FileInfo fileToRecover;
while ((fileToRecover = files.poll(0L, TimeUnit.MILLISECONDS)) != null) {
restoreFile(fileToRecover, store); restoreFile(fileToRecover, store);
} }
} finally {
store.decRef();
}
}));
}
}
} }
private void restoreFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store store) throws IOException { private void restoreFile(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store store) throws IOException {
@ -1294,10 +1313,16 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
} }
} }
} }
}.restore(snapshotFiles, store); }.restore(snapshotFiles, store, l);
} catch (Exception e) { }));
throw new IndexShardRestoreFailedException(shardId, "failed to restore snapshot [" + snapshotId + "]", e);
} }
private static ActionListener<Void> fileQueueListener(BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> files, int workers,
ActionListener<Collection<Void>> listener) {
return ActionListener.delegateResponse(new GroupedActionListener<>(listener, workers), (l, e) -> {
files.clear(); // Stop uploading the remaining files if we run into any exception
l.onFailure(e);
});
} }
private static InputStream maybeRateLimit(InputStream stream, @Nullable RateLimiter rateLimiter, CounterMetric metric) { private static InputStream maybeRateLimit(InputStream stream, @Nullable RateLimiter rateLimiter, CounterMetric metric) {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.repositories.blobstore;
import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.message.ParameterizedMessage;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.util.iterable.Iterables;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
@ -74,7 +75,7 @@ public abstract class FileRestoreContext {
/** /**
* Performs restore operation * Performs restore operation
*/ */
public void restore(SnapshotFiles snapshotFiles, Store store) { public void restore(SnapshotFiles snapshotFiles, Store store, ActionListener<Void> listener) {
store.incRef(); store.incRef();
try { try {
logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, repositoryName, shardId); logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, repositoryName, shardId);
@ -150,11 +151,27 @@ public abstract class FileRestoreContext {
} }
} }
restoreFiles(filesToRecover, store); restoreFiles(filesToRecover, store, ActionListener.wrap(
v -> {
store.incRef();
try {
afterRestore(snapshotFiles, store, restoredSegmentsFile);
listener.onResponse(null);
} finally {
store.decRef();
}
}, listener::onFailure));
} catch (IOException ex) { } catch (IOException ex) {
throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex);
} }
} catch (Exception e) {
listener.onFailure(e);
} finally {
store.decRef();
}
}
private void afterRestore(SnapshotFiles snapshotFiles, Store store, StoreFileMetaData restoredSegmentsFile) {
// read the snapshot data persisted // read the snapshot data persisted
try { try {
Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory());
@ -178,9 +195,6 @@ public abstract class FileRestoreContext {
} catch (IOException e) { } catch (IOException e) {
logger.warn("[{}] [{}] failed to list directory - some of files might not be deleted", shardId, snapshotId); logger.warn("[{}] [{}] failed to list directory - some of files might not be deleted", shardId, snapshotId);
} }
} finally {
store.decRef();
}
} }
/** /**
@ -189,7 +203,8 @@ public abstract class FileRestoreContext {
* @param filesToRecover List of files to restore * @param filesToRecover List of files to restore
* @param store Store to restore into * @param store Store to restore into
*/ */
protected abstract void restoreFiles(List<BlobStoreIndexShardSnapshot.FileInfo> filesToRecover, Store store) throws IOException; protected abstract void restoreFiles(List<BlobStoreIndexShardSnapshot.FileInfo> filesToRecover, Store store,
ActionListener<Void> listener);
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
private static Iterable<StoreFileMetaData> concat(Store.RecoveryDiff diff) { private static Iterable<StoreFileMetaData> concat(Store.RecoveryDiff diff) {

View File

@ -107,7 +107,7 @@ import static org.elasticsearch.snapshots.SnapshotUtils.filterIndices;
* {@link RoutingTable.Builder#addAsRestore(IndexMetaData, SnapshotRecoverySource)} method. * {@link RoutingTable.Builder#addAsRestore(IndexMetaData, SnapshotRecoverySource)} method.
* <p> * <p>
* Individual shards are getting restored as part of normal recovery process in * Individual shards are getting restored as part of normal recovery process in
* {@link IndexShard#restoreFromRepository(Repository)} )} * {@link IndexShard#restoreFromRepository} )}
* method, which detects that shard should be restored from snapshot rather than recovered from gateway by looking * method, which detects that shard should be restored from snapshot rather than recovered from gateway by looking
* at the {@link ShardRouting#recoverySource()} property. * at the {@link ShardRouting#recoverySource()} property.
* <p> * <p>

View File

@ -2348,11 +2348,12 @@ public class IndexShardTests extends IndexShardTestCase {
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); target.markAsRecovering("store", new RecoveryState(routing, localNode, null));
assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
target.restoreFromRepository(new RestoreOnlyRepository("test") {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState, ActionListener<Void> listener) {
try { ActionListener.completeWith(listener, () -> {
cleanLuceneIndex(targetStore.directory()); cleanLuceneIndex(targetStore.directory());
for (String file : sourceStore.directory().listAll()) { for (String file : sourceStore.directory().listAll()) {
if (file.equals("write.lock") || file.startsWith("extra")) { if (file.equals("write.lock") || file.startsWith("extra")) {
@ -2360,11 +2361,11 @@ public class IndexShardTests extends IndexShardTestCase {
} }
targetStore.directory().copyFrom(sourceStore.directory(), file, file, IOContext.DEFAULT); targetStore.directory().copyFrom(sourceStore.directory(), file, file, IOContext.DEFAULT);
} }
} catch (Exception ex) { return null;
throw new RuntimeException(ex); });
} }
} }, future);
})); assertTrue(future.actionGet());
assertThat(target.getLocalCheckpoint(), equalTo(2L)); assertThat(target.getLocalCheckpoint(), equalTo(2L));
assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L)); assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L));
assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L)); assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L));

View File

@ -210,7 +210,7 @@ public class RepositoriesServiceTests extends ESTestCase {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState, ActionListener<Void> listener) {
} }

View File

@ -117,8 +117,9 @@ public class FsRepositoryTests extends ESTestCase {
new UnassignedInfo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED, "")); new UnassignedInfo(UnassignedInfo.Reason.EXISTING_INDEX_RESTORED, ""));
routing = ShardRoutingHelper.initialize(routing, localNode.getId(), 0); routing = ShardRoutingHelper.initialize(routing, localNode.getId(), 0);
RecoveryState state = new RecoveryState(routing, localNode, null); RecoveryState state = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> final PlainActionFuture<Void> futureA = PlainActionFuture.newFuture();
repository.restoreShard(store, snapshotId, indexId, shardId, state)); runGeneric(threadPool, () -> repository.restoreShard(store, snapshotId, indexId, shardId, state, futureA));
futureA.actionGet();
assertTrue(state.getIndex().recoveredBytes() > 0); assertTrue(state.getIndex().recoveredBytes() > 0);
assertEquals(0, state.getIndex().reusedFileCount()); assertEquals(0, state.getIndex().reusedFileCount());
assertEquals(indexCommit.getFileNames().size(), state.getIndex().recoveredFileCount()); assertEquals(indexCommit.getFileNames().size(), state.getIndex().recoveredFileCount());
@ -139,14 +140,16 @@ public class FsRepositoryTests extends ESTestCase {
// roll back to the first snap and then incrementally restore // roll back to the first snap and then incrementally restore
RecoveryState firstState = new RecoveryState(routing, localNode, null); RecoveryState firstState = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> final PlainActionFuture<Void> futureB = PlainActionFuture.newFuture();
repository.restoreShard(store, snapshotId, indexId, shardId, firstState)); runGeneric(threadPool, () -> repository.restoreShard(store, snapshotId, indexId, shardId, firstState, futureB));
futureB.actionGet();
assertEquals("should reuse everything except of .liv and .si", assertEquals("should reuse everything except of .liv and .si",
commitFileNames.size()-2, firstState.getIndex().reusedFileCount()); commitFileNames.size()-2, firstState.getIndex().reusedFileCount());
RecoveryState secondState = new RecoveryState(routing, localNode, null); RecoveryState secondState = new RecoveryState(routing, localNode, null);
runGeneric(threadPool, () -> final PlainActionFuture<Void> futureC = PlainActionFuture.newFuture();
repository.restoreShard(store, incSnapshotId, indexId, shardId, secondState)); runGeneric(threadPool, () -> repository.restoreShard(store, incSnapshotId, indexId, shardId, secondState, futureC));
futureC.actionGet();
assertEquals(secondState.getIndex().reusedFileCount(), commitFileNames.size()-2); assertEquals(secondState.getIndex().reusedFileCount(), commitFileNames.size()-2);
assertEquals(secondState.getIndex().recoveredFileCount(), 2); assertEquals(secondState.getIndex().recoveredFileCount(), 2);
List<RecoveryState.File> recoveredFiles = List<RecoveryState.File> recoveredFiles =

View File

@ -805,11 +805,14 @@ public abstract class IndexShardTestCase extends ESTestCase {
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, version, index); new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, version, index);
final ShardRouting shardRouting = newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, recoverySource); final ShardRouting shardRouting = newShardRouting(shardId, node.getId(), true, ShardRoutingState.INITIALIZING, recoverySource);
shard.markAsRecovering("from snapshot", new RecoveryState(shardRouting, node, null)); shard.markAsRecovering("from snapshot", new RecoveryState(shardRouting, node, null));
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
repository.restoreShard(shard.store(), repository.restoreShard(shard.store(),
snapshot.getSnapshotId(), snapshot.getSnapshotId(),
indexId, indexId,
shard.shardId(), shard.shardId(),
shard.recoveryState()); shard.recoveryState(),
future);
future.actionGet();
} }
/** /**

View File

@ -299,7 +299,10 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
} }
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState,
ActionListener<Void> listener) {
// TODO: Instead of blocking in the restore logic and synchronously completing the listener we should just make below logic async
ActionListener.completeWith(listener, () -> {
// TODO: Add timeouts to network calls / the restore process. // TODO: Add timeouts to network calls / the restore process.
createEmptyStore(store); createEmptyStore(store);
ShardId shardId = store.shardId(); ShardId shardId = store.shardId();
@ -358,6 +361,8 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
retentionLeaseId); retentionLeaseId);
renewable.cancel(); renewable.cancel();
} }
return null;
});
} }
private void createEmptyStore(Store store) { private void createEmptyStore(Store store) {
@ -473,11 +478,14 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
fileInfos.add(new FileInfo(fileMetaData.name(), fileMetaData, fileSize)); fileInfos.add(new FileInfo(fileMetaData.name(), fileMetaData, fileSize));
} }
SnapshotFiles snapshotFiles = new SnapshotFiles(LATEST, fileInfos); SnapshotFiles snapshotFiles = new SnapshotFiles(LATEST, fileInfos);
restore(snapshotFiles, store); final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
restore(snapshotFiles, store, future);
future.actionGet();
} }
@Override @Override
protected void restoreFiles(List<FileInfo> filesToRecover, Store store) { protected void restoreFiles(List<FileInfo> filesToRecover, Store store, ActionListener<Void> listener) {
ActionListener.completeWith(listener, () -> {
logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover); logger.trace("[{}] starting CCR restore of {} files", shardId, filesToRecover);
final PlainActionFuture<Void> restoreFilesFuture = new PlainActionFuture<>(); final PlainActionFuture<Void> restoreFilesFuture = new PlainActionFuture<>();
final List<StoreFileMetaData> mds = filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList()); final List<StoreFileMetaData> mds = filesToRecover.stream().map(FileInfo::metadata).collect(Collectors.toList());
@ -553,6 +561,8 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
multiFileTransfer.start(); multiFileTransfer.start();
restoreFilesFuture.actionGet(); restoreFilesFuture.actionGet();
logger.trace("[{}] completed CCR restore", shardId); logger.trace("[{}] completed CCR restore", shardId);
return null;
});
} }
@Override @Override

View File

@ -451,11 +451,12 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
ShardRouting routing = ShardRoutingHelper.newWithRestoreSource(primary.routingEntry(), ShardRouting routing = ShardRoutingHelper.newWithRestoreSource(primary.routingEntry(),
new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test")); new RecoverySource.SnapshotRecoverySource(UUIDs.randomBase64UUID(), snapshot, Version.CURRENT, "test"));
primary.markAsRecovering("remote recovery from leader", new RecoveryState(routing, localNode, null)); primary.markAsRecovering("remote recovery from leader", new RecoveryState(routing, localNode, null));
final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) { primary.restoreFromRepository(new RestoreOnlyRepository(index.getName()) {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState, ActionListener<Void> listener) {
try { ActionListener.completeWith(listener, () -> {
IndexShard leader = leaderGroup.getPrimary(); IndexShard leader = leaderGroup.getPrimary();
Lucene.cleanLuceneIndex(primary.store().directory()); Lucene.cleanLuceneIndex(primary.store().directory());
try (Engine.IndexCommitRef sourceCommit = leader.acquireSafeIndexCommit()) { try (Engine.IndexCommitRef sourceCommit = leader.acquireSafeIndexCommit()) {
@ -465,12 +466,16 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
leader.store().directory(), md.name(), md.name(), IOContext.DEFAULT); leader.store().directory(), md.name(), md.name(), IOContext.DEFAULT);
} }
} }
return null;
});
}
}, future);
try {
future.actionGet();
} catch (Exception ex) { } catch (Exception ex) {
throw new AssertionError(ex); throw new AssertionError(ex);
} }
} }
});
}
}; };
} }

View File

@ -9,6 +9,7 @@ import org.apache.lucene.store.IOContext;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RecoverySource;
@ -125,11 +126,12 @@ public class FollowEngineIndexShardTests extends IndexShardTestCase {
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT);
target.markAsRecovering("store", new RecoveryState(routing, localNode, null)); target.markAsRecovering("store", new RecoveryState(routing, localNode, null));
assertTrue(target.restoreFromRepository(new RestoreOnlyRepository("test") { final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
target.restoreFromRepository(new RestoreOnlyRepository("test") {
@Override @Override
public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, public void restoreShard(Store store, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId,
RecoveryState recoveryState) { RecoveryState recoveryState, ActionListener<Void> listener) {
try { ActionListener.completeWith(listener, () -> {
cleanLuceneIndex(targetStore.directory()); cleanLuceneIndex(targetStore.directory());
for (String file : sourceStore.directory().listAll()) { for (String file : sourceStore.directory().listAll()) {
if (file.equals("write.lock") || file.startsWith("extra")) { if (file.equals("write.lock") || file.startsWith("extra")) {
@ -137,11 +139,11 @@ public class FollowEngineIndexShardTests extends IndexShardTestCase {
} }
targetStore.directory().copyFrom(sourceStore.directory(), file, file, IOContext.DEFAULT); targetStore.directory().copyFrom(sourceStore.directory(), file, file, IOContext.DEFAULT);
} }
} catch (Exception ex) { return null;
throw new RuntimeException(ex); });
} }
} }, future);
})); assertTrue(future.actionGet());
assertThat(target.getLocalCheckpoint(), equalTo(0L)); assertThat(target.getLocalCheckpoint(), equalTo(0L));
assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L)); assertThat(target.seqNoStats().getMaxSeqNo(), equalTo(2L));
assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L)); assertThat(target.seqNoStats().getGlobalCheckpoint(), equalTo(0L));

View File

@ -233,8 +233,11 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY); restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY);
DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT); DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT);
restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null)); restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null));
runAsSnapshot(shard.getThreadPool(), () -> runAsSnapshot(shard.getThreadPool(), () -> {
assertTrue(restoredShard.restoreFromRepository(repository))); final PlainActionFuture<Boolean> future = PlainActionFuture.newFuture();
restoredShard.restoreFromRepository(repository, future);
assertTrue(future.actionGet());
});
assertEquals(restoredShard.recoveryState().getStage(), RecoveryState.Stage.DONE); assertEquals(restoredShard.recoveryState().getStage(), RecoveryState.Stage.DONE);
assertEquals(restoredShard.recoveryState().getTranslog().recoveredOperations(), 0); assertEquals(restoredShard.recoveryState().getTranslog().recoveredOperations(), 0);
assertEquals(IndexShardState.POST_RECOVERY, restoredShard.state()); assertEquals(IndexShardState.POST_RECOVERY, restoredShard.state());