|
@@ -32,6 +32,7 @@ import org.elasticsearch.ExceptionsHelper;
|
|
|
import org.elasticsearch.Version;
|
|
|
import org.elasticsearch.action.ActionListener;
|
|
|
import org.elasticsearch.action.ActionRunnable;
|
|
|
+import org.elasticsearch.action.StepListener;
|
|
|
import org.elasticsearch.action.support.GroupedActionListener;
|
|
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
|
|
import org.elasticsearch.cluster.metadata.MetaData;
|
|
@@ -109,6 +110,7 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Optional;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.Executor;
|
|
|
import java.util.stream.Collectors;
|
|
|
|
|
|
import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName;
|
|
@@ -883,9 +885,15 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
|
|
|
@Override
|
|
|
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 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 {
|
|
|
logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name());
|
|
|
|
|
@@ -907,132 +915,145 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
}
|
|
|
|
|
|
final List<BlobStoreIndexShardSnapshot.FileInfo> indexCommitPointFiles = new ArrayList<>();
|
|
|
+ ArrayList<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot = new ArrayList<>();
|
|
|
store.incRef();
|
|
|
+ final Collection<String> fileNames;
|
|
|
+ final Store.MetadataSnapshot metadataFromStore;
|
|
|
try {
|
|
|
- ArrayList<BlobStoreIndexShardSnapshot.FileInfo> filesToSnapshot = new ArrayList<>();
|
|
|
- final Store.MetadataSnapshot metadata;
|
|
|
// TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should
|
|
|
- final Collection<String> fileNames;
|
|
|
try {
|
|
|
logger.trace(
|
|
|
"[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit);
|
|
|
- metadata = store.getMetadata(snapshotIndexCommit);
|
|
|
+ metadataFromStore = store.getMetadata(snapshotIndexCommit);
|
|
|
fileNames = snapshotIndexCommit.getFileNames();
|
|
|
} catch (IOException e) {
|
|
|
throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e);
|
|
|
}
|
|
|
- int indexIncrementalFileCount = 0;
|
|
|
- int indexTotalNumberOfFiles = 0;
|
|
|
- long indexIncrementalSize = 0;
|
|
|
- long indexTotalFileCount = 0;
|
|
|
- for (String fileName : fileNames) {
|
|
|
- if (snapshotStatus.isAborted()) {
|
|
|
- logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName);
|
|
|
- throw new IndexShardSnapshotFailedException(shardId, "Aborted");
|
|
|
- }
|
|
|
+ } finally {
|
|
|
+ store.decRef();
|
|
|
+ }
|
|
|
+ int indexIncrementalFileCount = 0;
|
|
|
+ int indexTotalNumberOfFiles = 0;
|
|
|
+ long indexIncrementalSize = 0;
|
|
|
+ long indexTotalFileCount = 0;
|
|
|
+ for (String fileName : fileNames) {
|
|
|
+ if (snapshotStatus.isAborted()) {
|
|
|
+ logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName);
|
|
|
+ throw new IndexShardSnapshotFailedException(shardId, "Aborted");
|
|
|
+ }
|
|
|
|
|
|
- logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
|
|
|
- final StoreFileMetaData md = metadata.get(fileName);
|
|
|
- BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null;
|
|
|
- List<BlobStoreIndexShardSnapshot.FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName);
|
|
|
- if (filesInfo != null) {
|
|
|
- for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) {
|
|
|
- if (fileInfo.isSame(md)) {
|
|
|
- // a commit point file with the same name, size and checksum was already copied to repository
|
|
|
- // we will reuse it for this snapshot
|
|
|
- existingFileInfo = fileInfo;
|
|
|
- break;
|
|
|
- }
|
|
|
+ logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
|
|
|
+ final StoreFileMetaData md = metadataFromStore.get(fileName);
|
|
|
+ BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null;
|
|
|
+ List<BlobStoreIndexShardSnapshot.FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName);
|
|
|
+ if (filesInfo != null) {
|
|
|
+ for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) {
|
|
|
+ if (fileInfo.isSame(md)) {
|
|
|
+ // a commit point file with the same name, size and checksum was already copied to repository
|
|
|
+ // we will reuse it for this snapshot
|
|
|
+ existingFileInfo = fileInfo;
|
|
|
+ break;
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- indexTotalFileCount += md.length();
|
|
|
- indexTotalNumberOfFiles++;
|
|
|
-
|
|
|
- if (existingFileInfo == null) {
|
|
|
- indexIncrementalFileCount++;
|
|
|
- indexIncrementalSize += md.length();
|
|
|
- // create a new FileInfo
|
|
|
- BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo =
|
|
|
- new BlobStoreIndexShardSnapshot.FileInfo(DATA_BLOB_PREFIX + UUIDs.randomBase64UUID(), md, chunkSize());
|
|
|
- indexCommitPointFiles.add(snapshotFileInfo);
|
|
|
- filesToSnapshot.add(snapshotFileInfo);
|
|
|
- } else {
|
|
|
- indexCommitPointFiles.add(existingFileInfo);
|
|
|
- }
|
|
|
}
|
|
|
|
|
|
- snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount,
|
|
|
- indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount);
|
|
|
-
|
|
|
- for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) {
|
|
|
- try {
|
|
|
- snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store);
|
|
|
- } catch (IOException e) {
|
|
|
- throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e);
|
|
|
- }
|
|
|
+ indexTotalFileCount += md.length();
|
|
|
+ indexTotalNumberOfFiles++;
|
|
|
+
|
|
|
+ if (existingFileInfo == null) {
|
|
|
+ indexIncrementalFileCount++;
|
|
|
+ indexIncrementalSize += md.length();
|
|
|
+ // create a new FileInfo
|
|
|
+ BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo =
|
|
|
+ new BlobStoreIndexShardSnapshot.FileInfo(DATA_BLOB_PREFIX + UUIDs.randomBase64UUID(), md, chunkSize());
|
|
|
+ indexCommitPointFiles.add(snapshotFileInfo);
|
|
|
+ filesToSnapshot.add(snapshotFileInfo);
|
|
|
+ } else {
|
|
|
+ indexCommitPointFiles.add(existingFileInfo);
|
|
|
}
|
|
|
- } finally {
|
|
|
- store.decRef();
|
|
|
}
|
|
|
|
|
|
- final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration());
|
|
|
+ snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount,
|
|
|
+ indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount);
|
|
|
|
|
|
- // now create and write the commit point
|
|
|
- final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(),
|
|
|
- lastSnapshotStatus.getIndexVersion(),
|
|
|
- indexCommitPointFiles,
|
|
|
- lastSnapshotStatus.getStartTime(),
|
|
|
- threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(),
|
|
|
- lastSnapshotStatus.getIncrementalFileCount(),
|
|
|
- lastSnapshotStatus.getIncrementalSize()
|
|
|
- );
|
|
|
+ assert indexIncrementalFileCount == filesToSnapshot.size();
|
|
|
|
|
|
- logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId);
|
|
|
- try {
|
|
|
- indexShardSnapshotFormat.write(snapshot, shardContainer, snapshotId.getUUID());
|
|
|
- } catch (IOException e) {
|
|
|
- throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
|
|
|
- }
|
|
|
+ final StepListener<Collection<Void>> allFilesUploadedListener = new StepListener<>();
|
|
|
+ allFilesUploadedListener.whenComplete(v -> {
|
|
|
+ final IndexShardSnapshotStatus.Copy lastSnapshotStatus =
|
|
|
+ snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration());
|
|
|
|
|
|
- // delete all files that are not referenced by any commit point
|
|
|
- // build a new BlobStoreIndexShardSnapshot, that includes this one and all the saved ones
|
|
|
- List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
|
|
|
- newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()));
|
|
|
- for (SnapshotFiles point : snapshots) {
|
|
|
- newSnapshotsList.add(point);
|
|
|
- }
|
|
|
- final String indexGeneration = Long.toString(fileListGeneration + 1);
|
|
|
- final List<String> blobsToDelete;
|
|
|
- try {
|
|
|
- final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(newSnapshotsList);
|
|
|
- indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, shardContainer, indexGeneration);
|
|
|
- // Delete all previous index-N blobs
|
|
|
- blobsToDelete =
|
|
|
- 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, "")))
|
|
|
- .max().orElse(-1L) < Long.parseLong(indexGeneration)
|
|
|
- : "Tried to delete an index-N blob newer than the current generation [" + indexGeneration + "] when deleting index-N" +
|
|
|
- " blobs " + blobsToDelete;
|
|
|
- } catch (IOException e) {
|
|
|
- throw new IndexShardSnapshotFailedException(shardId,
|
|
|
- "Failed to finalize snapshot creation [" + snapshotId + "] with shard index ["
|
|
|
- + indexShardSnapshotsFormat.blobName(indexGeneration) + "]", e);
|
|
|
+ // now create and write the commit point
|
|
|
+ final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(),
|
|
|
+ lastSnapshotStatus.getIndexVersion(),
|
|
|
+ indexCommitPointFiles,
|
|
|
+ lastSnapshotStatus.getStartTime(),
|
|
|
+ threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(),
|
|
|
+ lastSnapshotStatus.getIncrementalFileCount(),
|
|
|
+ lastSnapshotStatus.getIncrementalSize()
|
|
|
+ );
|
|
|
+
|
|
|
+ logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId);
|
|
|
+ try {
|
|
|
+ indexShardSnapshotFormat.write(snapshot, shardContainer, snapshotId.getUUID());
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e);
|
|
|
+ }
|
|
|
+ // delete all files that are not referenced by any commit point
|
|
|
+ // build a new BlobStoreIndexShardSnapshot, that includes this one and all the saved ones
|
|
|
+ List<SnapshotFiles> newSnapshotsList = new ArrayList<>();
|
|
|
+ newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()));
|
|
|
+ for (SnapshotFiles point : snapshots) {
|
|
|
+ newSnapshotsList.add(point);
|
|
|
+ }
|
|
|
+ final String indexGeneration = Long.toString(fileListGeneration + 1);
|
|
|
+ final List<String> blobsToDelete;
|
|
|
+ try {
|
|
|
+ final BlobStoreIndexShardSnapshots updatedSnapshots = new BlobStoreIndexShardSnapshots(newSnapshotsList);
|
|
|
+ indexShardSnapshotsFormat.writeAtomic(updatedSnapshots, shardContainer, indexGeneration);
|
|
|
+ // Delete all previous index-N blobs
|
|
|
+ blobsToDelete =
|
|
|
+ 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, "")))
|
|
|
+ .max().orElse(-1L) < Long.parseLong(indexGeneration)
|
|
|
+ : "Tried to delete an index-N blob newer than the current generation [" + indexGeneration
|
|
|
+ + "] when deleting index-N blobs " + blobsToDelete;
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new IndexShardSnapshotFailedException(shardId,
|
|
|
+ "Failed to finalize snapshot creation [" + snapshotId + "] with shard index ["
|
|
|
+ + indexShardSnapshotsFormat.blobName(indexGeneration) + "]", e);
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ shardContainer.deleteBlobsIgnoringIfNotExists(blobsToDelete);
|
|
|
+ } catch (IOException e) {
|
|
|
+ logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete old index-N blobs during finalization",
|
|
|
+ snapshotId, shardId), e);
|
|
|
+ }
|
|
|
+ snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis());
|
|
|
+ snapshotDoneListener.onResponse(null);
|
|
|
+ }, snapshotDoneListener::onFailure);
|
|
|
+ if (indexIncrementalFileCount == 0) {
|
|
|
+ allFilesUploadedListener.onResponse(Collections.emptyList());
|
|
|
+ return;
|
|
|
}
|
|
|
- try {
|
|
|
- shardContainer.deleteBlobsIgnoringIfNotExists(blobsToDelete);
|
|
|
- } catch (IOException e) {
|
|
|
- logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete old index-N blobs during finalization",
|
|
|
- snapshotId, shardId), e);
|
|
|
+ final GroupedActionListener<Void> filesListener =
|
|
|
+ new GroupedActionListener<>(allFilesUploadedListener, indexIncrementalFileCount);
|
|
|
+ final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT);
|
|
|
+ for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) {
|
|
|
+ executor.execute(new ActionRunnable<>(filesListener) {
|
|
|
+ @Override
|
|
|
+ protected void doRun() {
|
|
|
+ try {
|
|
|
+ snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store);
|
|
|
+ filesListener.onResponse(null);
|
|
|
+ } catch (IOException e) {
|
|
|
+ throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ });
|
|
|
}
|
|
|
- snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis());
|
|
|
} catch (Exception e) {
|
|
|
- snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e));
|
|
|
- if (e instanceof IndexShardSnapshotFailedException) {
|
|
|
- throw (IndexShardSnapshotFailedException) e;
|
|
|
- } else {
|
|
|
- throw new IndexShardSnapshotFailedException(store.shardId(), e);
|
|
|
- }
|
|
|
+ snapshotDoneListener.onFailure(e);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1219,6 +1240,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
IndexShardSnapshotStatus snapshotStatus, Store store) throws IOException {
|
|
|
final BlobContainer shardContainer = shardContainer(indexId, shardId);
|
|
|
final String file = fileInfo.physicalName();
|
|
|
+ store.incRef();
|
|
|
try (IndexInput indexInput = store.openVerifyingInput(file, IOContext.READONCE, fileInfo.metadata())) {
|
|
|
for (int i = 0; i < fileInfo.numberOfParts(); i++) {
|
|
|
final long partBytes = fileInfo.partBytes(i);
|
|
@@ -1258,6 +1280,8 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
|
|
failStoreIfCorrupted(store, t);
|
|
|
snapshotStatus.addProcessedFile(0);
|
|
|
throw t;
|
|
|
+ } finally {
|
|
|
+ store.decRef();
|
|
|
}
|
|
|
}
|
|
|
|