* More Efficient Ordering of Shard Upload Execution (#42791) * Change the upload order of of snapshots to work file by file in parallel on the snapshot pool instead of merely shard-by-shard * Inspired by #39657 * Cleanup BlobStoreRepository Abort and Failure Handling (#46208)
This commit is contained in:
parent
80bb08fbda
commit
41633cb9b5
|
@ -22,6 +22,7 @@ package org.elasticsearch.action;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.common.CheckedConsumer;
|
import org.elasticsearch.common.CheckedConsumer;
|
||||||
import org.elasticsearch.common.CheckedFunction;
|
import org.elasticsearch.common.CheckedFunction;
|
||||||
|
import org.elasticsearch.common.CheckedRunnable;
|
||||||
import org.elasticsearch.common.CheckedSupplier;
|
import org.elasticsearch.common.CheckedSupplier;
|
||||||
|
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -226,6 +227,37 @@ public interface ActionListener<Response> {
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Wraps a given listener and returns a new listener which executes the provided {@code runBefore}
|
||||||
|
* callback before the listener is notified via either {@code #onResponse} or {@code #onFailure}.
|
||||||
|
* If the callback throws an exception then it will be passed to the listener's {@code #onFailure} and its {@code #onResponse} will
|
||||||
|
* not be executed.
|
||||||
|
*/
|
||||||
|
static <Response> ActionListener<Response> runBefore(ActionListener<Response> delegate, CheckedRunnable<?> runBefore) {
|
||||||
|
return new ActionListener<Response>() {
|
||||||
|
@Override
|
||||||
|
public void onResponse(Response response) {
|
||||||
|
try {
|
||||||
|
runBefore.run();
|
||||||
|
} catch (Exception ex) {
|
||||||
|
delegate.onFailure(ex);
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
delegate.onResponse(response);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception e) {
|
||||||
|
try {
|
||||||
|
runBefore.run();
|
||||||
|
} catch (Exception ex) {
|
||||||
|
e.addSuppressed(ex);
|
||||||
|
}
|
||||||
|
delegate.onFailure(e);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wraps a given listener and returns a new listener which makes sure {@link #onResponse(Object)}
|
* Wraps a given listener and returns a new listener which makes sure {@link #onResponse(Object)}
|
||||||
* and {@link #onFailure(Exception)} of the provided listener will be called at most once.
|
* and {@link #onFailure(Exception)} of the provided listener will be called at most once.
|
||||||
|
|
|
@ -121,13 +121,11 @@ public class FilterRepository implements Repository {
|
||||||
return in.isReadOnly();
|
return in.isReadOnly();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
||||||
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus);
|
in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void restoreShard(Store store, SnapshotId snapshotId,
|
public void restoreShard(Store store, SnapshotId snapshotId,
|
||||||
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
|
Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) {
|
||||||
|
|
|
@ -51,7 +51,7 @@ import java.util.function.Function;
|
||||||
* <ul>
|
* <ul>
|
||||||
* <li>Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
|
* <li>Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
|
||||||
* with list of indices that will be included into the snapshot</li>
|
* with list of indices that will be included into the snapshot</li>
|
||||||
* <li>Data nodes call {@link Repository#snapshotShard(Store, MapperService, SnapshotId, IndexId, IndexCommit, IndexShardSnapshotStatus)}
|
* <li>Data nodes call {@link Repository#snapshotShard}
|
||||||
* for each shard</li>
|
* for each shard</li>
|
||||||
* <li>When all shard calls return master calls {@link #finalizeSnapshot} with possible list of failures</li>
|
* <li>When all shard calls return master calls {@link #finalizeSnapshot} with possible list of failures</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
|
@ -191,27 +191,6 @@ public interface Repository extends LifecycleComponent {
|
||||||
*/
|
*/
|
||||||
boolean isReadOnly();
|
boolean isReadOnly();
|
||||||
|
|
||||||
/**
|
|
||||||
* Creates a snapshot of the shard based on the index commit point.
|
|
||||||
* <p>
|
|
||||||
* The index commit point can be obtained by using {@link org.elasticsearch.index.engine.Engine#acquireLastIndexCommit} method.
|
|
||||||
* Repository implementations shouldn't release the snapshot index commit point. It is done by the method caller.
|
|
||||||
* <p>
|
|
||||||
* As snapshot process progresses, implementation of this method should update {@link IndexShardSnapshotStatus} object and check
|
|
||||||
* {@link IndexShardSnapshotStatus#isAborted()} to see if the snapshot process should be aborted.
|
|
||||||
* @param indexShard the shard to be snapshotted
|
|
||||||
* @param snapshotId snapshot id
|
|
||||||
* @param indexId id for the index being snapshotted
|
|
||||||
* @param snapshotIndexCommit commit point
|
|
||||||
* @param snapshotStatus snapshot status
|
|
||||||
* @deprecated use {@link #snapshotShard(Store, MapperService, SnapshotId, IndexId, IndexCommit, IndexShardSnapshotStatus)} instead
|
|
||||||
*/
|
|
||||||
@Deprecated
|
|
||||||
default void snapshotShard(IndexShard indexShard, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit,
|
|
||||||
IndexShardSnapshotStatus snapshotStatus) {
|
|
||||||
snapshotShard(indexShard.store(), indexShard.mapperService(), snapshotId, indexId, snapshotIndexCommit, snapshotStatus);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a snapshot of the shard based on the index commit point.
|
* Creates a snapshot of the shard based on the index commit point.
|
||||||
* <p>
|
* <p>
|
||||||
|
@ -226,9 +205,10 @@ public interface Repository extends LifecycleComponent {
|
||||||
* @param indexId id for the index being snapshotted
|
* @param indexId id for the index being snapshotted
|
||||||
* @param snapshotIndexCommit commit point
|
* @param snapshotIndexCommit commit point
|
||||||
* @param snapshotStatus snapshot status
|
* @param snapshotStatus snapshot status
|
||||||
|
* @param listener listener invoked on completion
|
||||||
*/
|
*/
|
||||||
void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit,
|
void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit,
|
||||||
IndexShardSnapshotStatus snapshotStatus);
|
IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Restores snapshot of the shard.
|
* Restores snapshot of the shard.
|
||||||
|
|
|
@ -32,6 +32,7 @@ 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.ActionRunnable;
|
||||||
|
import org.elasticsearch.action.StepListener;
|
||||||
import org.elasticsearch.action.support.GroupedActionListener;
|
import org.elasticsearch.action.support.GroupedActionListener;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
@ -108,6 +109,8 @@ import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Executor;
|
||||||
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName;
|
import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName;
|
||||||
|
@ -909,9 +912,15 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
||||||
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
final ShardId shardId = store.shardId();
|
final ShardId shardId = store.shardId();
|
||||||
final long startTime = threadPool.absoluteTimeInMillis();
|
final long startTime = threadPool.absoluteTimeInMillis();
|
||||||
|
final StepListener<Void> snapshotDoneListener = new StepListener<>();
|
||||||
|
snapshotDoneListener.whenComplete(listener::onResponse, e -> {
|
||||||
|
snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e));
|
||||||
|
listener.onFailure(e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e
|
||||||
|
: new IndexShardSnapshotFailedException(store.shardId(), e));
|
||||||
|
});
|
||||||
try {
|
try {
|
||||||
logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name());
|
logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name());
|
||||||
|
|
||||||
|
@ -933,20 +942,23 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
}
|
}
|
||||||
|
|
||||||
final List<BlobStoreIndexShardSnapshot.FileInfo> indexCommitPointFiles = new ArrayList<>();
|
final List<BlobStoreIndexShardSnapshot.FileInfo> indexCommitPointFiles = new ArrayList<>();
|
||||||
store.incRef();
|
|
||||||
try {
|
|
||||||
ArrayList<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot = new ArrayList<>();
|
ArrayList<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot = new ArrayList<>();
|
||||||
final Store.MetadataSnapshot metadata;
|
store.incRef();
|
||||||
// TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should
|
|
||||||
final Collection<String> fileNames;
|
final Collection<String> fileNames;
|
||||||
|
final Store.MetadataSnapshot metadataFromStore;
|
||||||
|
try {
|
||||||
|
// TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should
|
||||||
try {
|
try {
|
||||||
logger.trace(
|
logger.trace(
|
||||||
"[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit);
|
"[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit);
|
||||||
metadata = store.getMetadata(snapshotIndexCommit);
|
metadataFromStore = store.getMetadata(snapshotIndexCommit);
|
||||||
fileNames = snapshotIndexCommit.getFileNames();
|
fileNames = snapshotIndexCommit.getFileNames();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
|
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
|
||||||
}
|
}
|
||||||
|
} finally {
|
||||||
|
store.decRef();
|
||||||
|
}
|
||||||
int indexIncrementalFileCount = 0;
|
int indexIncrementalFileCount = 0;
|
||||||
int indexTotalNumberOfFiles = 0;
|
int indexTotalNumberOfFiles = 0;
|
||||||
long indexIncrementalSize = 0;
|
long indexIncrementalSize = 0;
|
||||||
|
@ -958,7 +970,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
}
|
}
|
||||||
|
|
||||||
logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
|
logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
|
||||||
final StoreFileMetaData md = metadata.get(fileName);
|
final StoreFileMetaData md = metadataFromStore.get(fileName);
|
||||||
BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null;
|
BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null;
|
||||||
List<BlobStoreIndexShardSnapshot.FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName);
|
List<BlobStoreIndexShardSnapshot.FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName);
|
||||||
if (filesInfo != null) {
|
if (filesInfo != null) {
|
||||||
|
@ -991,18 +1003,12 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount,
|
snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount,
|
||||||
indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount);
|
indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount);
|
||||||
|
|
||||||
for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) {
|
assert indexIncrementalFileCount == filesToSnapshot.size();
|
||||||
try {
|
|
||||||
snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store);
|
|
||||||
} catch (IOException e) {
|
|
||||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
store.decRef();
|
|
||||||
}
|
|
||||||
|
|
||||||
final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration());
|
final StepListener<Collection<Void>> allFilesUploadedListener = new StepListener<>();
|
||||||
|
allFilesUploadedListener.whenComplete(v -> {
|
||||||
|
final IndexShardSnapshotStatus.Copy lastSnapshotStatus =
|
||||||
|
snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration());
|
||||||
|
|
||||||
// now create and write the commit point
|
// now create and write the commit point
|
||||||
final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(),
|
final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(),
|
||||||
|
@ -1020,7 +1026,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
|
throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
|
||||||
}
|
}
|
||||||
|
|
||||||
// delete all files that are not referenced by any commit point
|
// delete all files that are not referenced by any commit point
|
||||||
// build a new BlobStoreIndexShardSnapshot, that includes this one and all the saved ones
|
// build a new BlobStoreIndexShardSnapshot, that includes this one and all the saved ones
|
||||||
List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
|
List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
|
||||||
|
@ -1038,8 +1043,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
blobs.keySet().stream().filter(blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)).collect(Collectors.toList());
|
blobs.keySet().stream().filter(blob -> blob.startsWith(SNAPSHOT_INDEX_PREFIX)).collect(Collectors.toList());
|
||||||
assert blobsToDelete.stream().mapToLong(b -> Long.parseLong(b.replaceFirst(SNAPSHOT_INDEX_PREFIX, "")))
|
assert blobsToDelete.stream().mapToLong(b -> Long.parseLong(b.replaceFirst(SNAPSHOT_INDEX_PREFIX, "")))
|
||||||
.max().orElse(-1L) < Long.parseLong(indexGeneration)
|
.max().orElse(-1L) < Long.parseLong(indexGeneration)
|
||||||
: "Tried to delete an index-N blob newer than the current generation [" + indexGeneration + "] when deleting index-N" +
|
: "Tried to delete an index-N blob newer than the current generation [" + indexGeneration
|
||||||
" blobs " + blobsToDelete;
|
+ "] when deleting index-N blobs " + blobsToDelete;
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
throw new IndexShardSnapshotFailedException(shardId,
|
throw new IndexShardSnapshotFailedException(shardId,
|
||||||
"Failed to finalize snapshot creation [" + snapshotId + "] with shard index ["
|
"Failed to finalize snapshot creation [" + snapshotId + "] with shard index ["
|
||||||
|
@ -1052,13 +1057,40 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
snapshotId, shardId), e);
|
snapshotId, shardId), e);
|
||||||
}
|
}
|
||||||
snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis());
|
snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis());
|
||||||
} catch (Exception e) {
|
snapshotDoneListener.onResponse(null);
|
||||||
snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e));
|
}, snapshotDoneListener::onFailure);
|
||||||
if (e instanceof IndexShardSnapshotFailedException) {
|
if (indexIncrementalFileCount == 0) {
|
||||||
throw (IndexShardSnapshotFailedException) e;
|
allFilesUploadedListener.onResponse(Collections.emptyList());
|
||||||
} else {
|
return;
|
||||||
throw new IndexShardSnapshotFailedException(store.shardId(), e);
|
|
||||||
}
|
}
|
||||||
|
final GroupedActionListener<Void> filesListener =
|
||||||
|
new GroupedActionListener<>(allFilesUploadedListener, indexIncrementalFileCount);
|
||||||
|
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
|
||||||
|
// Flag to signal that the snapshot has been aborted/failed so we can stop any further blob uploads from starting
|
||||||
|
final AtomicBoolean alreadyFailed = new AtomicBoolean();
|
||||||
|
for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) {
|
||||||
|
executor.execute(new ActionRunnable<Void>(filesListener) {
|
||||||
|
@Override
|
||||||
|
protected void doRun() {
|
||||||
|
try {
|
||||||
|
if (alreadyFailed.get() == false) {
|
||||||
|
snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store);
|
||||||
|
}
|
||||||
|
filesListener.onResponse(null);
|
||||||
|
} catch (IOException e) {
|
||||||
|
throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void onFailure(Exception e) {
|
||||||
|
alreadyFailed.set(true);
|
||||||
|
super.onFailure(e);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
snapshotDoneListener.onFailure(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1245,6 +1277,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
IndexShardSnapshotStatus snapshotStatus, Store store) throws IOException {
|
IndexShardSnapshotStatus snapshotStatus, Store store) throws IOException {
|
||||||
final BlobContainer shardContainer = shardContainer(indexId, shardId);
|
final BlobContainer shardContainer = shardContainer(indexId, shardId);
|
||||||
final String file = fileInfo.physicalName();
|
final String file = fileInfo.physicalName();
|
||||||
|
store.incRef();
|
||||||
try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) {
|
try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) {
|
||||||
for (int i = 0; i < fileInfo.numberOfParts(); i++) {
|
for (int i = 0; i < fileInfo.numberOfParts(); i++) {
|
||||||
final long partBytes = fileInfo.partBytes(i);
|
final long partBytes = fileInfo.partBytes(i);
|
||||||
|
@ -1284,6 +1317,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
failStoreIfCorrupted(store, t);
|
failStoreIfCorrupted(store, t);
|
||||||
snapshotStatus.addProcessedFile(0);
|
snapshotStatus.addProcessedFile(0);
|
||||||
throw t;
|
throw t;
|
||||||
|
} finally {
|
||||||
|
store.decRef();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -23,7 +23,6 @@ import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||||
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.apache.lucene.util.SetOnce;
|
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.ActionRequestValidationException;
|
import org.elasticsearch.action.ActionRequestValidationException;
|
||||||
|
@ -53,9 +52,8 @@ import org.elasticsearch.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.SnapshotFailedEngineException;
|
|
||||||
import org.elasticsearch.index.shard.IndexEventListener;
|
import org.elasticsearch.index.shard.IndexEventListener;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardState;
|
import org.elasticsearch.index.shard.IndexShardState;
|
||||||
|
@ -79,7 +77,6 @@ import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.Executor;
|
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
@ -297,46 +294,33 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
|
||||||
}
|
}
|
||||||
|
|
||||||
private void startNewShards(SnapshotsInProgress.Entry entry, Map<ShardId, IndexShardSnapshotStatus> startedShards) {
|
private void startNewShards(SnapshotsInProgress.Entry entry, Map<ShardId, IndexShardSnapshotStatus> startedShards) {
|
||||||
|
threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> {
|
||||||
final Snapshot snapshot = entry.snapshot();
|
final Snapshot snapshot = entry.snapshot();
|
||||||
final Map<String, IndexId> indicesMap = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()));
|
final Map<String, IndexId> indicesMap =
|
||||||
final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
|
entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity()));
|
||||||
for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : startedShards.entrySet()) {
|
for (final Map.Entry<ShardId, IndexShardSnapshotStatus> shardEntry : startedShards.entrySet()) {
|
||||||
final ShardId shardId = shardEntry.getKey();
|
final ShardId shardId = shardEntry.getKey();
|
||||||
|
final IndexShardSnapshotStatus snapshotStatus = shardEntry.getValue();
|
||||||
final IndexId indexId = indicesMap.get(shardId.getIndexName());
|
final IndexId indexId = indicesMap.get(shardId.getIndexName());
|
||||||
assert indexId != null;
|
assert indexId != null;
|
||||||
executor.execute(new AbstractRunnable() {
|
snapshot(shardId, snapshot, indexId, snapshotStatus, new ActionListener<Void>() {
|
||||||
|
|
||||||
private final SetOnce<Exception> failure = new SetOnce<>();
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void doRun() {
|
public void onResponse(final Void aVoid) {
|
||||||
final IndexShard indexShard =
|
if (logger.isDebugEnabled()) {
|
||||||
indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id());
|
final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy();
|
||||||
snapshot(indexShard, snapshot, indexId, shardEntry.getValue());
|
logger.debug("snapshot ({}) completed to {} with {}", snapshot, snapshot.getRepository(), lastSnapshotStatus);
|
||||||
|
}
|
||||||
|
notifySuccessfulSnapshotShard(snapshot, shardId);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onFailure(Exception e) {
|
public void onFailure(Exception e) {
|
||||||
logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e);
|
logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e);
|
||||||
failure.set(e);
|
notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(e));
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onRejection(Exception e) {
|
|
||||||
failure.set(e);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void onAfter() {
|
|
||||||
final Exception exception = failure.get();
|
|
||||||
if (exception != null) {
|
|
||||||
notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(exception));
|
|
||||||
} else {
|
|
||||||
notifySuccessfulSnapshotShard(snapshot, shardId);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -345,9 +329,10 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
|
||||||
* @param snapshot snapshot
|
* @param snapshot snapshot
|
||||||
* @param snapshotStatus snapshot status
|
* @param snapshotStatus snapshot status
|
||||||
*/
|
*/
|
||||||
private void snapshot(final IndexShard indexShard, final Snapshot snapshot, final IndexId indexId,
|
private void snapshot(final ShardId shardId, final Snapshot snapshot, final IndexId indexId,
|
||||||
final IndexShardSnapshotStatus snapshotStatus) {
|
final IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
final ShardId shardId = indexShard.shardId();
|
try {
|
||||||
|
final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id());
|
||||||
if (indexShard.routingEntry().primary() == false) {
|
if (indexShard.routingEntry().primary() == false) {
|
||||||
throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary");
|
throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary");
|
||||||
}
|
}
|
||||||
|
@ -363,19 +348,18 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements
|
||||||
}
|
}
|
||||||
|
|
||||||
final Repository repository = repositoriesService.repository(snapshot.getRepository());
|
final Repository repository = repositoriesService.repository(snapshot.getRepository());
|
||||||
|
Engine.IndexCommitRef snapshotRef = null;
|
||||||
try {
|
try {
|
||||||
// we flush first to make sure we get the latest writes snapshotted
|
// we flush first to make sure we get the latest writes snapshotted
|
||||||
try (Engine.IndexCommitRef snapshotRef = indexShard.acquireLastIndexCommit(true)) {
|
snapshotRef = indexShard.acquireLastIndexCommit(true);
|
||||||
repository.snapshotShard(indexShard, snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), snapshotStatus);
|
repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId,
|
||||||
if (logger.isDebugEnabled()) {
|
snapshotRef.getIndexCommit(), snapshotStatus, ActionListener.runBefore(listener, snapshotRef::close));
|
||||||
final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy();
|
|
||||||
logger.debug("snapshot ({}) completed to {} with {}", snapshot, repository, lastSnapshotStatus);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} catch (SnapshotFailedEngineException | IndexShardSnapshotFailedException e) {
|
|
||||||
throw e;
|
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new IndexShardSnapshotFailedException(shardId, "Failed to snapshot", e);
|
IOUtils.close(snapshotRef);
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
listener.onFailure(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -171,6 +171,23 @@ public class ActionListenerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testRunBefore() {
|
||||||
|
{
|
||||||
|
AtomicBoolean afterSuccess = new AtomicBoolean();
|
||||||
|
ActionListener<Object> listener =
|
||||||
|
ActionListener.runBefore(ActionListener.wrap(r -> {}, e -> {}), () -> afterSuccess.set(true));
|
||||||
|
listener.onResponse(null);
|
||||||
|
assertThat(afterSuccess.get(), equalTo(true));
|
||||||
|
}
|
||||||
|
{
|
||||||
|
AtomicBoolean afterFailure = new AtomicBoolean();
|
||||||
|
ActionListener<Object> listener =
|
||||||
|
ActionListener.runBefore(ActionListener.wrap(r -> {}, e -> {}), () -> afterFailure.set(true));
|
||||||
|
listener.onFailure(null);
|
||||||
|
assertThat(afterFailure.get(), equalTo(true));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void testNotifyOnce() {
|
public void testNotifyOnce() {
|
||||||
AtomicInteger onResponseTimes = new AtomicInteger();
|
AtomicInteger onResponseTimes = new AtomicInteger();
|
||||||
AtomicInteger onFailureTimes = new AtomicInteger();
|
AtomicInteger onFailureTimes = new AtomicInteger();
|
||||||
|
|
|
@ -202,7 +202,7 @@ public class RepositoriesServiceTests extends ESTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit
|
||||||
snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.IOSupplier;
|
import org.apache.lucene.util.IOSupplier;
|
||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
|
import org.elasticsearch.cluster.metadata.RepositoryMetaData;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
@ -99,10 +100,12 @@ public class FsRepositoryTests extends ESTestCase {
|
||||||
IndexId indexId = new IndexId(idxSettings.getIndex().getName(), idxSettings.getUUID());
|
IndexId indexId = new IndexId(idxSettings.getIndex().getName(), idxSettings.getUUID());
|
||||||
|
|
||||||
IndexCommit indexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory());
|
IndexCommit indexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory());
|
||||||
|
final PlainActionFuture<Void> future1 = PlainActionFuture.newFuture();
|
||||||
runGeneric(threadPool, () -> {
|
runGeneric(threadPool, () -> {
|
||||||
IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
repository.snapshotShard(store, null, snapshotId, indexId, indexCommit,
|
repository.snapshotShard(store, null, snapshotId, indexId, indexCommit,
|
||||||
snapshotStatus);
|
snapshotStatus, future1);
|
||||||
|
future1.actionGet();
|
||||||
IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy();
|
||||||
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
||||||
});
|
});
|
||||||
|
@ -124,9 +127,11 @@ public class FsRepositoryTests extends ESTestCase {
|
||||||
SnapshotId incSnapshotId = new SnapshotId("test1", "test1");
|
SnapshotId incSnapshotId = new SnapshotId("test1", "test1");
|
||||||
IndexCommit incIndexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory());
|
IndexCommit incIndexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory());
|
||||||
Collection<String> commitFileNames = incIndexCommit.getFileNames();
|
Collection<String> commitFileNames = incIndexCommit.getFileNames();
|
||||||
|
final PlainActionFuture<Void> future2 = PlainActionFuture.newFuture();
|
||||||
runGeneric(threadPool, () -> {
|
runGeneric(threadPool, () -> {
|
||||||
IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
repository.snapshotShard(store, null, incSnapshotId, indexId, incIndexCommit, snapshotStatus);
|
repository.snapshotShard(store, null, incSnapshotId, indexId, incIndexCommit, snapshotStatus, future2);
|
||||||
|
future2.actionGet();
|
||||||
IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy();
|
||||||
assertEquals(2, copy.getIncrementalFileCount());
|
assertEquals(2, copy.getIncrementalFileCount());
|
||||||
assertEquals(commitFileNames.size(), copy.getTotalFileCount());
|
assertEquals(commitFileNames.size(), copy.getTotalFileCount());
|
||||||
|
@ -198,4 +203,5 @@ public class FsRepositoryTests extends ESTestCase {
|
||||||
return docs;
|
return docs;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -832,12 +832,14 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
final Snapshot snapshot,
|
final Snapshot snapshot,
|
||||||
final Repository repository) throws IOException {
|
final Repository repository) throws IOException {
|
||||||
final IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
final IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) {
|
try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) {
|
||||||
Index index = shard.shardId().getIndex();
|
Index index = shard.shardId().getIndex();
|
||||||
IndexId indexId = new IndexId(index.getName(), index.getUUID());
|
IndexId indexId = new IndexId(index.getName(), index.getUUID());
|
||||||
|
|
||||||
repository.snapshotShard(shard.store(), shard.mapperService(), snapshot.getSnapshotId(), indexId,
|
repository.snapshotShard(shard.store(), shard.mapperService(), snapshot.getSnapshotId(), indexId,
|
||||||
indexCommitRef.getIndexCommit(), snapshotStatus);
|
indexCommitRef.getIndexCommit(), snapshotStatus, future);
|
||||||
|
future.actionGet();
|
||||||
}
|
}
|
||||||
|
|
||||||
final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy();
|
final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy();
|
||||||
|
|
|
@ -135,7 +135,7 @@ public abstract class RestoreOnlyRepository extends AbstractLifecycleComponent i
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
||||||
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -296,7 +296,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
||||||
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
|
throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -15,6 +15,7 @@ import org.apache.lucene.store.Directory;
|
||||||
import org.apache.lucene.store.FSDirectory;
|
import org.apache.lucene.store.FSDirectory;
|
||||||
import org.apache.lucene.store.FilterDirectory;
|
import org.apache.lucene.store.FilterDirectory;
|
||||||
import org.apache.lucene.store.SimpleFSDirectory;
|
import org.apache.lucene.store.SimpleFSDirectory;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
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.metadata.MetaData;
|
import org.elasticsearch.cluster.metadata.MetaData;
|
||||||
|
@ -24,6 +25,7 @@ import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||||
import org.elasticsearch.common.lucene.search.Queries;
|
import org.elasticsearch.common.lucene.search.Queries;
|
||||||
import org.elasticsearch.common.settings.Setting;
|
import org.elasticsearch.common.settings.Setting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
import org.elasticsearch.env.ShardLock;
|
import org.elasticsearch.env.ShardLock;
|
||||||
import org.elasticsearch.index.engine.EngineFactory;
|
import org.elasticsearch.index.engine.EngineFactory;
|
||||||
import org.elasticsearch.index.engine.ReadOnlyEngine;
|
import org.elasticsearch.index.engine.ReadOnlyEngine;
|
||||||
|
@ -35,9 +37,11 @@ import org.elasticsearch.repositories.FilterRepository;
|
||||||
import org.elasticsearch.repositories.IndexId;
|
import org.elasticsearch.repositories.IndexId;
|
||||||
import org.elasticsearch.repositories.Repository;
|
import org.elasticsearch.repositories.Repository;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
@ -108,11 +112,13 @@ public final class SourceOnlySnapshotRepository extends FilterRepository {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId,
|
||||||
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) {
|
IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener<Void> listener) {
|
||||||
if (mapperService.documentMapper() != null // if there is no mapping this is null
|
if (mapperService.documentMapper() != null // if there is no mapping this is null
|
||||||
&& mapperService.documentMapper().sourceMapper().isComplete() == false) {
|
&& mapperService.documentMapper().sourceMapper().isComplete() == false) {
|
||||||
throw new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " +
|
listener.onFailure(
|
||||||
"or filters the source");
|
new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " +
|
||||||
|
"or filters the source"));
|
||||||
|
return;
|
||||||
}
|
}
|
||||||
Directory unwrap = FilterDirectory.unwrap(store.directory());
|
Directory unwrap = FilterDirectory.unwrap(store.directory());
|
||||||
if (unwrap instanceof FSDirectory == false) {
|
if (unwrap instanceof FSDirectory == false) {
|
||||||
|
@ -121,7 +127,10 @@ public final class SourceOnlySnapshotRepository extends FilterRepository {
|
||||||
Path dataPath = ((FSDirectory) unwrap).getDirectory().getParent();
|
Path dataPath = ((FSDirectory) unwrap).getDirectory().getParent();
|
||||||
// TODO should we have a snapshot tmp directory per shard that is maintained by the system?
|
// TODO should we have a snapshot tmp directory per shard that is maintained by the system?
|
||||||
Path snapPath = dataPath.resolve(SNAPSHOT_DIR_NAME);
|
Path snapPath = dataPath.resolve(SNAPSHOT_DIR_NAME);
|
||||||
try (FSDirectory directory = new SimpleFSDirectory(snapPath)) {
|
final List<Closeable> toClose = new ArrayList<>(3);
|
||||||
|
try {
|
||||||
|
FSDirectory directory = new SimpleFSDirectory(snapPath);
|
||||||
|
toClose.add(directory);
|
||||||
Store tempStore = new Store(store.shardId(), store.indexSettings(), directory, new ShardLock(store.shardId()) {
|
Store tempStore = new Store(store.shardId(), store.indexSettings(), directory, new ShardLock(store.shardId()) {
|
||||||
@Override
|
@Override
|
||||||
protected void closeInternal() {
|
protected void closeInternal() {
|
||||||
|
@ -137,16 +146,20 @@ public final class SourceOnlySnapshotRepository extends FilterRepository {
|
||||||
final long maxDoc = segmentInfos.totalMaxDoc();
|
final long maxDoc = segmentInfos.totalMaxDoc();
|
||||||
tempStore.bootstrapNewHistory(maxDoc, maxDoc);
|
tempStore.bootstrapNewHistory(maxDoc, maxDoc);
|
||||||
store.incRef();
|
store.incRef();
|
||||||
try (DirectoryReader reader = DirectoryReader.open(tempStore.directory(),
|
toClose.add(store::decRef);
|
||||||
Collections.singletonMap(BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name()))) {
|
DirectoryReader reader = DirectoryReader.open(tempStore.directory(),
|
||||||
|
Collections.singletonMap(BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name()));
|
||||||
|
toClose.add(reader);
|
||||||
IndexCommit indexCommit = reader.getIndexCommit();
|
IndexCommit indexCommit = reader.getIndexCommit();
|
||||||
super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus);
|
super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus,
|
||||||
} finally {
|
ActionListener.runBefore(listener, () -> IOUtils.close(toClose)));
|
||||||
store.decRef();
|
|
||||||
}
|
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
// why on earth does this super method not declare IOException
|
try {
|
||||||
throw new UncheckedIOException(e);
|
IOUtils.close(toClose);
|
||||||
|
} catch (IOException ex) {
|
||||||
|
e.addSuppressed(ex);
|
||||||
|
}
|
||||||
|
listener.onFailure(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -96,12 +96,13 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
|
||||||
repository.start();
|
repository.start();
|
||||||
try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) {
|
try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) {
|
||||||
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () ->
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
runAsSnapshot(shard.getThreadPool(),
|
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
||||||
() -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future));
|
||||||
snapshotRef.getIndexCommit(), indexShardSnapshotStatus)));
|
IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, future::actionGet);
|
||||||
assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source"
|
assertEquals(
|
||||||
, illegalStateException.getMessage());
|
"Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source",
|
||||||
|
illegalStateException.getMessage());
|
||||||
}
|
}
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
}
|
}
|
||||||
|
@ -120,8 +121,10 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
|
||||||
try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) {
|
try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) {
|
||||||
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
SnapshotId snapshotId = new SnapshotId("test", "test");
|
SnapshotId snapshotId = new SnapshotId("test", "test");
|
||||||
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
||||||
snapshotRef.getIndexCommit(), indexShardSnapshotStatus));
|
snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future));
|
||||||
|
future.actionGet();
|
||||||
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
||||||
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
||||||
totalFileCount = copy.getTotalFileCount();
|
totalFileCount = copy.getTotalFileCount();
|
||||||
|
@ -134,8 +137,10 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
|
||||||
SnapshotId snapshotId = new SnapshotId("test_1", "test_1");
|
SnapshotId snapshotId = new SnapshotId("test_1", "test_1");
|
||||||
|
|
||||||
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
||||||
snapshotRef.getIndexCommit(), indexShardSnapshotStatus));
|
snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future));
|
||||||
|
future.actionGet();
|
||||||
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
||||||
// we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt
|
// we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt
|
||||||
assertEquals(5, copy.getIncrementalFileCount());
|
assertEquals(5, copy.getIncrementalFileCount());
|
||||||
|
@ -148,8 +153,10 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
|
||||||
SnapshotId snapshotId = new SnapshotId("test_2", "test_2");
|
SnapshotId snapshotId = new SnapshotId("test_2", "test_2");
|
||||||
|
|
||||||
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing();
|
||||||
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId,
|
||||||
snapshotRef.getIndexCommit(), indexShardSnapshotStatus));
|
snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future));
|
||||||
|
future.actionGet();
|
||||||
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
||||||
// we processed the segments_N file plus _1_1.liv
|
// we processed the segments_N file plus _1_1.liv
|
||||||
assertEquals(2, copy.getIncrementalFileCount());
|
assertEquals(2, copy.getIncrementalFileCount());
|
||||||
|
@ -197,8 +204,10 @@ public class SourceOnlySnapshotShardTests extends IndexShardTestCase {
|
||||||
repository.initializeSnapshot(snapshotId, Arrays.asList(indexId),
|
repository.initializeSnapshot(snapshotId, Arrays.asList(indexId),
|
||||||
MetaData.builder().put(shard.indexSettings()
|
MetaData.builder().put(shard.indexSettings()
|
||||||
.getIndexMetaData(), false).build());
|
.getIndexMetaData(), false).build());
|
||||||
|
final PlainActionFuture<Void> future = PlainActionFuture.newFuture();
|
||||||
repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(),
|
repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(),
|
||||||
indexShardSnapshotStatus);
|
indexShardSnapshotStatus, future);
|
||||||
|
future.actionGet();
|
||||||
});
|
});
|
||||||
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy();
|
||||||
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount());
|
||||||
|
|
Loading…
Reference in New Issue