From 503645563fd6662538d8ec2d1ff10e5badb40473 Mon Sep 17 00:00:00 2001 From: Armin Date: Sun, 2 Jun 2019 13:46:58 +0200 Subject: [PATCH 01/23] startt --- .../blobstore/BlobStoreRepository.java | 258 ++++++++++-------- 1 file changed, 141 insertions(+), 117 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 1cb50f0f1a0da..bcd0f0e79aace 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -33,6 +33,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.GroupedActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; @@ -60,6 +61,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.concurrent.RunOnce; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -799,17 +801,28 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b @Override public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { - SnapshotContext snapshotContext = new SnapshotContext(store, snapshotId, indexId, snapshotStatus, System.currentTimeMillis()); - try { - snapshotContext.snapshot(snapshotIndexCommit); - } catch (Exception e) { - snapshotStatus.moveToFailed(System.currentTimeMillis(), ExceptionsHelper.detailedMessage(e)); - if (e instanceof IndexShardSnapshotFailedException) { - throw (IndexShardSnapshotFailedException) e; - } else { - throw new IndexShardSnapshotFailedException(store.shardId(), e); - } - } + final PlainActionFuture future = PlainActionFuture.newFuture(); + new SnapshotContext(store, snapshotId, indexId, snapshotStatus, threadPool.absoluteTimeInMillis()).snapshot(snapshotIndexCommit, + new ActionListener<>() { + @Override + public void onResponse(IndexShardSnapshotStatus indexShardSnapshotStatus) { + snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis()); + future.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e)); + final IndexShardSnapshotFailedException ex; + if (e instanceof IndexShardSnapshotFailedException) { + ex = (IndexShardSnapshotFailedException) e; + } else { + ex = new IndexShardSnapshotFailedException(store.shardId(), e); + } + future.onFailure(ex); + } + }); + future.actionGet(); } @Override @@ -1088,127 +1101,138 @@ private class SnapshotContext extends Context { * * @param snapshotIndexCommit snapshot commit point */ - public void snapshot(final IndexCommit snapshotIndexCommit) { + public void snapshot(IndexCommit snapshotIndexCommit, ActionListener listener) { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); + listBlobs(ActionListener.wrap( + blobs -> { + Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs); + BlobStoreIndexShardSnapshots snapshots = tuple.v1(); + int fileListGeneration = tuple.v2(); + + if (snapshots.snapshots().stream().anyMatch(sf -> sf.snapshot().equals(snapshotId.getName()))) { + throw new IndexShardSnapshotFailedException(shardId, + "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting"); + } - final Map blobs; - try { - blobs = blobContainer.listBlobs(); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "failed to list blobs", e); - } - - Tuple tuple = buildBlobStoreIndexShardSnapshots(blobs); - BlobStoreIndexShardSnapshots snapshots = tuple.v1(); - int fileListGeneration = tuple.v2(); - - if (snapshots.snapshots().stream().anyMatch(sf -> sf.snapshot().equals(snapshotId.getName()))) { - throw new IndexShardSnapshotFailedException(shardId, - "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting"); - } - - final List indexCommitPointFiles = new ArrayList<>(); + final Runnable decStoreRef = new RunOnce(store::decRef); + store.incRef(); + snapshotFiles(snapshotIndexCommit, blobs, snapshots, ActionListener.wrap(snapshot -> { + decStoreRef.run(); + //TODO: The time stored in snapshot doesn't include cleanup time. + logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); + try { + indexShardSnapshotFormat.write(snapshot, blobContainer, 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 newSnapshotsList = new ArrayList<>(); + newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); + for (SnapshotFiles point : snapshots) { + newSnapshotsList.add(point); + } + // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index + finalize(newSnapshotsList, fileListGeneration + 1, blobs, "snapshot creation [" + snapshotId + "]"); + listener.onResponse(snapshotStatus); + }, e -> { + decStoreRef.run(); + listener.onFailure(e); + })); + }, listener::onFailure + )); + } - store.incRef(); - int indexIncrementalFileCount = 0; - int indexTotalNumberOfFiles = 0; - long indexIncrementalSize = 0; - long indexTotalFileCount = 0; - try { - ArrayList filesToSnapshot = new ArrayList<>(); - final Store.MetadataSnapshot metadata; - // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should - final Collection fileNames; + private void listBlobs(ActionListener> listener) { + ActionListener.completeWith(listener, () -> { try { - logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); - metadata = store.getMetadata(snapshotIndexCommit); - fileNames = snapshotIndexCommit.getFileNames(); + return blobContainer.listBlobs(); } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); + throw new IndexShardSnapshotFailedException(shardId, "failed to list blobs", e); } - for (String fileName : fileNames) { - if (snapshotStatus.isAborted()) { - logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); - throw new IndexShardSnapshotFailedException(shardId, "Aborted"); + }); + } + + private void snapshotFiles(IndexCommit snapshotIndexCommit, Map blobs, + BlobStoreIndexShardSnapshots snapshots, + ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final Store.MetadataSnapshot meta; + // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should + final Collection fileNames; + try { + logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); + meta = 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; + final List filesToSnapshot = new ArrayList<>(); + final List indexCommitPointFiles = new ArrayList<>(); + 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 filesInfo = snapshots.findPhysicalIndexFiles(fileName); - if (filesInfo != null) { - for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { - if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { - // 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 = meta.get(fileName); + BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null; + List filesInfo = snapshots.findPhysicalIndexFiles(fileName); + if (filesInfo != null) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { + if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { + // 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); + 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); + snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount, + indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount); - for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { - try { - snapshotFile(snapshotFileInfo); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); + for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { + try { + snapshotFile(snapshotFileInfo); + } 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()); - - // now create and write the commit point - final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(), - lastSnapshotStatus.getIndexVersion(), - indexCommitPointFiles, - lastSnapshotStatus.getStartTime(), - // snapshotStatus.startTime() is assigned on the same machine, - // so it's safe to use with VLong - System.currentTimeMillis() - lastSnapshotStatus.getStartTime(), - lastSnapshotStatus.getIncrementalFileCount(), - lastSnapshotStatus.getIncrementalSize() - ); - - //TODO: The time stored in snapshot doesn't include cleanup time. - logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); - try { - indexShardSnapshotFormat.write(snapshot, blobContainer, 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 newSnapshotsList = new ArrayList<>(); - newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); - for (SnapshotFiles point : snapshots) { - newSnapshotsList.add(point); - } - // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index - finalize(newSnapshotsList, fileListGeneration + 1, blobs, "snapshot creation [" + snapshotId + "]"); - snapshotStatus.moveToDone(System.currentTimeMillis()); + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = + snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); + return new BlobStoreIndexShardSnapshot(snapshotId.getName(), + lastSnapshotStatus.getIndexVersion(), + indexCommitPointFiles, + lastSnapshotStatus.getStartTime(), + // snapshotStatus.startTime() is assigned on the same machine, + // so it's safe to use with VLong + System.currentTimeMillis() - lastSnapshotStatus.getStartTime(), + lastSnapshotStatus.getIncrementalFileCount(), + lastSnapshotStatus.getIncrementalSize() + ); + }); } /** From 019682dd0e691e2a581b050166081eca8d6fd812 Mon Sep 17 00:00:00 2001 From: Armin Date: Sun, 2 Jun 2019 14:05:22 +0200 Subject: [PATCH 02/23] simpler --- .../snapshots/SnapshotShardsService.java | 22 ++----------------- 1 file changed, 2 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 65e1191211ec2..fe0f23537d942 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -23,7 +23,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; @@ -304,35 +303,18 @@ private void startNewShards(SnapshotsInProgress.Entry entry, Map failure = new SetOnce<>(); - @Override public void doRun() { final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id()); snapshot(indexShard, snapshot, indexId, shardEntry.getValue()); + notifySuccessfulSnapshotShard(snapshot, shardId); } @Override public void onFailure(Exception e) { logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); - failure.set(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); - } + notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(e)); } }); } From 3529f3645e3805c5b30f18ff4caca55fc7d07ccd Mon Sep 17 00:00:00 2001 From: Armin Date: Sun, 2 Jun 2019 22:20:00 +0200 Subject: [PATCH 03/23] start --- .../repositories/FilterRepository.java | 6 +- .../repositories/Repository.java | 20 +- .../blobstore/BlobStoreRepository.java | 195 +++++++++++------- .../snapshots/SnapshotShardsService.java | 114 ++++++---- .../RepositoriesServiceTests.java | 5 +- .../repositories/fs/FsRepositoryTests.java | 64 +++++- .../index/shard/IndexShardTestCase.java | 34 ++- .../index/shard/RestoreOnlyRepository.java | 6 +- .../xpack/ccr/repository/CcrRepository.java | 5 +- .../SourceOnlySnapshotRepository.java | 67 +++++- .../SourceOnlySnapshotShardTests.java | 30 +-- 11 files changed, 386 insertions(+), 160 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 1fa42579617e1..dba671f16dd6f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.repositories; -import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -121,9 +120,8 @@ public boolean isReadOnly() { @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { - in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus); + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, ShardSnapshotContext context) { + in.snapshotShard(mapperService, snapshotId, indexId, context); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 0eca92039fbf8..b4ecb37a8ec4b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -35,6 +35,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; +import java.io.Closeable; import java.io.IOException; import java.util.List; import java.util.function.Function; @@ -49,7 +50,7 @@ *
    *
  • Master calls {@link #initializeSnapshot(SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)} * with list of indices that will be included into the snapshot
  • - *
  • Data nodes call {@link Repository#snapshotShard(Store, MapperService, SnapshotId, IndexId, IndexCommit, IndexShardSnapshotStatus)} + *
  • Data nodes call {@link Repository#snapshotShard} * for each shard
  • *
  • When all shard calls return master calls {@link #finalizeSnapshot} with possible list of failures
  • *
@@ -196,15 +197,12 @@ SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long *

* 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 store store to be snapshotted * @param mapperService the shards mapper service * @param snapshotId snapshot id * @param indexId id for the index being snapshotted - * @param snapshotIndexCommit commit point - * @param snapshotStatus snapshot status + * @param context shard snapshot context */ - void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, - IndexShardSnapshotStatus snapshotStatus); + void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, ShardSnapshotContext context); /** * Restores snapshot of the shard. @@ -232,4 +230,14 @@ void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId i IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId); + interface ShardSnapshotContext extends Closeable { + + IndexCommit indexCommit(); + + Store store(); + + IndexShardSnapshotStatus status(); + + ActionListener completionListener(); + } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index bcd0f0e79aace..be9dccbce08d3 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -33,7 +33,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.support.GroupedActionListener; -import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.RepositoryMetaData; @@ -61,7 +60,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.concurrent.RunOnce; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -112,6 +110,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; @@ -799,30 +798,31 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b } @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { - final PlainActionFuture future = PlainActionFuture.newFuture(); - new SnapshotContext(store, snapshotId, indexId, snapshotStatus, threadPool.absoluteTimeInMillis()).snapshot(snapshotIndexCommit, + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + Repository.ShardSnapshotContext context) { + final Store store = context.store(); + final IndexShardSnapshotStatus status = context.status(); + new SnapshotContext(store, snapshotId, indexId, status, threadPool.absoluteTimeInMillis()).snapshot( + context, new ActionListener<>() { @Override public void onResponse(IndexShardSnapshotStatus indexShardSnapshotStatus) { - snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis()); - future.onResponse(null); + status.moveToDone(threadPool.absoluteTimeInMillis()); + context.completionListener().onResponse(null); } @Override public void onFailure(Exception e) { - snapshotStatus.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e)); + status.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e)); final IndexShardSnapshotFailedException ex; if (e instanceof IndexShardSnapshotFailedException) { ex = (IndexShardSnapshotFailedException) e; } else { ex = new IndexShardSnapshotFailedException(store.shardId(), e); } - future.onFailure(ex); + context.completionListener().onFailure(ex); } }); - future.actionGet(); } @Override @@ -1099,9 +1099,9 @@ private class SnapshotContext extends Context { /** * Create snapshot from index commit point * - * @param snapshotIndexCommit snapshot commit point + * @param context shard snapshot context */ - public void snapshot(IndexCommit snapshotIndexCommit, ActionListener listener) { + public void snapshot(Repository.ShardSnapshotContext context, ActionListener listener) { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); listBlobs(ActionListener.wrap( blobs -> { @@ -1114,10 +1114,7 @@ public void snapshot(IndexCommit snapshotIndexCommit, ActionListener { - decStoreRef.run(); + snapshotFiles(context, blobs, snapshots, ActionListener.wrap(snapshot -> { //TODO: The time stored in snapshot doesn't include cleanup time. logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); try { @@ -1135,10 +1132,7 @@ public void snapshot(IndexCommit snapshotIndexCommit, ActionListener { - decStoreRef.run(); - listener.onFailure(e); - })); + }, listener::onFailure)); }, listener::onFailure )); } @@ -1153,73 +1147,73 @@ private void listBlobs(ActionListener> listener) { }); } - private void snapshotFiles(IndexCommit snapshotIndexCommit, Map blobs, + private void snapshotFiles(Repository.ShardSnapshotContext context, Map blobs, BlobStoreIndexShardSnapshots snapshots, ActionListener listener) { - ActionListener.completeWith(listener, () -> { - final Store.MetadataSnapshot meta; + try { // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should + store.incRef(); + final IndexCommit snapshotIndexCommit = context.indexCommit(); final Collection fileNames; - try { - logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); - meta = store.getMetadata(snapshotIndexCommit); - fileNames = snapshotIndexCommit.getFileNames(); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); + final Store.MetadataSnapshot meta; + try { + logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); + meta = store.getMetadata(snapshotIndexCommit); + fileNames = snapshotIndexCommit.getFileNames(); + } catch (IOException e) { + throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); + } finally { + store.decRef(); + } + int indexIncrementalFileCount = 0; + int indexTotalNumberOfFiles = 0; + long indexIncrementalSize = 0; + long indexTotalFileCount = 0; + final List filesToSnapshot = new ArrayList<>(); + final List indexCommitPointFiles = new ArrayList<>(); + for (String fileName : fileNames) { + if (snapshotStatus.isAborted()) { + logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); + throw new IndexShardSnapshotFailedException(shardId, "Aborted"); } - int indexIncrementalFileCount = 0; - int indexTotalNumberOfFiles = 0; - long indexIncrementalSize = 0; - long indexTotalFileCount = 0; - final List filesToSnapshot = new ArrayList<>(); - final List indexCommitPointFiles = new ArrayList<>(); - 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 = meta.get(fileName); - BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null; - List filesInfo = snapshots.findPhysicalIndexFiles(fileName); - if (filesInfo != null) { - for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { - if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { - // 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 = meta.get(fileName); + BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null; + List filesInfo = snapshots.findPhysicalIndexFiles(fileName); + if (filesInfo != null) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { + if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { + // 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); - } + 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); + snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount, + indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount); - for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { - try { - snapshotFile(snapshotFileInfo); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); - } - } + assert indexIncrementalFileCount == filesToSnapshot.size(); + final Runnable afterSegmentFiles = () -> ActionListener.completeWith(listener, () -> { + context.close(); final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); return new BlobStoreIndexShardSnapshot(snapshotId.getName(), @@ -1232,7 +1226,51 @@ private void snapshotFiles(IndexCommit snapshotIndexCommit, Map 0) { + final GroupedActionListener fileCompletionListener = new GroupedActionListener<>( + ActionListener.wrap(v -> afterSegmentFiles.run(), listener::onFailure), indexIncrementalFileCount); + final ExecutorService executorService = threadPool.executor(ThreadPool.Names.SNAPSHOT); + for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { + executorService.execute(new ActionRunnable<>(fileCompletionListener) { + @Override + protected void doRun() { + try { + if (snapshotStatus.isAborted()) { + context.close(); + throw new IndexShardSnapshotFailedException(shardId, "Aborted"); + } + snapshotFile(snapshotFileInfo); + fileCompletionListener.onResponse(null); + } catch (IOException e) { + try { + context.close(); + } catch (Exception ex) { + e.addSuppressed(ex); + } + throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); + } catch (Exception e) { + try { + context.close(); + } catch (Exception ex) { + e.addSuppressed(ex); + } + throw e; + } + } + }); + } + } else { + afterSegmentFiles.run(); + } + } catch (Exception e) { + try { + context.close(); + } catch (Exception ex) { + e.addSuppressed(ex); + } + listener.onFailure(e); + } } /** @@ -1242,6 +1280,7 @@ private void snapshotFiles(IndexCommit snapshotIndexCommit, Map startedShards) { final Snapshot snapshot = entry.snapshot(); final Map indicesMap = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity())); - final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); - for (final Map.Entry shardEntry : startedShards.entrySet()) { - final ShardId shardId = shardEntry.getKey(); - final IndexId indexId = indicesMap.get(shardId.getIndexName()); - assert indexId != null; - executor.execute(new AbstractRunnable() { - @Override - public void doRun() { - final IndexShard indexShard = - indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id()); - snapshot(indexShard, snapshot, indexId, shardEntry.getValue()); - notifySuccessfulSnapshotShard(snapshot, shardId); - } + threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> { + for (final Map.Entry shardEntry : startedShards.entrySet()) { + final ShardId shardId = shardEntry.getKey(); + final IndexId indexId = indicesMap.get(shardId.getIndexName()); + assert indexId != null; + final IndexShardSnapshotStatus snapshotStatus = shardEntry.getValue(); + final ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(final Void aVoid) { + if (logger.isDebugEnabled()) { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); + logger.debug("snapshot ({}) completed to {} with {}", snapshot, snapshot.getRepository(), lastSnapshotStatus); + } + notifySuccessfulSnapshotShard(snapshot, shardId); + } - @Override - public void onFailure(Exception e) { - logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); - notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(e)); + @Override + public void onFailure(Exception e) { + logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); + notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(e)); + } + }; + try { + snapshot(indicesService.indexServiceSafe( + shardId.getIndex()).getShardOrNull(shardId.id()), snapshot, indexId, snapshotStatus, listener); + } catch (Exception e) { + listener.onFailure(e); } - }); - } + } + }); } /** @@ -327,7 +336,7 @@ public void onFailure(Exception e) { * @param snapshotStatus snapshot status */ private void snapshot(final IndexShard indexShard, final Snapshot snapshot, final IndexId indexId, - final IndexShardSnapshotStatus snapshotStatus) { + final IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { final ShardId shardId = indexShard.shardId(); if (indexShard.routingEntry().primary() == false) { throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary"); @@ -344,21 +353,52 @@ private void snapshot(final IndexShard indexShard, final Snapshot snapshot, fina } final Repository repository = snapshotsService.getRepositoriesService().repository(snapshot.getRepository()); - try { - // we flush first to make sure we get the latest writes snapshotted - try (Engine.IndexCommitRef snapshotRef = indexShard.acquireLastIndexCommit(true)) { - repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId, - snapshotRef.getIndexCommit(), snapshotStatus); - if (logger.isDebugEnabled()) { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); - logger.debug("snapshot ({}) completed to {} with {}", snapshot, repository, lastSnapshotStatus); + // we flush first to make sure we get the latest writes snapshotted + repository.snapshotShard(indexShard.mapperService(), snapshot.getSnapshotId(), indexId, + new Repository.ShardSnapshotContext() { + + private final AtomicBoolean closed = new AtomicBoolean(false); + private Engine.IndexCommitRef snapshotRef; + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + synchronized (this) { + if (snapshotRef != null) { + snapshotRef.close(); + } + } + } } - } - } catch (SnapshotFailedEngineException | IndexShardSnapshotFailedException e) { - throw e; - } catch (Exception e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to snapshot", e); - } + + @Override + public IndexCommit indexCommit() { + synchronized (this) { + if (closed.get()) { + throw new IllegalStateException("Tried to get index commit from closed context"); + } + if (snapshotRef == null) { + snapshotRef = indexShard.acquireLastIndexCommit(true); + } + return snapshotRef.getIndexCommit(); + } + } + + @Override + public Store store() { + return indexShard.store(); + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return listener; + } + }); } /** diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index ae703795ec622..4f4954be61f31 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.repositories; -import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; @@ -200,8 +199,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit - snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + Repository.ShardSnapshotContext context) { } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index ec8a444d84fae..2148bf97cb162 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -35,6 +35,8 @@ import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.TestUtil; 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.RepositoryMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -53,6 +55,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.Repository; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.DummyShardLock; @@ -99,10 +102,37 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { IndexId indexId = new IndexId(idxSettings.getIndex().getName(), idxSettings.getUUID()); IndexCommit indexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory()); + final PlainActionFuture future1 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(store, null, snapshotId, indexId, indexCommit, - snapshotStatus); + repository.snapshotShard(null, snapshotId, indexId, new Repository.ShardSnapshotContext() { + + @Override + public void close() { + + } + + @Override + public IndexCommit indexCommit() { + return indexCommit; + } + + @Override + public Store store() { + return store; + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return future1; + } + }); + future1.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); }); @@ -124,9 +154,37 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { SnapshotId incSnapshotId = new SnapshotId("test1", "test1"); IndexCommit incIndexCommit = Lucene.getIndexCommit(Lucene.readSegmentInfos(store.directory()), store.directory()); Collection commitFileNames = incIndexCommit.getFileNames(); + final PlainActionFuture future2 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(store, null, incSnapshotId, indexId, incIndexCommit, snapshotStatus); + repository.snapshotShard(null, incSnapshotId, indexId, new Repository.ShardSnapshotContext() { + + @Override + public void close() { + + } + + @Override + public IndexCommit indexCommit() { + return incIndexCommit; + } + + @Override + public Store store() { + return store; + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return future2; + } + }); + future2.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(2, copy.getIncrementalFileCount()); assertEquals(commitFileNames.size(), copy.getTotalFileCount()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 2a2176f1c100d..943c82bea7c0f 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -18,9 +18,11 @@ */ package org.elasticsearch.index.shard; +import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexNotFoundException; import org.apache.lucene.store.Directory; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; @@ -822,9 +824,35 @@ protected void snapshotShard(final IndexShard shard, try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) { Index index = shard.shardId().getIndex(); IndexId indexId = new IndexId(index.getName(), index.getUUID()); - - repository.snapshotShard(shard.store(), shard.mapperService(), snapshot.getSnapshotId(), indexId, - indexCommitRef.getIndexCommit(), snapshotStatus); + final PlainActionFuture future = PlainActionFuture.newFuture(); + final IndexCommit indexCommit = indexCommitRef.getIndexCommit(); + repository.snapshotShard(shard.mapperService(), snapshot.getSnapshotId(), indexId, + new Repository.ShardSnapshotContext() { + @Override + public IndexCommit indexCommit() { + return indexCommit; + } + + @Override + public Store store() { + return shard.store(); + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return future; + } + + @Override + public void close() { + } + }); + future.actionGet(); } final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index 2279b48c3c023..1bbd3f70b96bd 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.index.shard; -import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -28,7 +27,6 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; -import org.elasticsearch.index.store.Store; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; @@ -134,8 +132,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + Repository.ShardSnapshotContext context) { } @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 3010f90b803e9..7fb2af6bb4858 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.index.IndexCommit; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; @@ -294,8 +293,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + Repository.ShardSnapshotContext context) { throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index bb5819e1bda43..9eb5ab6e2b436 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -14,6 +14,7 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.SimpleFSDirectory; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -39,6 +40,7 @@ import java.nio.file.Path; import java.util.Iterator; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.function.Supplier; @@ -105,13 +107,14 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met } @Override - public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus) { + public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + Repository.ShardSnapshotContext context) { if (mapperService.documentMapper() != null // if there is no mapping this is null && mapperService.documentMapper().sourceMapper().isComplete() == false) { throw new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " + "or filters the source"); } + final Store store = context.store(); Directory unwrap = FilterDirectory.unwrap(store.directory()); if (unwrap instanceof FSDirectory == false) { throw new AssertionError("expected FSDirectory but got " + unwrap.toString()); @@ -129,15 +132,67 @@ protected void closeInternal() { Supplier querySupplier = mapperService.hasNested() ? Queries::newNestedFilter : null; // SourceOnlySnapshot will take care of soft- and hard-deletes no special casing needed here SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), querySupplier); - snapshot.syncSnapshot(snapshotIndexCommit); + snapshot.syncSnapshot(context.indexCommit()); // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID SegmentInfos segmentInfos = tempStore.readLastCommittedSegmentsInfo(); final long maxDoc = segmentInfos.totalMaxDoc(); tempStore.bootstrapNewHistory(maxDoc, maxDoc); store.incRef(); - try (DirectoryReader reader = DirectoryReader.open(tempStore.directory())) { - IndexCommit indexCommit = reader.getIndexCommit(); - super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus); + try { + super.snapshotShard(mapperService, snapshotId, indexId, new Repository.ShardSnapshotContext() { + + private final AtomicBoolean closed = new AtomicBoolean(false); + private DirectoryReader reader; + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + synchronized (this) { + if (reader != null) { + reader.close(); + } + } + } + } + + @Override + public IndexCommit indexCommit() { + synchronized (this) { + if (closed.get()) { + throw new IllegalStateException("Tried to get index commit from closed context"); + } + if (reader == null) { + try { + reader = DirectoryReader.open(tempStore.directory()); + } catch (IOException e) { + completionListener().onFailure(e); + throw new UncheckedIOException(e); + } + } + try { + return reader.getIndexCommit(); + } catch (IOException e) { + completionListener().onFailure(e); + throw new UncheckedIOException(e); + } + } + } + + @Override + public Store store() { + return tempStore; + } + + @Override + public IndexShardSnapshotStatus status() { + return context.status(); + } + + @Override + public ActionListener completionListener() { + return context.completionListener(); + } + }); } finally { store.decRef(); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 948503b33478c..9e02ed96a5a77 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -19,6 +19,7 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.Bits; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.support.PlainActionFuture; @@ -69,6 +70,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; +@LuceneTestCase.AwaitsFix(bugUrl = "TODO") public class SourceOnlySnapshotShardTests extends IndexShardTestCase { public void testSourceIncomplete() throws IOException { @@ -96,12 +98,12 @@ public void testSourceIncomplete() throws IOException { repository.start(); try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> - runAsSnapshot(shard.getThreadPool(), - () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - snapshotRef.getIndexCommit(), indexShardSnapshotStatus))); - assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" - , illegalStateException.getMessage()); + //IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> + // runAsSnapshot(shard.getThreadPool(), + // () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + // snapshotRef.getIndexCommit(), indexShardSnapshotStatus))); + // assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" + // , illegalStateException.getMessage()); } closeShards(shard); } @@ -120,8 +122,8 @@ public void testIncrementalSnapshot() throws IOException { try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); SnapshotId snapshotId = new SnapshotId("test", "test"); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + // runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); totalFileCount = copy.getTotalFileCount(); @@ -134,8 +136,8 @@ public void testIncrementalSnapshot() throws IOException { SnapshotId snapshotId = new SnapshotId("test_1", "test_1"); IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + //runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt assertEquals(5, copy.getIncrementalFileCount()); @@ -148,8 +150,8 @@ public void testIncrementalSnapshot() throws IOException { SnapshotId snapshotId = new SnapshotId("test_2", "test_2"); IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + //runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv assertEquals(2, copy.getIncrementalFileCount()); @@ -197,8 +199,8 @@ public void testRestoreMinmal() throws IOException { repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), MetaData.builder().put(shard.indexSettings() .getIndexMetaData(), false).build()); - repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(), - indexShardSnapshotStatus); + //repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(), + // indexShardSnapshotStatus); }); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); From cdd132e914e9b809a358c91c6630c4978c73f8f2 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 10:00:22 +0200 Subject: [PATCH 04/23] fixed tests --- .../SourceOnlySnapshotRepository.java | 11 +- .../SourceOnlySnapshotShardTests.java | 136 +++++++++++++----- 2 files changed, 110 insertions(+), 37 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 9eb5ab6e2b436..fa7f462d9c28f 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -122,7 +122,9 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In Path dataPath = ((FSDirectory) unwrap).getDirectory().getParent(); // TODO should we have a snapshot tmp directory per shard that is maintained by the system? Path snapPath = dataPath.resolve(SNAPSHOT_DIR_NAME); - try (FSDirectory directory = new SimpleFSDirectory(snapPath)) { + FSDirectory directory = null; + try { + directory = new SimpleFSDirectory(snapPath); Store tempStore = new Store(store.shardId(), store.indexSettings(), directory, new ShardLock(store.shardId()) { @Override protected void closeInternal() { @@ -197,6 +199,13 @@ public ActionListener completionListener() { store.decRef(); } } catch (IOException e) { + if (directory != null) { + try { + directory.close(); + } catch (IOException ex) { + e.addSuppressed(ex); + } + } // why on earth does this super method not declare IOException throw new UncheckedIOException(e); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 9e02ed96a5a77..3951e016f74fc 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -7,6 +7,7 @@ import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -19,9 +20,9 @@ import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; import org.apache.lucene.util.Bits; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ExceptionsHelper; 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.MappingMetaData; @@ -57,6 +58,7 @@ import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -69,8 +71,8 @@ import java.util.Arrays; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicBoolean; -@LuceneTestCase.AwaitsFix(bugUrl = "TODO") public class SourceOnlySnapshotShardTests extends IndexShardTestCase { public void testSourceIncomplete() throws IOException { @@ -96,15 +98,17 @@ public void testSourceIncomplete() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { - IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - //IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> - // runAsSnapshot(shard.getThreadPool(), - // () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - // snapshotRef.getIndexCommit(), indexShardSnapshotStatus))); - // assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" - // , illegalStateException.getMessage()); - } + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> + runAsSnapshot(shard.getThreadPool(), + () -> { + final PlainActionFuture future = PlainActionFuture.newFuture(); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, + new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + future.actionGet(); + })); + assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" + , illegalStateException.getMessage()); closeShards(shard); } @@ -118,12 +122,14 @@ public void testIncrementalSnapshot() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - int totalFileCount = -1; - try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + int totalFileCount; + { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); SnapshotId snapshotId = new SnapshotId("test", "test"); - // runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + final PlainActionFuture future = PlainActionFuture.newFuture(); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, + new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); totalFileCount = copy.getTotalFileCount(); @@ -132,31 +138,34 @@ public void testIncrementalSnapshot() throws IOException { indexDoc(shard, "_doc", Integer.toString(10)); indexDoc(shard, "_doc", Integer.toString(11)); - try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + { SnapshotId snapshotId = new SnapshotId("test_1", "test_1"); - IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - //runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + final PlainActionFuture future = PlainActionFuture.newFuture(); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, + new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt assertEquals(5, copy.getIncrementalFileCount()); // in total we have 4 more files than the previous snap since we don't count the segments_N twice - assertEquals(totalFileCount+4, copy.getTotalFileCount()); + assertEquals(totalFileCount + 4, copy.getTotalFileCount()); assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); } deleteDoc(shard, "_doc", Integer.toString(10)); - try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + { SnapshotId snapshotId = new SnapshotId("test_2", "test_2"); IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - //runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, - // snapshotRef.getIndexCommit(), indexShardSnapshotStatus)); + final PlainActionFuture future = PlainActionFuture.newFuture(); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, + new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv assertEquals(2, copy.getIncrementalFileCount()); // in total we have 5 more files than the previous snap since we don't count the segments_N twice - assertEquals(totalFileCount+5, copy.getTotalFileCount()); + assertEquals(totalFileCount + 5, copy.getTotalFileCount()); assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); } closeShards(shard); @@ -193,19 +202,19 @@ public void testRestoreMinmal() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { - IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - runAsSnapshot(shard.getThreadPool(), () -> { - repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), - MetaData.builder().put(shard.indexSettings() + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + runAsSnapshot(shard.getThreadPool(), () -> { + repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), + MetaData.builder().put(shard.indexSettings() .getIndexMetaData(), false).build()); - //repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(), - // indexShardSnapshotStatus); - }); - IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); - assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); - assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); - } + final PlainActionFuture future = PlainActionFuture.newFuture(); + repository.snapshotShard(shard.mapperService(), snapshotId, indexId, + new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future)); + future.actionGet(); + }); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); shard.refresh("test"); ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, ShardRoutingState.INITIALIZING, @@ -362,4 +371,59 @@ private static T runAsSnapshot(ThreadPool pool, Callable runnable) { throw new AssertionError(e); } } + + private static final class TestSnapshotShardContext implements Repository.ShardSnapshotContext { + + private final AtomicBoolean closed; + private final IndexShard shard; + private final IndexShardSnapshotStatus indexShardSnapshotStatus; + private final PlainActionFuture future; + private Engine.IndexCommitRef snapshotRef; + + TestSnapshotShardContext(IndexShard shard, IndexShardSnapshotStatus indexShardSnapshotStatus, PlainActionFuture future) { + this.shard = shard; + this.indexShardSnapshotStatus = indexShardSnapshotStatus; + this.future = future; + closed = new AtomicBoolean(false); + } + + @Override + public void close() throws IOException { + if (closed.compareAndSet(false, true)) { + synchronized (this) { + if (snapshotRef != null) { + snapshotRef.close(); + } + } + } + } + + @Override + public IndexCommit indexCommit() { + synchronized (this) { + if (closed.get()) { + throw new IllegalStateException("Tried to get index commit from closed context"); + } + if (snapshotRef == null) { + snapshotRef = shard.acquireLastIndexCommit(true); + } + return snapshotRef.getIndexCommit(); + } + } + + @Override + public Store store() { + return shard.store(); + } + + @Override + public IndexShardSnapshotStatus status() { + return indexShardSnapshotStatus; + } + + @Override + public ActionListener completionListener() { + return future; + } + } } From 7b5c3259549869124976a9c063d94d9c9b7626cc Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 11:06:46 +0200 Subject: [PATCH 05/23] snapshot pool for setting up new shards is fine after all --- .../java/org/elasticsearch/snapshots/SnapshotShardsService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 9e158898ef079..e1435833cdcbc 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -297,7 +297,7 @@ private void startNewSnapshots(SnapshotsInProgress snapshotsInProgress) { private void startNewShards(SnapshotsInProgress.Entry entry, Map startedShards) { final Snapshot snapshot = entry.snapshot(); final Map indicesMap = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity())); - threadPool.executor(ThreadPool.Names.GENERIC).execute(() -> { + threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { for (final Map.Entry shardEntry : startedShards.entrySet()) { final ShardId shardId = shardEntry.getKey(); final IndexId indexId = indicesMap.get(shardId.getIndexName()); From f25c58861b2c26335a1bc62c07a13d4670811dee Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 11:39:01 +0200 Subject: [PATCH 06/23] drier --- .../repositories/Repository.java | 13 --- .../repositories/ShardSnapshotContext.java | 91 +++++++++++++++++ .../blobstore/BlobStoreRepository.java | 17 ++-- .../snapshots/SnapshotShardsService.java | 50 +--------- .../RepositoriesServiceTests.java | 2 +- .../repositories/fs/FsRepositoryTests.java | 99 +++++++++---------- .../index/shard/IndexShardTestCase.java | 48 ++------- .../index/shard/RestoreOnlyRepository.java | 3 +- .../xpack/ccr/repository/CcrRepository.java | 3 +- .../SourceOnlySnapshotRepository.java | 7 +- .../SourceOnlySnapshotShardTests.java | 5 +- 11 files changed, 168 insertions(+), 170 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index b4ecb37a8ec4b..5df2dd2f6a104 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.repositories; -import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -35,7 +34,6 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; -import java.io.Closeable; import java.io.IOException; import java.util.List; import java.util.function.Function; @@ -229,15 +227,4 @@ void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId i */ IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId); - - interface ShardSnapshotContext extends Closeable { - - IndexCommit indexCommit(); - - Store store(); - - IndexShardSnapshotStatus status(); - - ActionListener completionListener(); - } } diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java new file mode 100644 index 0000000000000..cf8fda30e171b --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -0,0 +1,91 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.repositories; + +import org.apache.lucene.index.IndexCommit; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +public interface ShardSnapshotContext { + + IndexCommit indexCommit(); + + Store store(); + + IndexShardSnapshotStatus status(); + + ActionListener completionListener(); + + void releaseIndexCommit() throws IOException; + + static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, + ActionListener listener) { + return new ShardSnapshotContext() { + + private final AtomicBoolean closed = new AtomicBoolean(false); + private Engine.IndexCommitRef snapshotRef; + + @Override + public void releaseIndexCommit() throws IOException { + if (closed.compareAndSet(false, true)) { + synchronized (this) { + if (snapshotRef != null) { + snapshotRef.close(); + } + } + } + } + + @Override + public IndexCommit indexCommit() { + synchronized (this) { + if (closed.get()) { + throw new IllegalStateException("Tried to get index commit from closed context"); + } + if (snapshotRef == null) { + snapshotRef = indexShard.acquireLastIndexCommit(true); + } + return snapshotRef.getIndexCommit(); + } + } + + @Override + public Store store() { + return indexShard.store(); + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return listener; + } + }; + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index be9dccbce08d3..4eff51d2113a3 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -88,6 +88,7 @@ import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.RepositoryVerificationException; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.InvalidSnapshotNameException; import org.elasticsearch.snapshots.SnapshotCreationException; import org.elasticsearch.snapshots.SnapshotException; @@ -799,7 +800,7 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b @Override public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - Repository.ShardSnapshotContext context) { + ShardSnapshotContext context) { final Store store = context.store(); final IndexShardSnapshotStatus status = context.status(); new SnapshotContext(store, snapshotId, indexId, status, threadPool.absoluteTimeInMillis()).snapshot( @@ -1101,7 +1102,7 @@ private class SnapshotContext extends Context { * * @param context shard snapshot context */ - public void snapshot(Repository.ShardSnapshotContext context, ActionListener listener) { + public void snapshot(ShardSnapshotContext context, ActionListener listener) { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); listBlobs(ActionListener.wrap( blobs -> { @@ -1147,7 +1148,7 @@ private void listBlobs(ActionListener> listener) { }); } - private void snapshotFiles(Repository.ShardSnapshotContext context, Map blobs, + private void snapshotFiles(ShardSnapshotContext context, Map blobs, BlobStoreIndexShardSnapshots snapshots, ActionListener listener) { try { @@ -1213,7 +1214,7 @@ private void snapshotFiles(Repository.ShardSnapshotContext context, Map ActionListener.completeWith(listener, () -> { - context.close(); + context.releaseIndexCommit(); final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); return new BlobStoreIndexShardSnapshot(snapshotId.getName(), @@ -1237,21 +1238,21 @@ private void snapshotFiles(Repository.ShardSnapshotContext context, Map completionListener() { - return listener; - } - }); + ShardSnapshotContext.create(indexShard, snapshotStatus, listener)); } /** diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index 4f4954be61f31..c5711aedf4378 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -200,7 +200,7 @@ public boolean isReadOnly() { @Override public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - Repository.ShardSnapshotContext context) { + ShardSnapshotContext context) { } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index 2148bf97cb162..e6d01a7a20585 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -55,7 +55,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; -import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.DummyShardLock; @@ -105,33 +105,7 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { final PlainActionFuture future1 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(null, snapshotId, indexId, new Repository.ShardSnapshotContext() { - - @Override - public void close() { - - } - - @Override - public IndexCommit indexCommit() { - return indexCommit; - } - - @Override - public Store store() { - return store; - } - - @Override - public IndexShardSnapshotStatus status() { - return snapshotStatus; - } - - @Override - public ActionListener completionListener() { - return future1; - } - }); + repository.snapshotShard(null, snapshotId, indexId, new TestSnapshotContext(indexCommit, store, snapshotStatus, future1)); future1.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -157,33 +131,8 @@ public ActionListener completionListener() { final PlainActionFuture future2 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(null, incSnapshotId, indexId, new Repository.ShardSnapshotContext() { - - @Override - public void close() { - - } - - @Override - public IndexCommit indexCommit() { - return incIndexCommit; - } - - @Override - public Store store() { - return store; - } - - @Override - public IndexShardSnapshotStatus status() { - return snapshotStatus; - } - - @Override - public ActionListener completionListener() { - return future2; - } - }); + repository.snapshotShard(null, incSnapshotId, indexId, + new TestSnapshotContext(incIndexCommit, store, snapshotStatus, future2)); future2.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(2, copy.getIncrementalFileCount()); @@ -256,4 +205,44 @@ private int indexDocs(Directory directory) throws IOException { return docs; } } + + private static class TestSnapshotContext implements ShardSnapshotContext { + + private final IndexCommit indexCommit; + private final Store store; + private final IndexShardSnapshotStatus snapshotStatus; + private final ActionListener listener; + + TestSnapshotContext(IndexCommit indexCommit, Store store, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { + this.indexCommit = indexCommit; + this.store = store; + this.snapshotStatus = snapshotStatus; + this.listener = listener; + } + + @Override + public void releaseIndexCommit() { + + } + + @Override + public IndexCommit indexCommit() { + return indexCommit; + } + + @Override + public Store store() { + return store; + } + + @Override + public IndexShardSnapshotStatus status() { + return snapshotStatus; + } + + @Override + public ActionListener completionListener() { + return listener; + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 943c82bea7c0f..bb1a21804a18f 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -18,11 +18,9 @@ */ package org.elasticsearch.index.shard; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexNotFoundException; import org.apache.lucene.store.Directory; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; @@ -79,6 +77,7 @@ import org.elasticsearch.indices.recovery.StartRecoveryRequest; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; @@ -817,43 +816,16 @@ protected void recoverShardFromSnapshot(final IndexShard shard, } /** Snapshot a shard using a given repository **/ - protected void snapshotShard(final IndexShard shard, - final Snapshot snapshot, - final Repository repository) throws IOException { + protected static void snapshotShard(final IndexShard shard, + final Snapshot snapshot, + final Repository repository) throws IOException { final IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) { - Index index = shard.shardId().getIndex(); - IndexId indexId = new IndexId(index.getName(), index.getUUID()); - final PlainActionFuture future = PlainActionFuture.newFuture(); - final IndexCommit indexCommit = indexCommitRef.getIndexCommit(); - repository.snapshotShard(shard.mapperService(), snapshot.getSnapshotId(), indexId, - new Repository.ShardSnapshotContext() { - @Override - public IndexCommit indexCommit() { - return indexCommit; - } - - @Override - public Store store() { - return shard.store(); - } - - @Override - public IndexShardSnapshotStatus status() { - return snapshotStatus; - } - - @Override - public ActionListener completionListener() { - return future; - } - - @Override - public void close() { - } - }); - future.actionGet(); - } + Index index = shard.shardId().getIndex(); + IndexId indexId = new IndexId(index.getName(), index.getUUID()); + final PlainActionFuture future = PlainActionFuture.newFuture(); + repository.snapshotShard(shard.mapperService(), snapshot.getSnapshotId(), indexId, + ShardSnapshotContext.create(shard, snapshotStatus, future)); + future.actionGet(); final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); assertEquals(IndexShardSnapshotStatus.Stage.DONE, lastSnapshotStatus.getStage()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index 1bbd3f70b96bd..a819b3fed7c1b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -30,6 +30,7 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; @@ -133,7 +134,7 @@ public boolean isReadOnly() { @Override public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - Repository.ShardSnapshotContext context) { + ShardSnapshotContext context) { } @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 7fb2af6bb4858..77b9c8c2de95d 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -58,6 +58,7 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.repositories.blobstore.FileRestoreContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -294,7 +295,7 @@ public boolean isReadOnly() { @Override public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - Repository.ShardSnapshotContext context) { + ShardSnapshotContext context) { throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index fa7f462d9c28f..712374b5138b5 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -34,6 +34,7 @@ import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.ShardSnapshotContext; import java.io.IOException; import java.io.UncheckedIOException; @@ -108,7 +109,7 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met @Override public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - Repository.ShardSnapshotContext context) { + ShardSnapshotContext context) { if (mapperService.documentMapper() != null // if there is no mapping this is null && mapperService.documentMapper().sourceMapper().isComplete() == false) { throw new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " + @@ -141,13 +142,13 @@ protected void closeInternal() { tempStore.bootstrapNewHistory(maxDoc, maxDoc); store.incRef(); try { - super.snapshotShard(mapperService, snapshotId, indexId, new Repository.ShardSnapshotContext() { + super.snapshotShard(mapperService, snapshotId, indexId, new ShardSnapshotContext() { private final AtomicBoolean closed = new AtomicBoolean(false); private DirectoryReader reader; @Override - public void close() throws IOException { + public void releaseIndexCommit() throws IOException { if (closed.compareAndSet(false, true)) { synchronized (this) { if (reader != null) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 3951e016f74fc..06439bc42e850 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -62,6 +62,7 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matchers; @@ -372,7 +373,7 @@ private static T runAsSnapshot(ThreadPool pool, Callable runnable) { } } - private static final class TestSnapshotShardContext implements Repository.ShardSnapshotContext { + private static final class TestSnapshotShardContext implements ShardSnapshotContext { private final AtomicBoolean closed; private final IndexShard shard; @@ -388,7 +389,7 @@ private static final class TestSnapshotShardContext implements Repository.ShardS } @Override - public void close() throws IOException { + public void releaseIndexCommit() throws IOException { if (closed.compareAndSet(false, true)) { synchronized (this) { if (snapshotRef != null) { From d22272607aabf38f07fdc9d146e3ae72cb638e3b Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 12:19:35 +0200 Subject: [PATCH 07/23] drier --- .../SourceOnlySnapshotShardTests.java | 68 ++----------------- 1 file changed, 5 insertions(+), 63 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 06439bc42e850..50ce78833f30f 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -7,7 +7,6 @@ import org.apache.lucene.document.Document; import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.LeafReaderContext; @@ -22,7 +21,6 @@ import org.apache.lucene.util.Bits; import org.elasticsearch.ExceptionsHelper; 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.MappingMetaData; @@ -58,7 +56,6 @@ import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; -import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; @@ -72,7 +69,6 @@ import java.util.Arrays; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicBoolean; public class SourceOnlySnapshotShardTests extends IndexShardTestCase { @@ -105,7 +101,7 @@ public void testSourceIncomplete() throws IOException { () -> { final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); future.actionGet(); })); assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" @@ -129,7 +125,7 @@ public void testIncrementalSnapshot() throws IOException { SnapshotId snapshotId = new SnapshotId("test", "test"); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -144,7 +140,7 @@ public void testIncrementalSnapshot() throws IOException { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt @@ -160,7 +156,7 @@ public void testIncrementalSnapshot() throws IOException { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future))); + ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv @@ -210,7 +206,7 @@ public void testRestoreMinmal() throws IOException { .getIndexMetaData(), false).build()); final PlainActionFuture future = PlainActionFuture.newFuture(); repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new TestSnapshotShardContext(shard, indexShardSnapshotStatus, future)); + ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future)); future.actionGet(); }); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); @@ -373,58 +369,4 @@ private static T runAsSnapshot(ThreadPool pool, Callable runnable) { } } - private static final class TestSnapshotShardContext implements ShardSnapshotContext { - - private final AtomicBoolean closed; - private final IndexShard shard; - private final IndexShardSnapshotStatus indexShardSnapshotStatus; - private final PlainActionFuture future; - private Engine.IndexCommitRef snapshotRef; - - TestSnapshotShardContext(IndexShard shard, IndexShardSnapshotStatus indexShardSnapshotStatus, PlainActionFuture future) { - this.shard = shard; - this.indexShardSnapshotStatus = indexShardSnapshotStatus; - this.future = future; - closed = new AtomicBoolean(false); - } - - @Override - public void releaseIndexCommit() throws IOException { - if (closed.compareAndSet(false, true)) { - synchronized (this) { - if (snapshotRef != null) { - snapshotRef.close(); - } - } - } - } - - @Override - public IndexCommit indexCommit() { - synchronized (this) { - if (closed.get()) { - throw new IllegalStateException("Tried to get index commit from closed context"); - } - if (snapshotRef == null) { - snapshotRef = shard.acquireLastIndexCommit(true); - } - return snapshotRef.getIndexCommit(); - } - } - - @Override - public Store store() { - return shard.store(); - } - - @Override - public IndexShardSnapshotStatus status() { - return indexShardSnapshotStatus; - } - - @Override - public ActionListener completionListener() { - return future; - } - } } From 01ce0ac9182933121a8447153a9030180fa73972 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 12:37:29 +0200 Subject: [PATCH 08/23] drier --- .../repositories/ShardSnapshotContext.java | 51 ++++++++++--------- .../repositories/fs/FsRepositoryTests.java | 24 +-------- .../SourceOnlySnapshotRepository.java | 20 +------- 3 files changed, 31 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index cf8fda30e171b..49ad287803610 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -29,21 +29,39 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; -public interface ShardSnapshotContext { +public abstract class ShardSnapshotContext { - IndexCommit indexCommit(); + private final Store store; - Store store(); + private final ActionListener listener; - IndexShardSnapshotStatus status(); + private final IndexShardSnapshotStatus status; - ActionListener completionListener(); + public ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status) { + this.store = store; + this.listener = listener; + this.status = status; + } - void releaseIndexCommit() throws IOException; + public abstract IndexCommit indexCommit(); - static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, - ActionListener listener) { - return new ShardSnapshotContext() { + public Store store() { + return store; + } + + public IndexShardSnapshotStatus status() { + return status; + } + + public ActionListener completionListener() { + return listener; + } + + public abstract void releaseIndexCommit() throws IOException; + + public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, + ActionListener listener) { + return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus) { private final AtomicBoolean closed = new AtomicBoolean(false); private Engine.IndexCommitRef snapshotRef; @@ -71,21 +89,6 @@ public IndexCommit indexCommit() { return snapshotRef.getIndexCommit(); } } - - @Override - public Store store() { - return indexShard.store(); - } - - @Override - public IndexShardSnapshotStatus status() { - return snapshotStatus; - } - - @Override - public ActionListener completionListener() { - return listener; - } }; } } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index e6d01a7a20585..ecf690c4942f2 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -206,18 +206,13 @@ private int indexDocs(Directory directory) throws IOException { } } - private static class TestSnapshotContext implements ShardSnapshotContext { + private static class TestSnapshotContext extends ShardSnapshotContext { private final IndexCommit indexCommit; - private final Store store; - private final IndexShardSnapshotStatus snapshotStatus; - private final ActionListener listener; TestSnapshotContext(IndexCommit indexCommit, Store store, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { + super(store, listener, snapshotStatus); this.indexCommit = indexCommit; - this.store = store; - this.snapshotStatus = snapshotStatus; - this.listener = listener; } @Override @@ -229,20 +224,5 @@ public void releaseIndexCommit() { public IndexCommit indexCommit() { return indexCommit; } - - @Override - public Store store() { - return store; - } - - @Override - public IndexShardSnapshotStatus status() { - return snapshotStatus; - } - - @Override - public ActionListener completionListener() { - return listener; - } } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 712374b5138b5..9eebc3312a5a0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -14,7 +14,6 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.SimpleFSDirectory; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -28,7 +27,6 @@ import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.repositories.FilterRepository; @@ -142,7 +140,8 @@ protected void closeInternal() { tempStore.bootstrapNewHistory(maxDoc, maxDoc); store.incRef(); try { - super.snapshotShard(mapperService, snapshotId, indexId, new ShardSnapshotContext() { + super.snapshotShard(mapperService, snapshotId, indexId, new ShardSnapshotContext(tempStore, context.completionListener(), + context.status()) { private final AtomicBoolean closed = new AtomicBoolean(false); private DirectoryReader reader; @@ -180,21 +179,6 @@ public IndexCommit indexCommit() { } } } - - @Override - public Store store() { - return tempStore; - } - - @Override - public IndexShardSnapshotStatus status() { - return context.status(); - } - - @Override - public ActionListener completionListener() { - return context.completionListener(); - } }); } finally { store.decRef(); From a43d2dcf5e1b9e963e29a3bc7b527da26ff8dff2 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 13:30:05 +0200 Subject: [PATCH 09/23] drier --- .../repositories/ShardSnapshotContext.java | 27 +++++-- .../blobstore/BlobStoreRepository.java | 32 +------- .../SourceOnlySnapshotRepository.java | 74 ++++++++----------- 3 files changed, 57 insertions(+), 76 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index 49ad287803610..55185f4ffe3e6 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -37,18 +37,35 @@ public abstract class ShardSnapshotContext { private final IndexShardSnapshotStatus status; - public ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status) { + protected ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status) { this.store = store; this.listener = listener; this.status = status; } - public abstract IndexCommit indexCommit(); + public abstract IndexCommit indexCommit() throws IOException; + + public abstract void releaseIndexCommit() throws IOException; public Store store() { return store; } + public void finish(long endTime) { + status.moveToDone(endTime); + listener.onResponse(null); + } + + public void finish(long endTime, String failureMessage, Exception e) { + status.moveToFailed(endTime, failureMessage); + try { + releaseIndexCommit(); + } catch (Exception ex) { + e.addSuppressed(ex); + } + listener.onFailure(e); + } + public IndexShardSnapshotStatus status() { return status; } @@ -57,8 +74,6 @@ public ActionListener completionListener() { return listener; } - public abstract void releaseIndexCommit() throws IOException; - public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus) { @@ -81,13 +96,13 @@ public void releaseIndexCommit() throws IOException { public IndexCommit indexCommit() { synchronized (this) { if (closed.get()) { - throw new IllegalStateException("Tried to get index commit from closed context"); + throw new IllegalStateException("Tried to get index commit from closed context."); } if (snapshotRef == null) { snapshotRef = indexShard.acquireLastIndexCommit(true); } - return snapshotRef.getIndexCommit(); } + return snapshotRef.getIndexCommit(); } }; } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 4eff51d2113a3..1a67b71ecd830 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -808,20 +808,14 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In new ActionListener<>() { @Override public void onResponse(IndexShardSnapshotStatus indexShardSnapshotStatus) { - status.moveToDone(threadPool.absoluteTimeInMillis()); - context.completionListener().onResponse(null); + context.finish(threadPool.absoluteTimeInMillis()); } @Override public void onFailure(Exception e) { - status.moveToFailed(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e)); - final IndexShardSnapshotFailedException ex; - if (e instanceof IndexShardSnapshotFailedException) { - ex = (IndexShardSnapshotFailedException) e; - } else { - ex = new IndexShardSnapshotFailedException(store.shardId(), e); - } - context.completionListener().onFailure(ex); + context.finish(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e), + e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e + : new IndexShardSnapshotFailedException(store.shardId(), e)); } }); } @@ -1238,25 +1232,12 @@ private void snapshotFiles(ShardSnapshotContext context, Map Date: Mon, 3 Jun 2019 14:00:49 +0200 Subject: [PATCH 10/23] simpler --- .../repositories/ShardSnapshotContext.java | 6 + .../blobstore/BlobStoreRepository.java | 186 +++++++++--------- 2 files changed, 96 insertions(+), 96 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index 55185f4ffe3e6..2920de6677cdb 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -56,6 +56,12 @@ public void finish(long endTime) { listener.onResponse(null); } + public IndexShardSnapshotStatus.Copy prepareFinalize() throws IOException { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = status.moveToFinalize(indexCommit().getGeneration()); + releaseIndexCommit(); + return lastSnapshotStatus; + } + public void finish(long endTime, String failureMessage, Exception e) { status.moveToFailed(endTime, failureMessage); try { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 1a67b71ecd830..f9231f169f0f7 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -807,7 +807,7 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In context, new ActionListener<>() { @Override - public void onResponse(IndexShardSnapshotStatus indexShardSnapshotStatus) { + public void onResponse(Void aVoid) { context.finish(threadPool.absoluteTimeInMillis()); } @@ -1096,7 +1096,7 @@ private class SnapshotContext extends Context { * * @param context shard snapshot context */ - public void snapshot(ShardSnapshotContext context, ActionListener listener) { + public void snapshot(ShardSnapshotContext context, ActionListener listener) { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); listBlobs(ActionListener.wrap( blobs -> { @@ -1126,7 +1126,7 @@ public void snapshot(ShardSnapshotContext context, ActionListener> listener) { private void snapshotFiles(ShardSnapshotContext context, Map blobs, BlobStoreIndexShardSnapshots snapshots, - ActionListener listener) { + ActionListener listener) throws IOException { + // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should + final IndexCommit snapshotIndexCommit = context.indexCommit(); + store.incRef(); + final Store.MetadataSnapshot meta; + final Collection fileNames; try { - // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should - store.incRef(); - final IndexCommit snapshotIndexCommit = context.indexCommit(); - final Collection fileNames; - final Store.MetadataSnapshot meta; - try { - logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); - meta = store.getMetadata(snapshotIndexCommit); - fileNames = snapshotIndexCommit.getFileNames(); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); - } finally { - store.decRef(); + logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); + meta = store.getMetadata(snapshotIndexCommit); + fileNames = snapshotIndexCommit.getFileNames(); + } catch (IOException e) { + throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); + } finally { + store.decRef(); + } + int indexIncrementalFileCount = 0; + int indexTotalNumberOfFiles = 0; + long indexIncrementalSize = 0; + long indexTotalFileCount = 0; + final List filesToSnapshot = new ArrayList<>(); + final List indexCommitPointFiles = new ArrayList<>(); + for (String fileName : fileNames) { + if (snapshotStatus.isAborted()) { + logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); + throw new IndexShardSnapshotFailedException(shardId, "Aborted"); } - int indexIncrementalFileCount = 0; - int indexTotalNumberOfFiles = 0; - long indexIncrementalSize = 0; - long indexTotalFileCount = 0; - final List filesToSnapshot = new ArrayList<>(); - final List indexCommitPointFiles = new ArrayList<>(); - 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 = meta.get(fileName); - BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null; - List filesInfo = snapshots.findPhysicalIndexFiles(fileName); - if (filesInfo != null) { - for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { - if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { - // 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 = meta.get(fileName); + BlobStoreIndexShardSnapshot.FileInfo existingFileInfo = null; + List filesInfo = snapshots.findPhysicalIndexFiles(fileName); + if (filesInfo != null) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { + if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { + // 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); - } + 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); - - assert indexIncrementalFileCount == filesToSnapshot.size(); - final Runnable afterSegmentFiles = () -> ActionListener.completeWith(listener, () -> { - context.releaseIndexCommit(); - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = - snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); - return new BlobStoreIndexShardSnapshot(snapshotId.getName(), - lastSnapshotStatus.getIndexVersion(), - indexCommitPointFiles, - lastSnapshotStatus.getStartTime(), - // snapshotStatus.startTime() is assigned on the same machine, - // so it's safe to use with VLong - System.currentTimeMillis() - lastSnapshotStatus.getStartTime(), - lastSnapshotStatus.getIncrementalFileCount(), - lastSnapshotStatus.getIncrementalSize() - ); - }); - if (indexIncrementalFileCount > 0) { - final GroupedActionListener fileCompletionListener = new GroupedActionListener<>( - ActionListener.wrap(v -> afterSegmentFiles.run(), listener::onFailure), indexIncrementalFileCount); - final ExecutorService executorService = threadPool.executor(ThreadPool.Names.SNAPSHOT); - for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { - executorService.execute(new ActionRunnable<>(fileCompletionListener) { - @Override - protected void doRun() { - try { - if (snapshotStatus.isAborted()) { - throw new IndexShardSnapshotFailedException(shardId, "Aborted"); - } - snapshotFile(snapshotFileInfo); - fileCompletionListener.onResponse(null); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); - } + snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount, + indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount); + + assert indexIncrementalFileCount == filesToSnapshot.size(); + final Runnable afterSegmentFiles = () -> ActionListener.completeWith(listener, () -> { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = context.prepareFinalize(); + return new BlobStoreIndexShardSnapshot(snapshotId.getName(), + lastSnapshotStatus.getIndexVersion(), + indexCommitPointFiles, + lastSnapshotStatus.getStartTime(), + // snapshotStatus.startTime() is assigned on the same machine, + // so it's safe to use with VLong + threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(), + lastSnapshotStatus.getIncrementalFileCount(), + lastSnapshotStatus.getIncrementalSize() + ); + }); + if (indexIncrementalFileCount > 0) { + final GroupedActionListener fileCompletionListener = new GroupedActionListener<>( + ActionListener.wrap(v -> afterSegmentFiles.run(), listener::onFailure), indexIncrementalFileCount); + final ExecutorService executorService = threadPool.executor(ThreadPool.Names.SNAPSHOT); + for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { + executorService.execute(new ActionRunnable<>(fileCompletionListener) { + @Override + protected void doRun() { + if (snapshotStatus.isAborted()) { + throw new IndexShardSnapshotFailedException(shardId, "Aborted"); } - }); - } - } else { - afterSegmentFiles.run(); + try { + snapshotFile(snapshotFileInfo); + } catch (IOException e) { + throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); + } + fileCompletionListener.onResponse(null); + } + }); } - } catch (Exception e) { - listener.onFailure(e); + } else { + afterSegmentFiles.run(); } } From 8c64b96c1569473d4ffc42f4462f608c8bddcc23 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 14:17:55 +0200 Subject: [PATCH 11/23] safer looking --- .../org/elasticsearch/snapshots/SnapshotShardsService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index bf4fed85fab40..7a335eebf815d 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -297,8 +297,6 @@ private void startNewShards(SnapshotsInProgress.Entry entry, Map { for (final Map.Entry shardEntry : startedShards.entrySet()) { final ShardId shardId = shardEntry.getKey(); - final IndexId indexId = indicesMap.get(shardId.getIndexName()); - assert indexId != null; final IndexShardSnapshotStatus snapshotStatus = shardEntry.getValue(); final ActionListener listener = new ActionListener<>() { @Override @@ -317,6 +315,8 @@ public void onFailure(Exception e) { } }; try { + final IndexId indexId = indicesMap.get(shardId.getIndexName()); + assert indexId != null; snapshot(indicesService.indexServiceSafe( shardId.getIndex()).getShardOrNull(shardId.id()), snapshot, indexId, snapshotStatus, listener); } catch (Exception e) { From 84586f19df2b3b29cfde28cf43a025b170d2a3a1 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 3 Jun 2019 14:49:54 +0200 Subject: [PATCH 12/23] some docs --- .../repositories/ShardSnapshotContext.java | 37 +++++++++++++++++-- .../repositories/fs/FsRepositoryTests.java | 2 +- .../SourceOnlySnapshotRepository.java | 3 +- 3 files changed, 35 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index 2920de6677cdb..84b381f5f2772 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -29,6 +29,9 @@ import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; +/** + * Manages the {@link IndexCommit} associated with the shard snapshot as well as its {@link IndexShardSnapshotStatus} instance. + */ public abstract class ShardSnapshotContext { private final Store store; @@ -43,26 +46,52 @@ protected ShardSnapshotContext(Store store, ActionListener listener, Index this.status = status; } + /** + * Create and return an {@link IndexCommit} for this shard. Repeated invocations of this method return the same {@link IndexCommit}. + * The resources associated with this {@link IndexCommit} are released by {@link #releaseIndexCommit()} when either + * {@link #finish(long, String, Exception)} or {@link #prepareFinalize()} is invoked. + * @return IndexCommit index commit + * @throws IOException on failure + */ public abstract IndexCommit indexCommit() throws IOException; - public abstract void releaseIndexCommit() throws IOException; + /** + * Release resources backing the {@link IndexCommit} returned by {@link #indexCommit()}. + * @throws IOException on failure + */ + protected abstract void releaseIndexCommit() throws IOException; public Store store() { return store; } + /** + * Invoke once all writes to the repository have finished for the shard. + * @param endTime Timestamp of when the shard snapshot's writes to the repository finished + */ public void finish(long endTime) { status.moveToDone(endTime); listener.onResponse(null); } - public IndexShardSnapshotStatus.Copy prepareFinalize() throws IOException { + /** + * Invoke once all segments for this shard were written to the repository. + * @return IndexSnapshotStatus right after writing all segments to the repository + * @throws IOException On failure to release the resources backing this instance's {@link IndexCommit} + */ + public final IndexShardSnapshotStatus.Copy prepareFinalize() throws IOException { final IndexShardSnapshotStatus.Copy lastSnapshotStatus = status.moveToFinalize(indexCommit().getGeneration()); releaseIndexCommit(); return lastSnapshotStatus; } - public void finish(long endTime, String failureMessage, Exception e) { + /** + * Invoke in case the shard's snapshot operation failed. + * @param endTime time the shard's snapshot failed + * @param failureMessage failure message + * @param e Exception that caused the shard's snapshot to fail + */ + public final void finish(long endTime, String failureMessage, Exception e) { status.moveToFailed(endTime, failureMessage); try { releaseIndexCommit(); @@ -88,7 +117,7 @@ public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnaps private Engine.IndexCommitRef snapshotRef; @Override - public void releaseIndexCommit() throws IOException { + protected void releaseIndexCommit() throws IOException { if (closed.compareAndSet(false, true)) { synchronized (this) { if (snapshotRef != null) { diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index ecf690c4942f2..ba53bfe0c8794 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -216,7 +216,7 @@ private static class TestSnapshotContext extends ShardSnapshotContext { } @Override - public void releaseIndexCommit() { + protected void releaseIndexCommit() { } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index fe67e2e25b9d4..d094da422a985 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -146,7 +146,7 @@ protected void closeInternal() { private IndexCommit indexCommit; @Override - public void releaseIndexCommit() throws IOException { + protected void releaseIndexCommit() throws IOException { if (closed.compareAndSet(false, true)) { synchronized (this) { if (reader != null) { @@ -167,7 +167,6 @@ public IndexCommit indexCommit() throws IOException { assert indexCommit == null; indexCommit = reader.getIndexCommit(); } - assert indexCommit != null; return indexCommit; } From 396b3ae18d5f390a9d52895700e9b6258a17b278 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 24 Jun 2019 11:23:50 +0200 Subject: [PATCH 13/23] finished merge --- .../blobstore/BlobStoreRepository.java | 185 ++++++++++-------- 1 file changed, 100 insertions(+), 85 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index c0752fc5ec359..0baff968f1c86 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -107,6 +107,8 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.Executor; +import java.util.function.Consumer; import java.util.stream.Collectors; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; @@ -819,6 +821,9 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In final ShardId shardId = store.shardId(); final IndexShardSnapshotStatus snapshotStatus = context.status(); final long startTime = threadPool.relativeTimeInMillis(); + final Consumer onFailure = e -> context.finish(threadPool.relativeTimeInMillis(), ExceptionsHelper.detailedMessage(e), + e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e + : new IndexShardSnapshotFailedException(store.shardId(), e)); try { logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, metadata.name()); @@ -839,114 +844,124 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting"); } - final List indexCommitPointFiles = new ArrayList<>(); store.incRef(); final IndexCommit snapshotIndexCommit = context.indexCommit(); + final Collection fileNames; + final Store.MetadataSnapshot metadataFromStore; try { - ArrayList filesToSnapshot = new ArrayList<>(); - final Store.MetadataSnapshot metadata; // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should - final Collection 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; + final List indexCommitPointFiles = new ArrayList<>(); + final List filesToSnapshot = new ArrayList<>(); + 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 filesInfo = snapshots.findPhysicalIndexFiles(fileName); - if (filesInfo != null) { - for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { - if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { - // 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 filesInfo = snapshots.findPhysicalIndexFiles(fileName); + if (filesInfo != null) { + for (BlobStoreIndexShardSnapshot.FileInfo fileInfo : filesInfo) { + if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { + // 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); + 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); + } + } - for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { + snapshotStatus.moveToStarted(startTime, indexIncrementalFileCount, + indexTotalNumberOfFiles, indexIncrementalSize, indexTotalFileCount); + + assert indexIncrementalFileCount == filesToSnapshot.size(); + + final GroupedActionListener filesListener = new GroupedActionListener<>( + ActionListener.wrap(v -> { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = context.prepareFinalize(); + + // now create and write the commit point + final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(), + lastSnapshotStatus.getIndexVersion(), + indexCommitPointFiles, + lastSnapshotStatus.getStartTime(), + // snapshotStatus.startTime() is assigned on the same machine, + // so it's safe to use the relative time in millis + threadPool.relativeTimeInMillis() - lastSnapshotStatus.getStartTime(), + lastSnapshotStatus.getIncrementalFileCount(), + lastSnapshotStatus.getIncrementalSize() + ); + + //TODO: The time stored in snapshot doesn't include cleanup time. + logger.trace("[{}] [{}] writing shard snapshot file", shardId, snapshotId); try { - snapshotFile(snapshotFileInfo, indexId, shardId, snapshotId, snapshotStatus, store); + indexShardSnapshotFormat.write(snapshot, shardContainer, snapshotId.getUUID()); } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to perform snapshot (index files)", e); + throw new IndexShardSnapshotFailedException(shardId, "Failed to write commit point", e); } - } - } finally { - store.decRef(); - } - - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); - - // now create and write the commit point - final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(), - lastSnapshotStatus.getIndexVersion(), - indexCommitPointFiles, - lastSnapshotStatus.getStartTime(), - // snapshotStatus.startTime() is assigned on the same machine, - // so it's safe to use the relative time in millis - threadPool.relativeTimeInMillis() - lastSnapshotStatus.getStartTime(), - lastSnapshotStatus.getIncrementalFileCount(), - lastSnapshotStatus.getIncrementalSize() - ); - - //TODO: The time stored in snapshot doesn't include cleanup time. - 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 newSnapshotsList = new ArrayList<>(); - newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); - for (SnapshotFiles point : snapshots) { - newSnapshotsList.add(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 + List newSnapshotsList = new ArrayList<>(); + newSnapshotsList.add(new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles())); + for (SnapshotFiles point : snapshots) { + newSnapshotsList.add(point); + } + // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index + finalizeShard(newSnapshotsList, fileListGeneration, blobs, "snapshot creation [" + snapshotId + "]", + shardContainer, shardId, snapshotId); + context.finish(threadPool.relativeTimeInMillis()); + }, onFailure), indexIncrementalFileCount + 1); + filesListener.onResponse(null); + 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); + } + } + }); } - // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index - finalizeShard(newSnapshotsList, fileListGeneration, blobs, "snapshot creation [" + snapshotId + "]", shardContainer, - shardId, snapshotId); - context.finish(threadPool.relativeTimeInMillis()); } catch (Exception e) { - context.finish(threadPool.relativeTimeInMillis(), ExceptionsHelper.detailedMessage(e), - e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e - : new IndexShardSnapshotFailedException(store.shardId(), e)); + onFailure.accept(e); } } From 2099b11cc298573c16f519142844fe5f4f5dab29 Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 24 Jun 2019 13:21:20 +0200 Subject: [PATCH 14/23] bcl --- .../repositories/ShardSnapshotContext.java | 56 +++++++++++++------ .../repositories/fs/FsRepositoryTests.java | 4 +- .../SourceOnlySnapshotRepository.java | 54 +++++++----------- 3 files changed, 61 insertions(+), 53 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index 84b381f5f2772..dfc5bbd9b1a4b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -26,8 +26,10 @@ import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; +import java.io.Closeable; import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; /** * Manages the {@link IndexCommit} associated with the shard snapshot as well as its {@link IndexShardSnapshotStatus} instance. @@ -40,6 +42,8 @@ public abstract class ShardSnapshotContext { private final IndexShardSnapshotStatus status; + private final AtomicBoolean closed = new AtomicBoolean(false); + protected ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status) { this.store = store; this.listener = listener; @@ -53,13 +57,36 @@ protected ShardSnapshotContext(Store store, ActionListener listener, Index * @return IndexCommit index commit * @throws IOException on failure */ - public abstract IndexCommit indexCommit() throws IOException; + public IndexCommit indexCommit() throws IOException { + synchronized (this) { + if (closed.get()) { + throw new IllegalStateException("Tried to get index commit from closed context."); + } + return doIndexCommit(); + } + } + + /** + * See {@link #indexCommit()} + */ + protected abstract IndexCommit doIndexCommit() throws IOException; /** * Release resources backing the {@link IndexCommit} returned by {@link #indexCommit()}. * @throws IOException on failure */ - protected abstract void releaseIndexCommit() throws IOException; + private void releaseIndexCommit() throws IOException { + if (closed.compareAndSet(false, true)) { + synchronized (this) { + doReleaseIndexCommit(); + } + } + } + + /** + * See {@link #releaseIndexCommit()} + */ + protected abstract void doReleaseIndexCommit() throws IOException; public Store store() { return store; @@ -109,33 +136,26 @@ public ActionListener completionListener() { return listener; } + public interface IndexCommitProvider extends Closeable, Supplier { + } + public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus) { - private final AtomicBoolean closed = new AtomicBoolean(false); private Engine.IndexCommitRef snapshotRef; @Override - protected void releaseIndexCommit() throws IOException { - if (closed.compareAndSet(false, true)) { - synchronized (this) { - if (snapshotRef != null) { - snapshotRef.close(); - } - } + protected void doReleaseIndexCommit() throws IOException { + if (snapshotRef != null) { + snapshotRef.close(); } } @Override - public IndexCommit indexCommit() { - synchronized (this) { - if (closed.get()) { - throw new IllegalStateException("Tried to get index commit from closed context."); - } - if (snapshotRef == null) { - snapshotRef = indexShard.acquireLastIndexCommit(true); - } + protected IndexCommit doIndexCommit() { + if (snapshotRef == null) { + snapshotRef = indexShard.acquireLastIndexCommit(true); } return snapshotRef.getIndexCommit(); } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index 5b6f49a4f67d1..b9bb1cf3b10b3 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -216,12 +216,12 @@ private static class TestSnapshotContext extends ShardSnapshotContext { } @Override - protected void releaseIndexCommit() { + protected void doReleaseIndexCommit() { } @Override - public IndexCommit indexCommit() { + public IndexCommit doIndexCommit() { return indexCommit; } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 9483fce7c825b..0dca1ec14e7a0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -41,7 +41,6 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; import java.util.function.Supplier; @@ -141,40 +140,29 @@ protected void closeInternal() { final long maxDoc = segmentInfos.totalMaxDoc(); tempStore.bootstrapNewHistory(maxDoc, maxDoc); super.snapshotShard(mapperService, snapshotId, indexId, - new ShardSnapshotContext(tempStore, context.completionListener(), context.status()) { + new ShardSnapshotContext(tempStore, context.completionListener(), context.status()) { + private DirectoryReader reader; + private IndexCommit indexCommit; - private final AtomicBoolean closed = new AtomicBoolean(false); - private DirectoryReader reader; - private IndexCommit indexCommit; + @Override + protected void doReleaseIndexCommit() throws IOException { + if (reader != null) { + reader.close(); + } + } - @Override - protected void releaseIndexCommit() throws IOException { - if (closed.compareAndSet(false, true)) { - synchronized (this) { - if (reader != null) { - reader.close(); - } - } - } - } - - @Override - public IndexCommit indexCommit() throws IOException { - synchronized (this) { - if (closed.get()) { - throw new IllegalStateException("Tried to get index commit from closed context"); - } - if (reader == null) { - reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap( - BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); - assert indexCommit == null; - indexCommit = reader.getIndexCommit(); - } - assert indexCommit != null; - return indexCommit; - } - } - }); + @Override + protected IndexCommit doIndexCommit() throws IOException { + if (reader == null) { + reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap( + BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); + assert indexCommit == null; + indexCommit = reader.getIndexCommit(); + } + assert indexCommit != null; + return indexCommit; + } + }); } catch (IOException e) { if (directory != null) { try { From dcf374f48397160787618851579982a1f01d589c Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 24 Jun 2019 14:05:29 +0200 Subject: [PATCH 15/23] simplify --- .../repositories/ShardSnapshotContext.java | 41 +++++++++---------- .../repositories/fs/FsRepositoryTests.java | 32 ++++++--------- .../SourceOnlySnapshotRepository.java | 36 ++++++++-------- 3 files changed, 50 insertions(+), 59 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index dfc5bbd9b1a4b..9a13912970cf8 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -20,7 +20,9 @@ package org.elasticsearch.repositories; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -29,12 +31,11 @@ import java.io.Closeable; import java.io.IOException; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Supplier; /** * Manages the {@link IndexCommit} associated with the shard snapshot as well as its {@link IndexShardSnapshotStatus} instance. */ -public abstract class ShardSnapshotContext { +public class ShardSnapshotContext { private final Store store; @@ -44,10 +45,16 @@ public abstract class ShardSnapshotContext { private final AtomicBoolean closed = new AtomicBoolean(false); - protected ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status) { + private final IndexCommitProvider indexCommitProvider; + + private SetOnce indexCommit = new SetOnce<>(); + + public ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status, + IndexCommitProvider indexCommitProvider) { this.store = store; this.listener = listener; this.status = status; + this.indexCommitProvider = indexCommitProvider; } /** @@ -62,15 +69,13 @@ public IndexCommit indexCommit() throws IOException { if (closed.get()) { throw new IllegalStateException("Tried to get index commit from closed context."); } - return doIndexCommit(); + if (indexCommit.get() == null) { + indexCommit.set(indexCommitProvider.get()); + } + return indexCommit.get(); } } - /** - * See {@link #indexCommit()} - */ - protected abstract IndexCommit doIndexCommit() throws IOException; - /** * Release resources backing the {@link IndexCommit} returned by {@link #indexCommit()}. * @throws IOException on failure @@ -78,16 +83,11 @@ public IndexCommit indexCommit() throws IOException { private void releaseIndexCommit() throws IOException { if (closed.compareAndSet(false, true)) { synchronized (this) { - doReleaseIndexCommit(); + indexCommitProvider.close(); } } } - /** - * See {@link #releaseIndexCommit()} - */ - protected abstract void doReleaseIndexCommit() throws IOException; - public Store store() { return store; } @@ -136,29 +136,28 @@ public ActionListener completionListener() { return listener; } - public interface IndexCommitProvider extends Closeable, Supplier { + public interface IndexCommitProvider extends Closeable, CheckedSupplier { } public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { - return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus) { - + return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus, new IndexCommitProvider() { private Engine.IndexCommitRef snapshotRef; @Override - protected void doReleaseIndexCommit() throws IOException { + public void close() throws IOException { if (snapshotRef != null) { snapshotRef.close(); } } @Override - protected IndexCommit doIndexCommit() { + public IndexCommit get() throws IOException { if (snapshotRef == null) { snapshotRef = indexShard.acquireLastIndexCommit(true); } return snapshotRef.getIndexCommit(); } - }; + }); } } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index b9bb1cf3b10b3..6db31038cc06f 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -105,7 +105,7 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { final PlainActionFuture future1 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(null, snapshotId, indexId, new TestSnapshotContext(indexCommit, store, snapshotStatus, future1)); + repository.snapshotShard(null, snapshotId, indexId, testShardSnapshotContext(indexCommit, store, snapshotStatus, future1)); future1.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -132,7 +132,7 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); repository.snapshotShard(null, incSnapshotId, indexId, - new TestSnapshotContext(incIndexCommit, store, snapshotStatus, future2)); + testShardSnapshotContext(incIndexCommit, store, snapshotStatus, future2)); future2.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(2, copy.getIncrementalFileCount()); @@ -206,23 +206,17 @@ private int indexDocs(Directory directory) throws IOException { } } - private static class TestSnapshotContext extends ShardSnapshotContext { - - private final IndexCommit indexCommit; - - TestSnapshotContext(IndexCommit indexCommit, Store store, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { - super(store, listener, snapshotStatus); - this.indexCommit = indexCommit; - } - - @Override - protected void doReleaseIndexCommit() { - - } + private static ShardSnapshotContext testShardSnapshotContext(IndexCommit indexCommit, Store store, + IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { + return new ShardSnapshotContext(store, listener, snapshotStatus, new ShardSnapshotContext.IndexCommitProvider() { + @Override + public IndexCommit get() { + return indexCommit; + } - @Override - public IndexCommit doIndexCommit() { - return indexCommit; - } + @Override + public void close() { + } + }); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index 0dca1ec14e7a0..c11e79e777e13 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -140,29 +140,27 @@ protected void closeInternal() { final long maxDoc = segmentInfos.totalMaxDoc(); tempStore.bootstrapNewHistory(maxDoc, maxDoc); super.snapshotShard(mapperService, snapshotId, indexId, - new ShardSnapshotContext(tempStore, context.completionListener(), context.status()) { - private DirectoryReader reader; - private IndexCommit indexCommit; + new ShardSnapshotContext(tempStore, context.completionListener(), context.status(), + new ShardSnapshotContext.IndexCommitProvider() { + private DirectoryReader reader; - @Override - protected void doReleaseIndexCommit() throws IOException { - if (reader != null) { - reader.close(); + @Override + public IndexCommit get() throws IOException { + if (reader == null) { + reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap( + BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); + return reader.getIndexCommit(); + } + throw new AssertionError("Should not be called twice, the caller should cache the index commit"); } - } - @Override - protected IndexCommit doIndexCommit() throws IOException { - if (reader == null) { - reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap( - BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); - assert indexCommit == null; - indexCommit = reader.getIndexCommit(); + @Override + public void close() throws IOException { + if (reader != null) { + reader.close(); + } } - assert indexCommit != null; - return indexCommit; - } - }); + })); } catch (IOException e) { if (directory != null) { try { From fb0b81f690d74e76ad4a9c82b302663c49680d5c Mon Sep 17 00:00:00 2001 From: Armin Date: Mon, 24 Jun 2019 15:06:39 +0200 Subject: [PATCH 16/23] simpler --- .../repositories/ShardSnapshotContext.java | 43 +++++++++---------- .../snapshots/SnapshotShardsService.java | 2 +- .../index/shard/IndexShardTestCase.java | 2 +- .../SourceOnlySnapshotShardTests.java | 10 ++--- 4 files changed, 28 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java index 9a13912970cf8..a4db17a6d89be 100644 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java @@ -49,6 +49,27 @@ public class ShardSnapshotContext { private SetOnce indexCommit = new SetOnce<>(); + public ShardSnapshotContext(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { + this(indexShard.store(), listener, snapshotStatus, new IndexCommitProvider() { + private Engine.IndexCommitRef snapshotRef; + + @Override + public IndexCommit get() { + if (snapshotRef == null) { + snapshotRef = indexShard.acquireLastIndexCommit(true); + } + return snapshotRef.getIndexCommit(); + } + + @Override + public void close() throws IOException { + if (snapshotRef != null) { + snapshotRef.close(); + } + } + }); + } + public ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status, IndexCommitProvider indexCommitProvider) { this.store = store; @@ -138,26 +159,4 @@ public ActionListener completionListener() { public interface IndexCommitProvider extends Closeable, CheckedSupplier { } - - public static ShardSnapshotContext create(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, - ActionListener listener) { - return new ShardSnapshotContext(indexShard.store(), listener, snapshotStatus, new IndexCommitProvider() { - private Engine.IndexCommitRef snapshotRef; - - @Override - public void close() throws IOException { - if (snapshotRef != null) { - snapshotRef.close(); - } - } - - @Override - public IndexCommit get() throws IOException { - if (snapshotRef == null) { - snapshotRef = indexShard.acquireLastIndexCommit(true); - } - return snapshotRef.getIndexCommit(); - } - }); - } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 7a335eebf815d..c31f629b3003d 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -352,7 +352,7 @@ private void snapshot(final IndexShard indexShard, final Snapshot snapshot, fina final Repository repository = snapshotsService.getRepositoriesService().repository(snapshot.getRepository()); // we flush first to make sure we get the latest writes snapshotted repository.snapshotShard(indexShard.mapperService(), snapshot.getSnapshotId(), indexId, - ShardSnapshotContext.create(indexShard, snapshotStatus, listener)); + new ShardSnapshotContext(indexShard, snapshotStatus, listener)); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index f7ffa23e52421..9cd2fa5e6a94f 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -829,7 +829,7 @@ protected static void snapshotShard(final IndexShard shard, IndexId indexId = new IndexId(index.getName(), index.getUUID()); final PlainActionFuture future = PlainActionFuture.newFuture(); repository.snapshotShard(shard.mapperService(), snapshot.getSnapshotId(), indexId, - ShardSnapshotContext.create(shard, snapshotStatus, future)); + new ShardSnapshotContext(shard, snapshotStatus, future)); future.actionGet(); final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 50ce78833f30f..4e26c7e3d94c2 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -101,7 +101,7 @@ public void testSourceIncomplete() throws IOException { () -> { final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); + new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); future.actionGet(); })); assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" @@ -125,7 +125,7 @@ public void testIncrementalSnapshot() throws IOException { SnapshotId snapshotId = new SnapshotId("test", "test"); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); + new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -140,7 +140,7 @@ public void testIncrementalSnapshot() throws IOException { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); + new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt @@ -156,7 +156,7 @@ public void testIncrementalSnapshot() throws IOException { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future))); + new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv @@ -206,7 +206,7 @@ public void testRestoreMinmal() throws IOException { .getIndexMetaData(), false).build()); final PlainActionFuture future = PlainActionFuture.newFuture(); repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - ShardSnapshotContext.create(shard, indexShardSnapshotStatus, future)); + new ShardSnapshotContext(shard, indexShardSnapshotStatus, future)); future.actionGet(); }); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); From c2b6467d049bf64cd032d243fa89db8938c3d7b5 Mon Sep 17 00:00:00 2001 From: Armin Date: Fri, 5 Jul 2019 19:54:29 +0200 Subject: [PATCH 17/23] shorter diff --- .../repositories/blobstore/BlobStoreRepository.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index e2aeeb46fc461..16b3c3296db9c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -857,8 +857,8 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In final Store store = context.store(); final ShardId shardId = store.shardId(); final IndexShardSnapshotStatus snapshotStatus = context.status(); - final long startTime = threadPool.relativeTimeInMillis(); - final Consumer onFailure = e -> context.finish(threadPool.relativeTimeInMillis(), ExceptionsHelper.detailedMessage(e), + final long startTime = threadPool.absoluteTimeInMillis(); + final Consumer onFailure = e -> context.finish(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e), e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e : new IndexShardSnapshotFailedException(store.shardId(), e)); try { @@ -881,6 +881,8 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In "Duplicate snapshot name [" + snapshotId.getName() + "] detected, aborting"); } + final List indexCommitPointFiles = new ArrayList<>(); + ArrayList filesToSnapshot = new ArrayList<>(); store.incRef(); final IndexCommit snapshotIndexCommit = context.indexCommit(); final Collection fileNames; @@ -902,8 +904,6 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In int indexTotalNumberOfFiles = 0; long indexIncrementalSize = 0; long indexTotalFileCount = 0; - final List indexCommitPointFiles = new ArrayList<>(); - final List filesToSnapshot = new ArrayList<>(); for (String fileName : fileNames) { if (snapshotStatus.isAborted()) { logger.debug("[{}] [{}] Aborted on the file [{}], exiting", shardId, snapshotId, fileName); @@ -957,7 +957,7 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In lastSnapshotStatus.getStartTime(), // snapshotStatus.startTime() is assigned on the same machine, // so it's safe to use the relative time in millis - threadPool.relativeTimeInMillis() - lastSnapshotStatus.getStartTime(), + threadPool.absoluteTimeInMillis() - lastSnapshotStatus.getStartTime(), lastSnapshotStatus.getIncrementalFileCount(), lastSnapshotStatus.getIncrementalSize() ); @@ -979,7 +979,7 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In // finalize the snapshot and rewrite the snapshot index with the next sequential snapshot index finalizeShard(newSnapshotsList, fileListGeneration, blobs, "snapshot creation [" + snapshotId + "]", shardContainer, shardId, snapshotId); - context.finish(threadPool.relativeTimeInMillis()); + context.finish(threadPool.absoluteTimeInMillis()); }, onFailure), indexIncrementalFileCount + 1); filesListener.onResponse(null); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); From 27a072958cc23997da007cd0eba08a612fde17c9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 26 Aug 2019 13:38:10 +0200 Subject: [PATCH 18/23] CR: move try-catch into snapshot method --- .../snapshots/SnapshotShardsService.java | 58 +++++++++---------- 1 file changed, 28 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index ff161f8e339ae..0487665138009 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -294,13 +294,16 @@ private void startNewSnapshots(SnapshotsInProgress snapshotsInProgress) { } private void startNewShards(SnapshotsInProgress.Entry entry, Map startedShards) { - final Snapshot snapshot = entry.snapshot(); - final Map indicesMap = entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity())); threadPool.executor(ThreadPool.Names.SNAPSHOT).execute(() -> { + final Snapshot snapshot = entry.snapshot(); + final Map indicesMap = + entry.indices().stream().collect(Collectors.toMap(IndexId::getName, Function.identity())); for (final Map.Entry shardEntry : startedShards.entrySet()) { final ShardId shardId = shardEntry.getKey(); final IndexShardSnapshotStatus snapshotStatus = shardEntry.getValue(); - final ActionListener listener = new ActionListener<>() { + final IndexId indexId = indicesMap.get(shardId.getIndexName()); + assert indexId != null; + snapshot(shardId, snapshot, indexId, snapshotStatus, new ActionListener<>() { @Override public void onResponse(final Void aVoid) { if (logger.isDebugEnabled()) { @@ -315,15 +318,7 @@ public void onFailure(Exception e) { logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to snapshot shard", shardId, snapshot), e); notifyFailedSnapshotShard(snapshot, shardId, ExceptionsHelper.detailedMessage(e)); } - }; - try { - final IndexId indexId = indicesMap.get(shardId.getIndexName()); - assert indexId != null; - snapshot(indicesService.indexServiceSafe( - shardId.getIndex()).getShardOrNull(shardId.id()), snapshot, indexId, snapshotStatus, listener); - } catch (Exception e) { - listener.onFailure(e); - } + }); } }); } @@ -334,27 +329,30 @@ public void onFailure(Exception e) { * @param snapshot snapshot * @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, ActionListener listener) { - final ShardId shardId = indexShard.shardId(); - if (indexShard.routingEntry().primary() == false) { - throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary"); - } - if (indexShard.routingEntry().relocating()) { - // do not snapshot when in the process of relocation of primaries so we won't get conflicts - throw new IndexShardSnapshotFailedException(shardId, "cannot snapshot while relocating"); - } + try { + final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShardOrNull(shardId.id()); + if (indexShard.routingEntry().primary() == false) { + throw new IndexShardSnapshotFailedException(shardId, "snapshot should be performed only on primary"); + } + if (indexShard.routingEntry().relocating()) { + // do not snapshot when in the process of relocation of primaries so we won't get conflicts + throw new IndexShardSnapshotFailedException(shardId, "cannot snapshot while relocating"); + } - final IndexShardState indexShardState = indexShard.state(); - if (indexShardState == IndexShardState.CREATED || indexShardState == IndexShardState.RECOVERING) { - // shard has just been created, or still recovering - throw new IndexShardSnapshotFailedException(shardId, "shard didn't fully recover yet"); - } + final IndexShardState indexShardState = indexShard.state(); + if (indexShardState == IndexShardState.CREATED || indexShardState == IndexShardState.RECOVERING) { + // shard has just been created, or still recovering + throw new IndexShardSnapshotFailedException(shardId, "shard didn't fully recover yet"); + } - final Repository repository = repositoriesService.repository(snapshot.getRepository()); - // we flush first to make sure we get the latest writes snapshotted - repository.snapshotShard(indexShard.mapperService(), snapshot.getSnapshotId(), indexId, - new ShardSnapshotContext(indexShard, snapshotStatus, listener)); + final Repository repository = repositoriesService.repository(snapshot.getRepository()); + repository.snapshotShard(indexShard.mapperService(), snapshot.getSnapshotId(), indexId, + new ShardSnapshotContext(indexShard, snapshotStatus, listener)); + } catch (Exception e) { + listener.onFailure(e); + } } /** From 7a7864e972a9c14020037b68976b5ed0ea6be9e6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 27 Aug 2019 22:36:17 +0200 Subject: [PATCH 19/23] simplify signature --- .../repositories/FilterRepository.java | 8 +- .../repositories/Repository.java | 9 +- .../repositories/ShardSnapshotContext.java | 162 ------------------ .../blobstore/BlobStoreRepository.java | 19 +- .../snapshots/SnapshotShardsService.java | 28 ++- .../RepositoriesServiceTests.java | 5 +- .../repositories/fs/FsRepositoryTests.java | 21 +-- .../index/shard/IndexShardTestCase.java | 20 ++- .../index/shard/RestoreOnlyRepository.java | 7 +- .../xpack/ccr/repository/CcrRepository.java | 6 +- .../SourceOnlySnapshotRepository.java | 66 ++++--- .../SourceOnlySnapshotShardTests.java | 74 ++++---- 12 files changed, 145 insertions(+), 280 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index 2a69b3d684b98..39fd92f9eaae9 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.repositories; +import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -120,12 +121,11 @@ public boolean isReadOnly() { return in.isReadOnly(); } - @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, ShardSnapshotContext context) { - in.snapshotShard(mapperService, snapshotId, indexId, context); + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { + in.snapshotShard(store, mapperService, snapshotId, indexId, snapshotIndexCommit, snapshotStatus, listener); } - @Override public void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index c6ba4efaa8460..e7cdc6224204c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.repositories; +import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -197,12 +198,16 @@ SnapshotInfo finalizeSnapshot(SnapshotId snapshotId, List indices, long *

* 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 store store to be snapshotted * @param mapperService the shards mapper service * @param snapshotId snapshot id * @param indexId id for the index being snapshotted - * @param context shard snapshot context + * @param snapshotIndexCommit commit point + * @param snapshotStatus snapshot status + * @param listener listener invoked on completion */ - void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, ShardSnapshotContext context); + void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit snapshotIndexCommit, + IndexShardSnapshotStatus snapshotStatus, ActionListener listener); /** * Restores snapshot of the shard. diff --git a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java b/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java deleted file mode 100644 index a4db17a6d89be..0000000000000 --- a/server/src/main/java/org/elasticsearch/repositories/ShardSnapshotContext.java +++ /dev/null @@ -1,162 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.repositories; - -import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.CheckedSupplier; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; -import org.elasticsearch.index.store.Store; - -import java.io.Closeable; -import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * Manages the {@link IndexCommit} associated with the shard snapshot as well as its {@link IndexShardSnapshotStatus} instance. - */ -public class ShardSnapshotContext { - - private final Store store; - - private final ActionListener listener; - - private final IndexShardSnapshotStatus status; - - private final AtomicBoolean closed = new AtomicBoolean(false); - - private final IndexCommitProvider indexCommitProvider; - - private SetOnce indexCommit = new SetOnce<>(); - - public ShardSnapshotContext(IndexShard indexShard, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { - this(indexShard.store(), listener, snapshotStatus, new IndexCommitProvider() { - private Engine.IndexCommitRef snapshotRef; - - @Override - public IndexCommit get() { - if (snapshotRef == null) { - snapshotRef = indexShard.acquireLastIndexCommit(true); - } - return snapshotRef.getIndexCommit(); - } - - @Override - public void close() throws IOException { - if (snapshotRef != null) { - snapshotRef.close(); - } - } - }); - } - - public ShardSnapshotContext(Store store, ActionListener listener, IndexShardSnapshotStatus status, - IndexCommitProvider indexCommitProvider) { - this.store = store; - this.listener = listener; - this.status = status; - this.indexCommitProvider = indexCommitProvider; - } - - /** - * Create and return an {@link IndexCommit} for this shard. Repeated invocations of this method return the same {@link IndexCommit}. - * The resources associated with this {@link IndexCommit} are released by {@link #releaseIndexCommit()} when either - * {@link #finish(long, String, Exception)} or {@link #prepareFinalize()} is invoked. - * @return IndexCommit index commit - * @throws IOException on failure - */ - public IndexCommit indexCommit() throws IOException { - synchronized (this) { - if (closed.get()) { - throw new IllegalStateException("Tried to get index commit from closed context."); - } - if (indexCommit.get() == null) { - indexCommit.set(indexCommitProvider.get()); - } - return indexCommit.get(); - } - } - - /** - * Release resources backing the {@link IndexCommit} returned by {@link #indexCommit()}. - * @throws IOException on failure - */ - private void releaseIndexCommit() throws IOException { - if (closed.compareAndSet(false, true)) { - synchronized (this) { - indexCommitProvider.close(); - } - } - } - - public Store store() { - return store; - } - - /** - * Invoke once all writes to the repository have finished for the shard. - * @param endTime Timestamp of when the shard snapshot's writes to the repository finished - */ - public void finish(long endTime) { - status.moveToDone(endTime); - listener.onResponse(null); - } - - /** - * Invoke once all segments for this shard were written to the repository. - * @return IndexSnapshotStatus right after writing all segments to the repository - * @throws IOException On failure to release the resources backing this instance's {@link IndexCommit} - */ - public final IndexShardSnapshotStatus.Copy prepareFinalize() throws IOException { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = status.moveToFinalize(indexCommit().getGeneration()); - releaseIndexCommit(); - return lastSnapshotStatus; - } - - /** - * Invoke in case the shard's snapshot operation failed. - * @param endTime time the shard's snapshot failed - * @param failureMessage failure message - * @param e Exception that caused the shard's snapshot to fail - */ - public final void finish(long endTime, String failureMessage, Exception e) { - status.moveToFailed(endTime, failureMessage); - try { - releaseIndexCommit(); - } catch (Exception ex) { - e.addSuppressed(ex); - } - listener.onFailure(e); - } - - public IndexShardSnapshotStatus status() { - return status; - } - - public ActionListener completionListener() { - return listener; - } - - public interface IndexCommitProvider extends Closeable, CheckedSupplier { - } -} diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 89666867a3322..2c6f6998b83f3 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -86,7 +86,6 @@ import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryException; import org.elasticsearch.repositories.RepositoryVerificationException; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.InvalidSnapshotNameException; import org.elasticsearch.snapshots.SnapshotCreationException; import org.elasticsearch.snapshots.SnapshotException; @@ -885,14 +884,15 @@ private void writeAtomic(final String blobName, final BytesReference bytesRef, b } @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, ShardSnapshotContext context) { - final Store store = context.store(); + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { final ShardId shardId = store.shardId(); - final IndexShardSnapshotStatus snapshotStatus = context.status(); final long startTime = threadPool.absoluteTimeInMillis(); - final Consumer onFailure = e -> context.finish(threadPool.absoluteTimeInMillis(), ExceptionsHelper.detailedMessage(e), - e instanceof IndexShardSnapshotFailedException ? (IndexShardSnapshotFailedException) e + final Consumer onFailure = 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()); @@ -916,7 +916,6 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In final List indexCommitPointFiles = new ArrayList<>(); ArrayList filesToSnapshot = new ArrayList<>(); store.incRef(); - final IndexCommit snapshotIndexCommit = context.indexCommit(); final Collection fileNames; final Store.MetadataSnapshot metadataFromStore; try { @@ -980,7 +979,8 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In final GroupedActionListener filesListener = new GroupedActionListener<>( ActionListener.wrap(v -> { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = context.prepareFinalize(); + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = + snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); // now create and write the commit point final BlobStoreIndexShardSnapshot snapshot = new BlobStoreIndexShardSnapshot(snapshotId.getName(), @@ -1028,7 +1028,8 @@ public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, In logger.warn(() -> new ParameterizedMessage("[{}][{}] failed to delete old index-N blobs during finalization", snapshotId, shardId), e); } - context.finish(threadPool.absoluteTimeInMillis()); + snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis()); + listener.onResponse(null); }, onFailure), indexIncrementalFileCount + 1); filesListener.onResponse(null); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 0487665138009..e855b92e39ae0 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -52,6 +52,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; @@ -63,7 +64,6 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportException; @@ -348,8 +348,30 @@ private void snapshot(final ShardId shardId, final Snapshot snapshot, final Inde } final Repository repository = repositoriesService.repository(snapshot.getRepository()); - repository.snapshotShard(indexShard.mapperService(), snapshot.getSnapshotId(), indexId, - new ShardSnapshotContext(indexShard, snapshotStatus, listener)); + final Engine.IndexCommitRef snapshotRef = indexShard.acquireLastIndexCommit(true); + repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId, + snapshotRef.getIndexCommit(), snapshotStatus, new ActionListener<>() { + @Override + public void onResponse(Void aVoid) { + try { + snapshotRef.close(); + } catch (Exception ex) { + listener.onFailure(ex); + return; + } + listener.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + try { + snapshotRef.close(); + } catch (Exception ex) { + e.addSuppressed(ex); + } + listener.onFailure(e); + } + }); } catch (Exception e) { listener.onFailure(e); } diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index ee075ddab454c..55a365af5d5ff 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.repositories; +import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; @@ -200,8 +201,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - ShardSnapshotContext context) { + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, IndexCommit + snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { } diff --git a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java index 6db31038cc06f..6c48a19cbb5e6 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -35,7 +35,6 @@ import org.apache.lucene.util.IOSupplier; import org.apache.lucene.util.TestUtil; 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.RepositoryMetaData; @@ -55,7 +54,6 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.DummyShardLock; @@ -105,7 +103,8 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { final PlainActionFuture future1 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(null, snapshotId, indexId, testShardSnapshotContext(indexCommit, store, snapshotStatus, future1)); + repository.snapshotShard(store, null, snapshotId, indexId, indexCommit, + snapshotStatus, future1); future1.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -131,8 +130,7 @@ public void testSnapshotAndRestore() throws IOException, InterruptedException { final PlainActionFuture future2 = PlainActionFuture.newFuture(); runGeneric(threadPool, () -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - repository.snapshotShard(null, incSnapshotId, indexId, - testShardSnapshotContext(incIndexCommit, store, snapshotStatus, future2)); + repository.snapshotShard(store, null, incSnapshotId, indexId, incIndexCommit, snapshotStatus, future2); future2.actionGet(); IndexShardSnapshotStatus.Copy copy = snapshotStatus.asCopy(); assertEquals(2, copy.getIncrementalFileCount()); @@ -206,17 +204,4 @@ private int indexDocs(Directory directory) throws IOException { } } - private static ShardSnapshotContext testShardSnapshotContext(IndexCommit indexCommit, Store store, - IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { - return new ShardSnapshotContext(store, listener, snapshotStatus, new ShardSnapshotContext.IndexCommitProvider() { - @Override - public IndexCommit get() { - return indexCommit; - } - - @Override - public void close() { - } - }); - } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 1784e543dd6de..cce9780b09223 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -80,7 +80,6 @@ import org.elasticsearch.indices.recovery.StartRecoveryRequest; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; @@ -829,16 +828,19 @@ protected void recoverShardFromSnapshot(final IndexShard shard, } /** Snapshot a shard using a given repository **/ - protected static void snapshotShard(final IndexShard shard, - final Snapshot snapshot, - final Repository repository) throws IOException { + protected void snapshotShard(final IndexShard shard, + final Snapshot snapshot, + final Repository repository) throws IOException { final IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(); - Index index = shard.shardId().getIndex(); - IndexId indexId = new IndexId(index.getName(), index.getUUID()); final PlainActionFuture future = PlainActionFuture.newFuture(); - repository.snapshotShard(shard.mapperService(), snapshot.getSnapshotId(), indexId, - new ShardSnapshotContext(shard, snapshotStatus, future)); - future.actionGet(); + try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) { + Index index = shard.shardId().getIndex(); + IndexId indexId = new IndexId(index.getName(), index.getUUID()); + + repository.snapshotShard(shard.store(), shard.mapperService(), snapshot.getSnapshotId(), indexId, + indexCommitRef.getIndexCommit(), snapshotStatus, future); + future.actionGet(); + } final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.asCopy(); assertEquals(IndexShardSnapshotStatus.Stage.DONE, lastSnapshotStatus.getStage()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index bca63a0b1cc17..313bf7c5daaa1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.index.shard; +import org.apache.lucene.index.IndexCommit; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -27,10 +28,10 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.store.Store; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotShardFailure; @@ -133,8 +134,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - ShardSnapshotContext context) { + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { } @Override diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index 5be410394aa0e..e8e7567091342 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.IndexCommit; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.ExceptionsHelper; @@ -58,7 +59,6 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.repositories.blobstore.FileRestoreContext; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; @@ -295,8 +295,8 @@ public boolean isReadOnly() { } @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - ShardSnapshotContext context) { + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index c11e79e777e13..abeb4f782fb3e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -15,6 +15,7 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.SimpleFSDirectory; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -24,17 +25,19 @@ import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.env.ShardLock; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ReadOnlyEngine; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; -import org.elasticsearch.repositories.ShardSnapshotContext; +import java.io.Closeable; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; @@ -107,14 +110,15 @@ public void initializeSnapshot(SnapshotId snapshotId, List indices, Met } @Override - public void snapshotShard(MapperService mapperService, SnapshotId snapshotId, IndexId indexId, - ShardSnapshotContext context) { + public void snapshotShard(Store store, MapperService mapperService, SnapshotId snapshotId, IndexId indexId, + IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { if (mapperService.documentMapper() != null // if there is no mapping this is null && mapperService.documentMapper().sourceMapper().isComplete() == false) { - throw new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " + - "or filters the source"); + listener.onFailure( + new IllegalStateException("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled " + + "or filters the source")); + return; } - final Store store = context.store(); Directory unwrap = FilterDirectory.unwrap(store.directory()); if (unwrap instanceof FSDirectory == false) { throw new AssertionError("expected FSDirectory but got " + unwrap.toString()); @@ -134,33 +138,40 @@ protected void closeInternal() { Supplier querySupplier = mapperService.hasNested() ? Queries::newNestedFilter : null; // SourceOnlySnapshot will take care of soft- and hard-deletes no special casing needed here SourceOnlySnapshot snapshot = new SourceOnlySnapshot(tempStore.directory(), querySupplier); - snapshot.syncSnapshot(context.indexCommit()); + snapshot.syncSnapshot(snapshotIndexCommit); // we will use the lucene doc ID as the seq ID so we set the local checkpoint to maxDoc with a new index UUID SegmentInfos segmentInfos = tempStore.readLastCommittedSegmentsInfo(); final long maxDoc = segmentInfos.totalMaxDoc(); tempStore.bootstrapNewHistory(maxDoc, maxDoc); - super.snapshotShard(mapperService, snapshotId, indexId, - new ShardSnapshotContext(tempStore, context.completionListener(), context.status(), - new ShardSnapshotContext.IndexCommitProvider() { - private DirectoryReader reader; - - @Override - public IndexCommit get() throws IOException { - if (reader == null) { - reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap( - BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); - return reader.getIndexCommit(); - } - throw new AssertionError("Should not be called twice, the caller should cache the index commit"); + store.incRef(); + DirectoryReader reader = DirectoryReader.open(tempStore.directory(), + Collections.singletonMap(BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); + IndexCommit indexCommit = reader.getIndexCommit(); + final Directory finalDirectory = directory; + final Closeable closeable = () -> IOUtils.close(finalDirectory, reader); + super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus, ActionListener.runAfter( + new ActionListener<>() { + @Override + public void onResponse(Void aVoid) { + try { + closeable.close(); + } catch (Exception e) { + listener.onFailure(e); + return; } + listener.onResponse(null); + } - @Override - public void close() throws IOException { - if (reader != null) { - reader.close(); - } + @Override + public void onFailure(Exception e) { + try { + closeable.close(); + } catch (Exception ex) { + e.addSuppressed(ex); } - })); + listener.onFailure(e); + } + }, store::decRef)); } catch (IOException e) { if (directory != null) { try { @@ -169,8 +180,7 @@ public void close() throws IOException { e.addSuppressed(ex); } } - // why on earth does this super method not declare IOException - throw new UncheckedIOException(e); + listener.onFailure(e); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 714090d9d6db8..b875f76ac59cf 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -59,7 +59,6 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.Repository; -import org.elasticsearch.repositories.ShardSnapshotContext; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.Matchers; @@ -95,17 +94,16 @@ public void testSourceIncomplete() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - IllegalStateException illegalStateException = expectThrows(IllegalStateException.class, () -> - runAsSnapshot(shard.getThreadPool(), - () -> { - final PlainActionFuture future = PlainActionFuture.newFuture(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); - future.actionGet(); - })); - assertEquals("Can't snapshot _source only on an index that has incomplete source ie. has _source disabled or filters the source" - , illegalStateException.getMessage()); + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + final PlainActionFuture future = PlainActionFuture.newFuture(); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future)); + 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", + illegalStateException.getMessage()); + } closeShards(shard); } @@ -119,13 +117,13 @@ public void testIncrementalSnapshot() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - int totalFileCount; - { + int totalFileCount = -1; + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); SnapshotId snapshotId = new SnapshotId("test", "test"); final PlainActionFuture future = PlainActionFuture.newFuture(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future)); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); @@ -135,34 +133,35 @@ public void testIncrementalSnapshot() throws IOException { indexDoc(shard, "_doc", Integer.toString(10)); indexDoc(shard, "_doc", Integer.toString(11)); - { + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { SnapshotId snapshotId = new SnapshotId("test_1", "test_1"); + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future)); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1.si, _1.fdx, _1.fnm, _1.fdt assertEquals(5, copy.getIncrementalFileCount()); // in total we have 4 more files than the previous snap since we don't count the segments_N twice - assertEquals(totalFileCount + 4, copy.getTotalFileCount()); + assertEquals(totalFileCount+4, copy.getTotalFileCount()); assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); } deleteDoc(shard, "_doc", Integer.toString(10)); - { + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { SnapshotId snapshotId = new SnapshotId("test_2", "test_2"); IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); final PlainActionFuture future = PlainActionFuture.newFuture(); - runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new ShardSnapshotContext(shard, indexShardSnapshotStatus, future))); + runAsSnapshot(shard.getThreadPool(), () -> repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, + snapshotRef.getIndexCommit(), indexShardSnapshotStatus, future)); future.actionGet(); IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); // we processed the segments_N file plus _1_1.liv assertEquals(2, copy.getIncrementalFileCount()); // in total we have 5 more files than the previous snap since we don't count the segments_N twice - assertEquals(totalFileCount + 5, copy.getTotalFileCount()); + assertEquals(totalFileCount+5, copy.getTotalFileCount()); assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); } closeShards(shard); @@ -199,19 +198,21 @@ public void testRestoreMinmal() throws IOException { IndexId indexId = new IndexId(shard.shardId().getIndexName(), shard.shardId().getIndex().getUUID()); SourceOnlySnapshotRepository repository = new SourceOnlySnapshotRepository(createRepository()); repository.start(); - IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); - runAsSnapshot(shard.getThreadPool(), () -> { - repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), - MetaData.builder().put(shard.indexSettings() + try (Engine.IndexCommitRef snapshotRef = shard.acquireLastIndexCommit(true)) { + IndexShardSnapshotStatus indexShardSnapshotStatus = IndexShardSnapshotStatus.newInitializing(); + runAsSnapshot(shard.getThreadPool(), () -> { + repository.initializeSnapshot(snapshotId, Arrays.asList(indexId), + MetaData.builder().put(shard.indexSettings() .getIndexMetaData(), false).build()); - final PlainActionFuture future = PlainActionFuture.newFuture(); - repository.snapshotShard(shard.mapperService(), snapshotId, indexId, - new ShardSnapshotContext(shard, indexShardSnapshotStatus, future)); - future.actionGet(); - }); - IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); - assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); - assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + final PlainActionFuture future = PlainActionFuture.newFuture(); + repository.snapshotShard(shard.store(), shard.mapperService(), snapshotId, indexId, snapshotRef.getIndexCommit(), + indexShardSnapshotStatus, future); + future.actionGet(); + }); + IndexShardSnapshotStatus.Copy copy = indexShardSnapshotStatus.asCopy(); + assertEquals(copy.getTotalFileCount(), copy.getIncrementalFileCount()); + assertEquals(copy.getStage(), IndexShardSnapshotStatus.Stage.DONE); + } shard.refresh("test"); ShardRouting shardRouting = TestShardRouting.newShardRouting(new ShardId("index", "_na_", 0), randomAlphaOfLength(10), true, ShardRoutingState.INITIALIZING, @@ -368,5 +369,4 @@ private static T runAsSnapshot(ThreadPool pool, Callable runnable) { throw new AssertionError(e); } } - } From 7a3e83e1b32c59d6141d0fb2b5d54bb68a9aff76 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 28 Aug 2019 08:17:53 +0200 Subject: [PATCH 20/23] nicer looking --- .../elasticsearch/action/ActionListener.java | 32 +++++++++++++ .../snapshots/SnapshotShardsService.java | 34 +++++--------- .../SourceOnlySnapshotRepository.java | 45 +++++-------------- 3 files changed, 54 insertions(+), 57 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index c21aa3b9d4b8f..af0134f268a70 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -22,6 +22,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.common.CheckedFunction; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.CheckedSupplier; import java.util.ArrayList; @@ -226,6 +227,37 @@ public void onFailure(Exception e) { }; } + /** + * Wraps a given listener and returns a new listener which executes the provided {@code runBefore} + * callback before the listener is notified ia 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 ActionListener runBefore(ActionListener delegate, CheckedRunnable runBefore) { + return new ActionListener<>() { + @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)} * and {@link #onFailure(Exception)} of the provided listener will be called at most once. diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index e855b92e39ae0..56a33896623ff 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -52,6 +52,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -348,30 +349,15 @@ private void snapshot(final ShardId shardId, final Snapshot snapshot, final Inde } final Repository repository = repositoriesService.repository(snapshot.getRepository()); - final Engine.IndexCommitRef snapshotRef = indexShard.acquireLastIndexCommit(true); - repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId, - snapshotRef.getIndexCommit(), snapshotStatus, new ActionListener<>() { - @Override - public void onResponse(Void aVoid) { - try { - snapshotRef.close(); - } catch (Exception ex) { - listener.onFailure(ex); - return; - } - listener.onResponse(null); - } - - @Override - public void onFailure(Exception e) { - try { - snapshotRef.close(); - } catch (Exception ex) { - e.addSuppressed(ex); - } - listener.onFailure(e); - } - }); + Engine.IndexCommitRef snapshotRef = null; + try { + snapshotRef = indexShard.acquireLastIndexCommit(true); + repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId, + snapshotRef.getIndexCommit(), snapshotStatus, ActionListener.runBefore(listener, snapshotRef::close)); + } catch (Exception e) { + IOUtils.close(snapshotRef); + throw e; + } } catch (Exception e) { listener.onFailure(e); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java index abeb4f782fb3e..8e0f7d04c3056 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/SourceOnlySnapshotRepository.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -126,9 +127,10 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s Path dataPath = ((FSDirectory) unwrap).getDirectory().getParent(); // TODO should we have a snapshot tmp directory per shard that is maintained by the system? Path snapPath = dataPath.resolve(SNAPSHOT_DIR_NAME); - FSDirectory directory = null; + final List toClose = new ArrayList<>(3); try { - directory = new SimpleFSDirectory(snapPath); + FSDirectory directory = new SimpleFSDirectory(snapPath); + toClose.add(directory); Store tempStore = new Store(store.shardId(), store.indexSettings(), directory, new ShardLock(store.shardId()) { @Override protected void closeInternal() { @@ -144,41 +146,18 @@ protected void closeInternal() { final long maxDoc = segmentInfos.totalMaxDoc(); tempStore.bootstrapNewHistory(maxDoc, maxDoc); store.incRef(); + toClose.add(store::decRef); DirectoryReader reader = DirectoryReader.open(tempStore.directory(), Collections.singletonMap(BlockTreeTermsReader.FST_MODE_KEY, BlockTreeTermsReader.FSTLoadMode.OFF_HEAP.name())); + toClose.add(reader); IndexCommit indexCommit = reader.getIndexCommit(); - final Directory finalDirectory = directory; - final Closeable closeable = () -> IOUtils.close(finalDirectory, reader); - super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus, ActionListener.runAfter( - new ActionListener<>() { - @Override - public void onResponse(Void aVoid) { - try { - closeable.close(); - } catch (Exception e) { - listener.onFailure(e); - return; - } - listener.onResponse(null); - } - - @Override - public void onFailure(Exception e) { - try { - closeable.close(); - } catch (Exception ex) { - e.addSuppressed(ex); - } - listener.onFailure(e); - } - }, store::decRef)); + super.snapshotShard(tempStore, mapperService, snapshotId, indexId, indexCommit, snapshotStatus, + ActionListener.runBefore(listener, () -> IOUtils.close(toClose))); } catch (IOException e) { - if (directory != null) { - try { - directory.close(); - } catch (IOException ex) { - e.addSuppressed(ex); - } + try { + IOUtils.close(toClose); + } catch (IOException ex) { + e.addSuppressed(ex); } listener.onFailure(e); } From e10d8c0b8816d3834242ce20f8a91e9da4b16ba9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 28 Aug 2019 08:19:17 +0200 Subject: [PATCH 21/23] fix noisy empty line --- .../src/main/java/org/elasticsearch/repositories/Repository.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index e7cdc6224204c..bb3634054daee 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -234,4 +234,5 @@ void restoreShard(Store store, SnapshotId snapshotId, Version version, IndexId i */ IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, Version version, IndexId indexId, ShardId shardId); + } From 8c53eb083deb235b75ca3049fc75d2c5583a76bc Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 28 Aug 2019 08:28:00 +0200 Subject: [PATCH 22/23] stop being clever aobut group action listener --- .../blobstore/BlobStoreRepository.java | 112 +++++++++--------- 1 file changed, 58 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 2c6f6998b83f3..5abdaecc02738 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -977,61 +977,65 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s assert indexIncrementalFileCount == filesToSnapshot.size(); + final Runnable afterUploads = () -> { + final IndexShardSnapshotStatus.Copy lastSnapshotStatus = + snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); + + // 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 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 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()); + listener.onResponse(null); + }; + if (indexIncrementalFileCount == 0) { + afterUploads.run(); + return; + } final GroupedActionListener filesListener = new GroupedActionListener<>( - ActionListener.wrap(v -> { - final IndexShardSnapshotStatus.Copy lastSnapshotStatus = - snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); - - // 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 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 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()); - listener.onResponse(null); - }, onFailure), indexIncrementalFileCount + 1); - filesListener.onResponse(null); + ActionListener.wrap(v -> afterUploads.run(), onFailure), indexIncrementalFileCount); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { executor.execute(new ActionRunnable<>(filesListener) { From a32812777417d409d9beb6aeb509b1abe81f63fe Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 28 Aug 2019 16:10:04 +0200 Subject: [PATCH 23/23] CR: step listeners, test, comment,typo --- .../elasticsearch/action/ActionListener.java | 2 +- .../blobstore/BlobStoreRepository.java | 22 ++++++++++--------- .../snapshots/SnapshotShardsService.java | 1 + .../action/ActionListenerTests.java | 17 ++++++++++++++ 4 files changed, 31 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index af0134f268a70..957f46e6116dc 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -229,7 +229,7 @@ public void onFailure(Exception e) { /** * Wraps a given listener and returns a new listener which executes the provided {@code runBefore} - * callback before the listener is notified ia either {@code #onResponse} or {@code #onFailure}. + * 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. */ diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 5abdaecc02738..4883fbd29f3ec 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -32,6 +32,7 @@ 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; @@ -110,7 +111,6 @@ import java.util.Optional; import java.util.Set; import java.util.concurrent.Executor; -import java.util.function.Consumer; import java.util.stream.Collectors; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; @@ -888,11 +888,12 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s IndexCommit snapshotIndexCommit, IndexShardSnapshotStatus snapshotStatus, ActionListener listener) { final ShardId shardId = store.shardId(); final long startTime = threadPool.absoluteTimeInMillis(); - final Consumer onFailure = e -> { + final StepListener 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()); @@ -977,7 +978,8 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s assert indexIncrementalFileCount == filesToSnapshot.size(); - final Runnable afterUploads = () -> { + final StepListener> allFilesUploadedListener = new StepListener<>(); + allFilesUploadedListener.whenComplete(v -> { final IndexShardSnapshotStatus.Copy lastSnapshotStatus = snapshotStatus.moveToFinalize(snapshotIndexCommit.getGeneration()); @@ -1028,14 +1030,14 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s snapshotId, shardId), e); } snapshotStatus.moveToDone(threadPool.absoluteTimeInMillis()); - listener.onResponse(null); - }; + snapshotDoneListener.onResponse(null); + }, snapshotDoneListener::onFailure); if (indexIncrementalFileCount == 0) { - afterUploads.run(); + allFilesUploadedListener.onResponse(Collections.emptyList()); return; } - final GroupedActionListener filesListener = new GroupedActionListener<>( - ActionListener.wrap(v -> afterUploads.run(), onFailure), indexIncrementalFileCount); + final GroupedActionListener filesListener = + new GroupedActionListener<>(allFilesUploadedListener, indexIncrementalFileCount); final Executor executor = threadPool.executor(ThreadPool.Names.SNAPSHOT); for (BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo : filesToSnapshot) { executor.execute(new ActionRunnable<>(filesListener) { @@ -1051,7 +1053,7 @@ protected void doRun() { }); } } catch (Exception e) { - onFailure.accept(e); + snapshotDoneListener.onFailure(e); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 56a33896623ff..06a8bb74c1648 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -351,6 +351,7 @@ private void snapshot(final ShardId shardId, final Snapshot snapshot, final Inde final Repository repository = repositoriesService.repository(snapshot.getRepository()); Engine.IndexCommitRef snapshotRef = null; try { + // we flush first to make sure we get the latest writes snapshotted snapshotRef = indexShard.acquireLastIndexCommit(true); repository.snapshotShard(indexShard.store(), indexShard.mapperService(), snapshot.getSnapshotId(), indexId, snapshotRef.getIndexCommit(), snapshotStatus, ActionListener.runBefore(listener, snapshotRef::close)); diff --git a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java index cd3735b4843e6..4f9b63fb75e6c 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java @@ -171,6 +171,23 @@ public void testRunAfter() { } } + public void testRunBefore() { + { + AtomicBoolean afterSuccess = new AtomicBoolean(); + ActionListener listener = + ActionListener.runBefore(ActionListener.wrap(r -> {}, e -> {}), () -> afterSuccess.set(true)); + listener.onResponse(null); + assertThat(afterSuccess.get(), equalTo(true)); + } + { + AtomicBoolean afterFailure = new AtomicBoolean(); + ActionListener listener = + ActionListener.runBefore(ActionListener.wrap(r -> {}, e -> {}), () -> afterFailure.set(true)); + listener.onFailure(null); + assertThat(afterFailure.get(), equalTo(true)); + } + } + public void testNotifyOnce() { AtomicInteger onResponseTimes = new AtomicInteger(); AtomicInteger onFailureTimes = new AtomicInteger();