From 7883efe24646750ebd8f83a79b5fbe44e8c257fb Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Fri, 24 Oct 2025 00:14:53 +1100 Subject: [PATCH 1/8] Extract SnapshotShardContext as base class Also add a factory interface which allows customisation from plugins. --- .../elasticsearch/node/NodeConstruction.java | 4 +- .../LocalPrimarySnapshotShardContext.java | 193 ++++++++++++++++++ ...calPrimarySnapshotShardContextFactory.java | 137 +++++++++++++ .../repositories/SnapshotShardContext.java | 118 ++++++----- .../SnapshotShardContextFactory.java | 37 ++++ .../blobstore/BlobStoreRepository.java | 74 +------ .../blobstore/ShardSnapshotTaskRunner.java | 2 +- .../snapshots/SnapshotShardsService.java | 120 +++-------- ...tateServiceShardsClosedListenersTests.java | 4 +- .../ShardSnapshotTaskRunnerTests.java | 3 +- .../repositories/fs/FsRepositoryTests.java | 10 +- .../snapshots/SnapshotResiliencyTests.java | 4 +- .../index/shard/IndexShardTestCase.java | 4 +- .../SourceOnlySnapshotRepository.java | 3 +- .../SourceOnlySnapshotShardTests.java | 14 +- .../SearchableSnapshotDirectoryTests.java | 4 +- 16 files changed, 508 insertions(+), 223 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java create mode 100644 server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java create mode 100644 server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index 6ae1ea9c9c6c8..5561a20a0e9ce 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -191,6 +191,7 @@ import org.elasticsearch.plugins.internal.RestExtension; import org.elasticsearch.plugins.internal.SettingsExtension; import org.elasticsearch.readiness.ReadinessService; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContextFactory; import org.elasticsearch.repositories.RepositoriesModule; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.SnapshotMetrics; @@ -1183,7 +1184,8 @@ public Map queryFields() { clusterService, repositoriesService, transportService, - indicesService + indicesService, + new LocalPrimarySnapshotShardContextFactory(indicesService) ); final CachingSnapshotAndShardByStateMetricsService cachingSnapshotAndShardByStateMetricsService = new CachingSnapshotAndShardByStateMetricsService(clusterService); diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java new file mode 100644 index 0000000000000..49c6028a82b29 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java @@ -0,0 +1,193 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories; + +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.core.IOUtils; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetadata; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.snapshots.AbortedSnapshotException; +import org.elasticsearch.snapshots.SnapshotId; + +import java.io.IOException; +import java.util.Collection; + +public final class LocalPrimarySnapshotShardContext extends SnapshotShardContext { + + private static final Logger logger = LogManager.getLogger(LocalPrimarySnapshotShardContext.class); + + private final Store store; + private final MapperService mapperService; + private final SnapshotIndexCommit commitRef; + + /** + * @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 commitRef commit point reference + * @param shardStateIdentifier a unique identifier of the state of the shard that is stored with the shard's snapshot and used + * to detect if the shard has changed between snapshots. If {@code null} is passed as the identifier + * snapshotting will be done by inspecting the physical files referenced by {@code snapshotIndexCommit} + * @param snapshotStatus snapshot status + * @param repositoryMetaVersion version of the updated repository metadata to write + * @param snapshotStartTime start time of the snapshot found in + * {@link org.elasticsearch.cluster.SnapshotsInProgress.Entry#startTime()} + * @param listener listener invoked on completion + */ + public LocalPrimarySnapshotShardContext( + Store store, + MapperService mapperService, + SnapshotId snapshotId, + IndexId indexId, + SnapshotIndexCommit commitRef, + @Nullable String shardStateIdentifier, + IndexShardSnapshotStatus snapshotStatus, + IndexVersion repositoryMetaVersion, + final long snapshotStartTime, + ActionListener listener + ) { + super( + snapshotId, + indexId, + shardStateIdentifier, + snapshotStatus, + repositoryMetaVersion, + snapshotStartTime, + commitRef.closingBefore(listener) + ); + this.store = store; + this.mapperService = mapperService; + this.commitRef = commitRef; + } + + @Override + public ShardId shardId() { + return store.shardId(); + } + + @Override + public Store store() { + return store; + } + + @Override + public MapperService mapperService() { + return mapperService; + } + + @Override + public IndexCommit indexCommit() { + return commitRef.indexCommit(); + } + + @Override + public Releasable withCommitRef() { + status().ensureNotAborted(); // check this first to avoid acquiring a ref when aborted even if refs are available + if (commitRef.tryIncRef()) { + return Releasables.releaseOnce(commitRef::decRef); + } else { + status().ensureNotAborted(); + assert false : "commit ref closed early in state " + status(); + throw new IndexShardSnapshotFailedException(shardId(), "Store got closed concurrently"); + } + } + + @Override + public boolean isSearchableSnapshot() { + return store.indexSettings().getIndexMetadata().isSearchableSnapshot(); + } + + @Override + public Store.MetadataSnapshot metadataSnapshot() { + final IndexCommit snapshotIndexCommit = indexCommit(); + logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId(), snapshotId(), snapshotIndexCommit); + try { + return store.getMetadata(snapshotIndexCommit); + } catch (IOException e) { + throw new IndexShardSnapshotFailedException(shardId(), "Failed to get store file metadata", e); + } + } + + @Override + public Collection fileNames() { + final IndexCommit snapshotIndexCommit = indexCommit(); + try { + return snapshotIndexCommit.getFileNames(); + } catch (IOException e) { + throw new IndexShardSnapshotFailedException(shardId(), "Failed to get store file names", e); + } + } + + @Override + public boolean assertFileContentsMatchHash(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + if (store.tryIncRef()) { + try (IndexInput indexInput = store.openVerifyingInput(fileInfo.physicalName(), IOContext.READONCE, fileInfo.metadata())) { + final byte[] tmp = new byte[Math.toIntExact(fileInfo.metadata().length())]; + indexInput.readBytes(tmp, 0, tmp.length); + assert fileInfo.metadata().hash().bytesEquals(new BytesRef(tmp)); + } catch (IOException e) { + throw new AssertionError(e); + } finally { + store.decRef(); + } + } else { + try { + status().ensureNotAborted(); + assert false : "if the store is already closed we must have been aborted"; + } catch (Exception e) { + assert e instanceof AbortedSnapshotException : e; + } + } + return true; + } + + @Override + public void failStoreIfCorrupted(Exception e) { + if (Lucene.isCorruptionException(e)) { + try { + store.markStoreCorrupted((IOException) e); + } catch (IOException inner) { + inner.addSuppressed(e); + logger.warn("store cannot be marked as corrupted", inner); + } + } + } + + @Override + public SnapshotShardContext.FileReader fileReader(String file, StoreFileMetadata metadata) throws IOException { + Releasable commitRefReleasable = null; + IndexInput indexInput = null; + try { + commitRefReleasable = withCommitRef(); + indexInput = store.openVerifyingInput(file, IOContext.DEFAULT, metadata); + return new IndexInputFileReader(commitRefReleasable, indexInput); + } catch (Exception e) { + IOUtils.close(e, indexInput, commitRefReleasable); + throw e; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java new file mode 100644 index 0000000000000..5aec229b9919d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java @@ -0,0 +1,137 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.IndexShardState; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.function.Consumer; + +import static org.elasticsearch.snapshots.SnapshotShardsService.getShardStateId; + +public class LocalPrimarySnapshotShardContextFactory implements SnapshotShardContextFactory { + + private static final Logger logger = LogManager.getLogger(LocalPrimarySnapshotShardContextFactory.class); + + private final IndicesService indicesService; + + public LocalPrimarySnapshotShardContextFactory(IndicesService indicesService) { + this.indicesService = indicesService; + } + + @Override + public void asyncCreate( + ShardId shardId, + Snapshot snapshot, + IndexId indexId, + IndexShardSnapshotStatus snapshotStatus, + IndexVersion repositoryMetaVersion, + long snapshotStartTime, + ActionListener listener, + Consumer snapshotShardContextConsumer + ) throws IOException { + + final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(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"); + } + + SnapshotIndexCommit snapshotIndexCommit = null; + try { + snapshotStatus.updateStatusDescription("acquiring commit reference from IndexShard: triggers a shard flush"); + snapshotIndexCommit = new SnapshotIndexCommit(indexShard.acquireIndexCommitForSnapshot()); + snapshotStatus.updateStatusDescription("commit reference acquired, proceeding with snapshot"); + final var shardStateId = getShardStateId(indexShard, snapshotIndexCommit.indexCommit()); // not aborted so indexCommit() ok + snapshotStatus.addAbortListener(makeAbortListener(indexShard.shardId(), snapshot, snapshotIndexCommit)); + snapshotStatus.ensureNotAborted(); + snapshotShardContextConsumer.accept( + new LocalPrimarySnapshotShardContext( + indexShard.store(), + indexShard.mapperService(), + snapshot.getSnapshotId(), + indexId, + snapshotIndexCommit, + shardStateId, + snapshotStatus, + repositoryMetaVersion, + snapshotStartTime, + listener + ) + ); + snapshotIndexCommit = null; + } finally { + if (snapshotIndexCommit != null) { + snapshotIndexCommit.closingBefore(new ActionListener() { + @Override + public void onResponse(Void unused) {} + + @Override + public void onFailure(Exception e) { + // we're already failing exceptionally, and prefer to propagate the original exception instead of this one + logger.warn(Strings.format("exception closing commit for [%s] in [%s]", shardId, snapshot), e); + } + }).onResponse(null); + } + } + } + + static ActionListener makeAbortListener( + ShardId shardId, + Snapshot snapshot, + SnapshotIndexCommit snapshotIndexCommit + ) { + return new ActionListener<>() { + @Override + public void onResponse(IndexShardSnapshotStatus.AbortStatus abortStatus) { + if (abortStatus == IndexShardSnapshotStatus.AbortStatus.ABORTED) { + assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.GENERIC, ThreadPool.Names.SNAPSHOT); + snapshotIndexCommit.onAbort(); + } + } + + @Override + public void onFailure(Exception e) { + logger.error(() -> Strings.format("unexpected failure in %s", description()), e); + assert false : e; + } + + @Override + public String toString() { + return description(); + } + + private String description() { + return Strings.format("abort listener for [%s] in [%s]", shardId, snapshot); + } + }; + } +} diff --git a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java index 42c91f8b7edf6..b2e0c0c515b6f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContext.java @@ -10,85 +10,63 @@ package org.elasticsearch.repositories; import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.store.IndexInput; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DelegatingActionListener; import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.common.lucene.store.InputStreamIndexInput; +import org.elasticsearch.core.IOUtils; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; -import org.elasticsearch.core.Releasables; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.snapshots.SnapshotId; +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; + /** * Context holding the state for creating a shard snapshot via {@link Repository#snapshotShard(SnapshotShardContext)}. * Wraps a {@link org.elasticsearch.index.engine.Engine.IndexCommitRef} that is released once this instances is completed by invoking * either its {@link #onResponse(ShardSnapshotResult)} or {@link #onFailure(Exception)} callback. */ -public final class SnapshotShardContext extends DelegatingActionListener { +public abstract class SnapshotShardContext extends DelegatingActionListener { - private final Store store; - private final MapperService mapperService; private final SnapshotId snapshotId; private final IndexId indexId; - private final SnapshotIndexCommit commitRef; @Nullable private final String shardStateIdentifier; private final IndexShardSnapshotStatus snapshotStatus; private final IndexVersion repositoryMetaVersion; private final long snapshotStartTime; - /** - * @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 commitRef commit point reference - * @param shardStateIdentifier a unique identifier of the state of the shard that is stored with the shard's snapshot and used - * to detect if the shard has changed between snapshots. If {@code null} is passed as the identifier - * snapshotting will be done by inspecting the physical files referenced by {@code snapshotIndexCommit} - * @param snapshotStatus snapshot status - * @param repositoryMetaVersion version of the updated repository metadata to write - * @param snapshotStartTime start time of the snapshot found in - * {@link org.elasticsearch.cluster.SnapshotsInProgress.Entry#startTime()} - * @param listener listener invoked on completion - */ - public SnapshotShardContext( - Store store, - MapperService mapperService, + @SuppressWarnings("this-escape") + protected SnapshotShardContext( SnapshotId snapshotId, IndexId indexId, - SnapshotIndexCommit commitRef, @Nullable String shardStateIdentifier, IndexShardSnapshotStatus snapshotStatus, IndexVersion repositoryMetaVersion, - final long snapshotStartTime, + long snapshotStartTime, ActionListener listener ) { super(new SubscribableListener<>()); - addListener(commitRef.closingBefore(listener)); - this.store = store; - this.mapperService = mapperService; + addListener(listener); this.snapshotId = snapshotId; this.indexId = indexId; - this.commitRef = commitRef; this.shardStateIdentifier = shardStateIdentifier; this.snapshotStatus = snapshotStatus; this.repositoryMetaVersion = repositoryMetaVersion; this.snapshotStartTime = snapshotStartTime; } - public Store store() { - return store; - } - - public MapperService mapperService() { - return mapperService; - } - public SnapshotId snapshotId() { return snapshotId; } @@ -97,10 +75,6 @@ public IndexId indexId() { return indexId; } - public IndexCommit indexCommit() { - return commitRef.indexCommit(); - } - @Nullable public String stateIdentifier() { return shardStateIdentifier; @@ -123,14 +97,58 @@ public void onResponse(ShardSnapshotResult result) { delegate.onResponse(result); } - public Releasable withCommitRef() { - snapshotStatus.ensureNotAborted(); // check this first to avoid acquiring a ref when aborted even if refs are available - if (commitRef.tryIncRef()) { - return Releasables.releaseOnce(commitRef::decRef); - } else { - snapshotStatus.ensureNotAborted(); - assert false : "commit ref closed early in state " + snapshotStatus; - throw new IndexShardSnapshotFailedException(store.shardId(), "Store got closed concurrently"); + public abstract ShardId shardId(); + + public abstract Store store(); + + public abstract MapperService mapperService(); + + public abstract IndexCommit indexCommit(); + + public abstract Releasable withCommitRef(); + + public abstract boolean isSearchableSnapshot(); + + public abstract Store.MetadataSnapshot metadataSnapshot(); + + public abstract Collection fileNames(); + + public abstract boolean assertFileContentsMatchHash(BlobStoreIndexShardSnapshot.FileInfo fileInfo); + + public abstract void failStoreIfCorrupted(Exception e); + + public abstract FileReader fileReader(String file, StoreFileMetadata metadata) throws IOException; + + public interface FileReader extends Closeable { + + InputStream openInput(long limit) throws IOException; + + void verify() throws IOException; + } + + public static class IndexInputFileReader implements FileReader { + + private final Releasable commitRefReleasable; + private final IndexInput indexInput; + + public IndexInputFileReader(Releasable commitRefReleasable, IndexInput indexInput) { + this.commitRefReleasable = commitRefReleasable; + this.indexInput = indexInput; + } + + @Override + public InputStream openInput(long limit) throws IOException { + return new InputStreamIndexInput(indexInput, limit); + } + + @Override + public void close() throws IOException { + IOUtils.close(indexInput, commitRefReleasable); + } + + @Override + public void verify() throws IOException { + Store.verify(indexInput); } } diff --git a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java new file mode 100644 index 0000000000000..b644a67ef4b9f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java @@ -0,0 +1,37 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ + +package org.elasticsearch.repositories; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.index.IndexVersion; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; +import org.elasticsearch.snapshots.Snapshot; + +import java.io.IOException; +import java.util.function.Consumer; + +public interface SnapshotShardContextFactory { + + default boolean ignoreShardCloseEvent() { + return false; + } + + void asyncCreate( + ShardId shardId, + Snapshot snapshot, + IndexId indexId, + IndexShardSnapshotStatus snapshotStatus, + IndexVersion repositoryMetaVersion, + long snapshotStartTime, + ActionListener listener, + Consumer snapshotShardContextConsumer + ) throws IOException; +} 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 fdac63cc5466c..7dc43986c7817 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -12,12 +12,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.BytesRef; @@ -72,8 +70,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.TruncatedOutputStream; -import org.elasticsearch.common.lucene.Lucene; -import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -3299,8 +3295,7 @@ private void doSnapshotShard(SnapshotShardContext context) { context.onFailure(new RepositoryException(metadata.name(), "cannot snapshot shard on a readonly repository")); return; } - final Store store = context.store(); - final ShardId shardId = store.shardId(); + final ShardId shardId = context.shardId(); final SnapshotId snapshotId = context.snapshotId(); final IndexShardSnapshotStatus snapshotStatus = context.status(); snapshotStatus.updateStatusDescription("snapshot task runner: setting up shard snapshot"); @@ -3361,7 +3356,7 @@ private void doSnapshotShard(SnapshotShardContext context) { int filesInShardMetadataCount = 0; long filesInShardMetadataSize = 0; - if (store.indexSettings().getIndexMetadata().isSearchableSnapshot()) { + if (context.isSearchableSnapshot()) { indexCommitPointFiles = Collections.emptyList(); } else if (filesFromSegmentInfos == null) { // If we did not find a set of files that is equal to the current commit we determine the files to upload by comparing files @@ -3371,14 +3366,8 @@ private void doSnapshotShard(SnapshotShardContext context) { final Store.MetadataSnapshot metadataFromStore; try (Releasable ignored = context.withCommitRef()) { // TODO apparently we don't use the MetadataSnapshot#.recoveryDiff(...) here but we should - try { - final IndexCommit snapshotIndexCommit = context.indexCommit(); - logger.trace("[{}] [{}] Loading store metadata using index commit [{}]", shardId, snapshotId, snapshotIndexCommit); - metadataFromStore = store.getMetadata(snapshotIndexCommit); - fileNames = snapshotIndexCommit.getFileNames(); - } catch (IOException e) { - throw new IndexShardSnapshotFailedException(shardId, "Failed to get store file metadata", e); - } + metadataFromStore = context.metadataSnapshot(); + fileNames = context.fileNames(); } for (String fileName : fileNames) { ensureNotAborted(shardId, snapshotId, snapshotStatus, fileName); @@ -3406,7 +3395,7 @@ private void doSnapshotShard(SnapshotShardContext context) { if (needsWrite) { filesToSnapshot.add(snapshotFileInfo); } else { - assert assertFileContentsMatchHash(snapshotStatus, snapshotFileInfo, store); + assert context.assertFileContentsMatchHash(snapshotFileInfo); filesInShardMetadataCount += 1; filesInShardMetadataSize += md.length(); } @@ -3639,32 +3628,6 @@ protected void snapshotFiles( } } - private static boolean assertFileContentsMatchHash( - IndexShardSnapshotStatus snapshotStatus, - BlobStoreIndexShardSnapshot.FileInfo fileInfo, - Store store - ) { - if (store.tryIncRef()) { - try (IndexInput indexInput = store.openVerifyingInput(fileInfo.physicalName(), IOContext.READONCE, fileInfo.metadata())) { - final byte[] tmp = new byte[Math.toIntExact(fileInfo.metadata().length())]; - indexInput.readBytes(tmp, 0, tmp.length); - assert fileInfo.metadata().hash().bytesEquals(new BytesRef(tmp)); - } catch (IOException e) { - throw new AssertionError(e); - } finally { - store.decRef(); - } - } else { - try { - snapshotStatus.ensureNotAborted(); - assert false : "if the store is already closed we must have been aborted"; - } catch (Exception e) { - assert e instanceof AbortedSnapshotException : e; - } - } - return true; - } - @Override public void restoreShard( Store store, @@ -4177,23 +4140,17 @@ private Tuple buildBlobStoreIndexShardSnapsh */ protected void snapshotFile(SnapshotShardContext context, FileInfo fileInfo) throws IOException { final IndexId indexId = context.indexId(); - final Store store = context.store(); - final ShardId shardId = store.shardId(); + final ShardId shardId = context.shardId(); final IndexShardSnapshotStatus snapshotStatus = context.status(); final SnapshotId snapshotId = context.snapshotId(); final BlobContainer shardContainer = shardContainer(indexId, shardId); final String file = fileInfo.physicalName(); - try ( - Releasable ignored = context.withCommitRef(); - IndexInput indexInput = store.openVerifyingInput(file, IOContext.DEFAULT, fileInfo.metadata()) - ) { + try (var fileReader = context.fileReader(file, fileInfo.metadata())) { for (int i = 0; i < fileInfo.numberOfParts(); i++) { final long partBytes = fileInfo.partBytes(i); // Make reads abortable by mutating the snapshotStatus object - final InputStream inputStream = new FilterInputStream( - maybeRateLimitSnapshots(new InputStreamIndexInput(indexInput, partBytes)) - ) { + final InputStream inputStream = new FilterInputStream(maybeRateLimitSnapshots(fileReader.openInput(partBytes))) { @Override public int read() throws IOException { checkAborted(); @@ -4232,26 +4189,15 @@ private void checkAborted() { ); } blobStoreSnapshotMetrics.incrementNumberOfBlobsUploaded(); - Store.verify(indexInput); + fileReader.verify(); snapshotStatus.addProcessedFile(fileInfo.length()); } catch (Exception t) { - failStoreIfCorrupted(store, t); + context.failStoreIfCorrupted(t); snapshotStatus.addProcessedFile(0); throw t; } } - private static void failStoreIfCorrupted(Store store, Exception e) { - if (Lucene.isCorruptionException(e)) { - try { - store.markStoreCorrupted((IOException) e); - } catch (IOException inner) { - inner.addSuppressed(e); - logger.warn("store cannot be marked as corrupted", inner); - } - } - } - public boolean supportURLRepo() { return supportURLRepo; } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunner.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunner.java index 715769665b117..4c7c8097e08e8 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunner.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunner.java @@ -62,7 +62,7 @@ private String snapshotUUID() { @SuppressWarnings("resource") private ShardId shardId() { - return context().store().shardId(); + return context().shardId(); } private static final Comparator COMPARATOR = Comparator diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 1eb5ede5848a2..349f4059d6607 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -37,9 +37,8 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.snapshots.IndexShardSnapshotFailedException; +import org.elasticsearch.index.snapshots.IndexShardSnapshotException; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus.Stage; import org.elasticsearch.indices.IndicesService; @@ -50,8 +49,7 @@ import org.elasticsearch.repositories.ShardGeneration; import org.elasticsearch.repositories.ShardGenerations; import org.elasticsearch.repositories.ShardSnapshotResult; -import org.elasticsearch.repositories.SnapshotIndexCommit; -import org.elasticsearch.repositories.SnapshotShardContext; +import org.elasticsearch.repositories.SnapshotShardContextFactory; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; @@ -103,18 +101,21 @@ public final class SnapshotShardsService extends AbstractLifecycleComponent impl // Runs the tasks that promptly notify shards of aborted snapshots so that resources can be released ASAP private final ThrottledTaskRunner notifyOnAbortTaskRunner; + private final SnapshotShardContextFactory snapshotShardContextFactory; public SnapshotShardsService( Settings settings, ClusterService clusterService, RepositoriesService repositoriesService, TransportService transportService, - IndicesService indicesService + IndicesService indicesService, + SnapshotShardContextFactory snapshotShardContextFactory ) { this.indicesService = indicesService; this.repositoriesService = repositoriesService; this.transportService = transportService; this.clusterService = clusterService; + this.snapshotShardContextFactory = snapshotShardContextFactory; this.threadPool = transportService.getThreadPool(); this.snapshotShutdownProgressTracker = new SnapshotShutdownProgressTracker( () -> clusterService.state().nodes().getLocalNodeId(), @@ -234,6 +235,9 @@ private static boolean isPausingProgressTrackedShutdown(@Nullable SingleNodeShut @Override public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexShard, Settings indexSettings) { + if (snapshotShardContextFactory.ignoreShardCloseEvent()) { + return; + } // abort any snapshots occurring on the soon-to-be closed shard synchronized (shardSnapshots) { for (Map.Entry> snapshotShards : shardSnapshots.entrySet()) { @@ -289,6 +293,20 @@ public Map currentSnapshotShards(Snapsho } } + public void ensureShardSnapshotNotAborted(Snapshot snapshot, ShardId shardId) { + synchronized (shardSnapshots) { + final var current = shardSnapshots.get(snapshot); + if (current == null) { + throw new SnapshotMissingException(snapshot.getRepository(), snapshot.getSnapshotId().getName()); + } + final var indexShardSnapshotStatus = current.get(shardId); + if (indexShardSnapshotStatus == null) { + throw new IndexShardSnapshotException(shardId, "shard snapshot [" + snapshot.getSnapshotId() + "] does not exist"); + } + indexShardSnapshotStatus.ensureNotAborted(); + } + } + /** * Cancels any snapshots that have been removed from the given list of SnapshotsInProgress. */ @@ -590,93 +608,21 @@ private void snapshot( ActionListener.run(resultListener, listener -> { snapshotStatus.updateStatusDescription("has started"); snapshotStatus.ensureNotAborted(); - final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(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 Repository repository = repositoriesService.repository(snapshot.getProjectId(), snapshot.getRepository()); - SnapshotIndexCommit snapshotIndexCommit = null; - try { - snapshotStatus.updateStatusDescription("acquiring commit reference from IndexShard: triggers a shard flush"); - snapshotIndexCommit = new SnapshotIndexCommit(indexShard.acquireIndexCommitForSnapshot()); - snapshotStatus.updateStatusDescription("commit reference acquired, proceeding with snapshot"); - final var shardStateId = getShardStateId(indexShard, snapshotIndexCommit.indexCommit()); // not aborted so indexCommit() ok - snapshotStatus.addAbortListener(makeAbortListener(indexShard.shardId(), snapshot, snapshotIndexCommit)); - snapshotStatus.ensureNotAborted(); - repository.snapshotShard( - new SnapshotShardContext( - indexShard.store(), - indexShard.mapperService(), - snapshot.getSnapshotId(), - indexId, - snapshotIndexCommit, - shardStateId, - snapshotStatus, - version, - entryStartTime, - listener - ) - ); - snapshotIndexCommit = null; // success - } finally { - if (snapshotIndexCommit != null) { - snapshotIndexCommit.closingBefore(new ActionListener() { - @Override - public void onResponse(Void unused) {} - - @Override - public void onFailure(Exception e) { - // we're already failing exceptionally, and prefer to propagate the original exception instead of this one - logger.warn(Strings.format("exception closing commit for [%s] in [%s]", indexShard.shardId(), snapshot), e); - } - }).onResponse(null); - } - } + snapshotShardContextFactory.asyncCreate( + shardId, + snapshot, + indexId, + snapshotStatus, + version, + entryStartTime, + listener, + repository::snapshotShard + ); }); } - private static ActionListener makeAbortListener( - ShardId shardId, - Snapshot snapshot, - SnapshotIndexCommit snapshotIndexCommit - ) { - return new ActionListener<>() { - @Override - public void onResponse(IndexShardSnapshotStatus.AbortStatus abortStatus) { - if (abortStatus == IndexShardSnapshotStatus.AbortStatus.ABORTED) { - assert ThreadPool.assertCurrentThreadPool(ThreadPool.Names.GENERIC, ThreadPool.Names.SNAPSHOT); - snapshotIndexCommit.onAbort(); - } - } - - @Override - public void onFailure(Exception e) { - logger.error(() -> Strings.format("unexpected failure in %s", description()), e); - assert false : e; - } - - @Override - public String toString() { - return description(); - } - - private String description() { - return Strings.format("abort listener for [%s] in [%s]", shardId, snapshot); - } - }; - } - /** * Generates an identifier from the current state of a shard that can be used to detect whether a shard's contents * have changed between two snapshots. diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceShardsClosedListenersTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceShardsClosedListenersTests.java index d645007e7c740..20979487f5ecf 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceShardsClosedListenersTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceShardsClosedListenersTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.indices.recovery.PeerRecoverySourceService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.SnapshotShardContextFactory; import org.elasticsearch.search.SearchService; import org.elasticsearch.snapshots.SnapshotShardsService; import org.elasticsearch.test.transport.MockTransport; @@ -158,7 +159,8 @@ class TestIndicesClusterStateService extends IndicesClusterStateService { new ClusterService(Settings.EMPTY, ClusterSettings.createBuiltInClusterSettings(), threadPool, null), mock(RepositoriesService.class), MockTransportService.createMockTransportService(new MockTransport(), threadPool), - mock(IndicesService.class) + mock(IndicesService.class), + mock(SnapshotShardContextFactory.class) ), mock(PrimaryReplicaSyncer.class), RetentionLeaseSyncer.EMPTY, diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunnerTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunnerTests.java index f3c0f419cde7a..ff3308d8e62a9 100644 --- a/server/src/test/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunnerTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/ShardSnapshotTaskRunnerTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.SnapshotIndexCommit; import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.snapshots.SnapshotId; @@ -125,7 +126,7 @@ public static SnapshotShardContext dummyContext(final SnapshotId snapshotId, fin Settings.EMPTY ); final var dummyStore = new Store(shardId, indexSettings, new ByteBuffersDirectory(), new DummyShardLock(shardId)); - return new SnapshotShardContext( + return new LocalPrimarySnapshotShardContext( dummyStore, null, snapshotId, 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 3b1eb8c1f39d9..11cf6c15f9fa0 100644 --- a/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/fs/FsRepositoryTests.java @@ -56,10 +56,10 @@ import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.ShardGeneration; import org.elasticsearch.repositories.ShardSnapshotResult; import org.elasticsearch.repositories.SnapshotIndexCommit; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; @@ -125,7 +125,7 @@ public void testSnapshotAndRestore() throws IOException { final PlainActionFuture snapshot1Future = new PlainActionFuture<>(); IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(null); repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( store, null, snapshotId, @@ -168,7 +168,7 @@ public void testSnapshotAndRestore() throws IOException { final PlainActionFuture snapshot2future = new PlainActionFuture<>(); IndexShardSnapshotStatus snapshotStatus2 = IndexShardSnapshotStatus.newInitializing(shardGeneration); repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( store, null, incSnapshotId, @@ -309,7 +309,7 @@ protected BlobContainer wrapChild(BlobContainer child) { canErrorForWriteBlob.set(true); shouldErrorForWriteMetadataBlob.set(false); repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( store1, null, snapshotId, @@ -344,7 +344,7 @@ protected BlobContainer wrapChild(BlobContainer child) { canErrorForWriteBlob.set(false); shouldErrorForWriteMetadataBlob.set(true); repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( store2, null, snapshotId, diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 929a6e513eb9b..56d36d5374840 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -172,6 +172,7 @@ import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.internal.DocumentParsingProvider; import org.elasticsearch.plugins.scanners.StablePluginsRegistry; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContextFactory; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; @@ -2502,7 +2503,8 @@ public RecyclerBytesStreamOutput newNetworkBytesStream() { clusterService, repositoriesService, transportService, - indicesService + indicesService, + new LocalPrimarySnapshotShardContextFactory(indicesService) ); final ShardStateAction shardStateAction = new ShardStateAction( clusterService, 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 394134c978c79..c2c37856fd4a0 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 @@ -83,11 +83,11 @@ import org.elasticsearch.indices.recovery.plan.PeerOnlyRecoveryPlannerService; import org.elasticsearch.indices.recovery.plan.RecoveryPlannerService; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.ShardGeneration; import org.elasticsearch.repositories.ShardSnapshotResult; import org.elasticsearch.repositories.SnapshotIndexCommit; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.test.DummyShardLock; @@ -1282,7 +1282,7 @@ protected ShardGeneration snapshotShard(final IndexShard shard, final Snapshot s final ShardGeneration shardGen; try (Engine.IndexCommitRef indexCommitRef = shard.acquireLastIndexCommit(true)) { repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshot.getSnapshotId(), diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java index c845df8501e45..0479b9f5db192 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotRepository.java @@ -41,6 +41,7 @@ import org.elasticsearch.repositories.FilterRepository; import org.elasticsearch.repositories.FinalizeSnapshotContext; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.SnapshotIndexCommit; import org.elasticsearch.repositories.SnapshotShardContext; @@ -211,7 +212,7 @@ protected void closeInternal() { toClose.add(reader); IndexCommit indexCommit = reader.getIndexCommit(); super.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( tempStore, mapperService, context.snapshotId(), diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java index 655547a3f26ea..45b44867123af 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/sourceonly/SourceOnlySnapshotShardTests.java @@ -70,12 +70,12 @@ import org.elasticsearch.repositories.FinalizeSnapshotContext; import org.elasticsearch.repositories.FinalizeSnapshotContext.UpdatedShardGenerations; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.ShardGeneration; import org.elasticsearch.repositories.ShardGenerations; import org.elasticsearch.repositories.ShardSnapshotResult; import org.elasticsearch.repositories.SnapshotIndexCommit; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.blobstore.ESBlobStoreRepositoryIntegTestCase; import org.elasticsearch.repositories.fs.FsRepository; @@ -135,7 +135,7 @@ public void testSourceIncomplete() throws IOException { runAsSnapshot( shard.getThreadPool(), () -> repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, @@ -186,7 +186,7 @@ public void testSourceIncompleteSyntheticSourceNoDoc() throws IOException { runAsSnapshot( shard.getThreadPool(), () -> repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, @@ -230,7 +230,7 @@ public void testIncrementalSnapshot() throws IOException { runAsSnapshot( shard.getThreadPool(), () -> repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, @@ -261,7 +261,7 @@ public void testIncrementalSnapshot() throws IOException { runAsSnapshot( shard.getThreadPool(), () -> repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, @@ -292,7 +292,7 @@ public void testIncrementalSnapshot() throws IOException { runAsSnapshot( shard.getThreadPool(), () -> repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, @@ -355,7 +355,7 @@ public void testRestoreMinimal() throws IOException { final PlainActionFuture future = new PlainActionFuture<>(); runAsSnapshot(shard.getThreadPool(), () -> { repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( shard.store(), shard.mapperService(), snapshotId, diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryTests.java index 35d71ba23e283..dd13dc439c2e4 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/store/SearchableSnapshotDirectoryTests.java @@ -82,9 +82,9 @@ import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.LocalPrimarySnapshotShardContext; import org.elasticsearch.repositories.ShardSnapshotResult; import org.elasticsearch.repositories.SnapshotIndexCommit; -import org.elasticsearch.repositories.SnapshotShardContext; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.blobstore.BlobStoreTestUtil; import org.elasticsearch.repositories.fs.FsRepository; @@ -620,7 +620,7 @@ private void testDirectories( threadPool.generic().submit(() -> { IndexShardSnapshotStatus snapshotStatus = IndexShardSnapshotStatus.newInitializing(null); repository.snapshotShard( - new SnapshotShardContext( + new LocalPrimarySnapshotShardContext( store, null, snapshotId, From ea99f1914c2076d178436ac5e5c90c128ceb5772 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Fri, 24 Oct 2025 22:32:01 +1100 Subject: [PATCH 2/8] tweak asyncCreate signature --- ...calPrimarySnapshotShardContextFactory.java | 32 +++++++++---------- .../SnapshotShardContextFactory.java | 7 ++-- .../snapshots/SnapshotShardsService.java | 12 ++----- 3 files changed, 20 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java index 5aec229b9919d..01df5e136b13f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java @@ -10,6 +10,7 @@ package org.elasticsearch.repositories; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.common.Strings; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.IndexShard; @@ -24,7 +25,6 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.function.Consumer; import static org.elasticsearch.snapshots.SnapshotShardsService.getShardStateId; @@ -39,15 +39,14 @@ public LocalPrimarySnapshotShardContextFactory(IndicesService indicesService) { } @Override - public void asyncCreate( + public SubscribableListener asyncCreate( ShardId shardId, Snapshot snapshot, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, IndexVersion repositoryMetaVersion, long snapshotStartTime, - ActionListener listener, - Consumer snapshotShardContextConsumer + ActionListener listener ) throws IOException { final IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id()); @@ -73,21 +72,20 @@ public void asyncCreate( final var shardStateId = getShardStateId(indexShard, snapshotIndexCommit.indexCommit()); // not aborted so indexCommit() ok snapshotStatus.addAbortListener(makeAbortListener(indexShard.shardId(), snapshot, snapshotIndexCommit)); snapshotStatus.ensureNotAborted(); - snapshotShardContextConsumer.accept( - new LocalPrimarySnapshotShardContext( - indexShard.store(), - indexShard.mapperService(), - snapshot.getSnapshotId(), - indexId, - snapshotIndexCommit, - shardStateId, - snapshotStatus, - repositoryMetaVersion, - snapshotStartTime, - listener - ) + final var snapshotShardContext = new LocalPrimarySnapshotShardContext( + indexShard.store(), + indexShard.mapperService(), + snapshot.getSnapshotId(), + indexId, + snapshotIndexCommit, + shardStateId, + snapshotStatus, + repositoryMetaVersion, + snapshotStartTime, + listener ); snapshotIndexCommit = null; + return SubscribableListener.newSucceeded(snapshotShardContext); } finally { if (snapshotIndexCommit != null) { snapshotIndexCommit.closingBefore(new ActionListener() { diff --git a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java index b644a67ef4b9f..653ab3d5505b8 100644 --- a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java +++ b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java @@ -10,13 +10,13 @@ package org.elasticsearch.repositories; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.SubscribableListener; import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; import org.elasticsearch.snapshots.Snapshot; import java.io.IOException; -import java.util.function.Consumer; public interface SnapshotShardContextFactory { @@ -24,14 +24,13 @@ default boolean ignoreShardCloseEvent() { return false; } - void asyncCreate( + SubscribableListener asyncCreate( ShardId shardId, Snapshot snapshot, IndexId indexId, IndexShardSnapshotStatus snapshotStatus, IndexVersion repositoryMetaVersion, long snapshotStartTime, - ActionListener listener, - Consumer snapshotShardContextConsumer + ActionListener listener ) throws IOException; } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 349f4059d6607..03bd843bb3a6c 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -610,16 +610,8 @@ private void snapshot( snapshotStatus.ensureNotAborted(); final Repository repository = repositoriesService.repository(snapshot.getProjectId(), snapshot.getRepository()); - snapshotShardContextFactory.asyncCreate( - shardId, - snapshot, - indexId, - snapshotStatus, - version, - entryStartTime, - listener, - repository::snapshotShard - ); + snapshotShardContextFactory.asyncCreate(shardId, snapshot, indexId, snapshotStatus, version, entryStartTime, listener) + .addListener(ActionListener.wrap(repository::snapshotShard, listener::onFailure)); }); } From 0f29e73a937727953e2e3cd3388083a2b3ed2e79 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Mon, 27 Oct 2025 15:56:29 +1100 Subject: [PATCH 3/8] extract resuable part from SnapshotStressTestsIT --- .../snapshots/SnapshotStressTestsIT.java | 1750 +--------------- .../snapshots/SnapshotStressTestsHelper.java | 1842 +++++++++++++++++ 2 files changed, 1844 insertions(+), 1748 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java index 16760bb7cb165..80832e4445243 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/SnapshotStressTestsIT.java @@ -9,82 +9,11 @@ package org.elasticsearch.snapshots; -import org.apache.logging.log4j.Level; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.tests.util.LuceneTestCase; -import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ShardOperationFailedException; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequestBuilder; -import org.elasticsearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequestBuilder; -import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequestBuilder; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.support.SubscribableListener; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.client.internal.Client; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ClusterStateUpdateTask; -import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; -import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Priority; -import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.ReferenceDocs; -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.util.concurrent.ListenableFuture; -import org.elasticsearch.core.CheckedConsumer; -import org.elasticsearch.core.CheckedRunnable; -import org.elasticsearch.core.Nullable; -import org.elasticsearch.core.Releasable; -import org.elasticsearch.core.Releasables; -import org.elasticsearch.core.TimeValue; -import org.elasticsearch.core.Tuple; -import org.elasticsearch.monitor.jvm.HotThreads; -import org.elasticsearch.repositories.RepositoryCleanupResult; -import org.elasticsearch.repositories.fs.FsRepository; -import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.threadpool.ScalingExecutorBuilder; -import org.elasticsearch.threadpool.TestThreadPool; -import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.snapshots.SnapshotStressTestsHelper.TrackedCluster; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.SNAPSHOT_ONLY_FORMAT_PARAMS; -import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThan; -import static org.hamcrest.Matchers.notNullValue; +import static org.elasticsearch.snapshots.SnapshotStressTestsHelper.nodeNames; @LuceneTestCase.SuppressFileSystems(value = "HandleLimitFS") // we sometimes have >2048 open files public class SnapshotStressTestsIT extends AbstractSnapshotIntegTestCase { @@ -99,1679 +28,4 @@ public void testRandomActivities() throws InterruptedException { new TrackedCluster(internalCluster(), nodeNames(discoveryNodes.getMasterNodes()), nodeNames(discoveryNodes.getDataNodes())).run(); disableRepoConsistencyCheck("have not necessarily written to all repositories"); } - - private static Set nodeNames(Map nodesMap) { - return nodesMap.values().stream().map(DiscoveryNode::getName).collect(Collectors.toSet()); - } - - /** - * Encapsulates a common pattern of trying to acquire a bunch of resources and then transferring ownership elsewhere on success, - * but releasing them on failure. - */ - private static class TransferableReleasables implements Releasable { - - private boolean transferred = false; - private final List releasables = new ArrayList<>(); - - T add(T releasable) { - assert transferred == false : "already transferred"; - releasables.add(releasable); - return releasable; - } - - Releasable transfer() { - assert transferred == false : "already transferred"; - transferred = true; - Collections.reverse(releasables); - return () -> Releasables.close(releasables); - } - - @Override - public void close() { - if (transferred == false) { - Releasables.close(releasables); - } - } - } - - @Nullable // if no permit was acquired - private static Releasable tryAcquirePermit(Semaphore permits) { - if (permits.tryAcquire()) { - return Releasables.releaseOnce(permits::release); - } else { - return null; - } - } - - @Nullable // if not all permits were acquired - private static Releasable tryAcquireAllPermits(Semaphore permits) { - if (permits.tryAcquire(Integer.MAX_VALUE)) { - return Releasables.releaseOnce(() -> permits.release(Integer.MAX_VALUE)); - } else { - return null; - } - } - - private static AbstractRunnable mustSucceed(CheckedRunnable runnable) { - return new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - logAndFailTest(e); - } - - @Override - protected void doRun() throws Exception { - runnable.run(); - } - - @Override - public void onRejection(Exception e) { - // ok, shutting down - } - }; - } - - private static ActionListener mustSucceed(CheckedConsumer consumer) { - return new ActionListener<>() { - @Override - public void onResponse(T t) { - try { - consumer.accept(t); - } catch (Exception e) { - logAndFailTest(e); - } - } - - @Override - public void onFailure(Exception e) { - logAndFailTest(e); - } - }; - } - - private static void logAndFailTest(Exception e) { - final AssertionError assertionError = new AssertionError("unexpected", e); - TrackedCluster.logger.error("test failed", assertionError); - throw assertionError; - } - - /** - * Test harness for snapshot stress tests. - * - * The test performs random operations on the cluster, as if from an external client: - * - * - indexing docs, deleting and re-creating the indices - * - restarting nodes - * - removing and adding repositories - * - taking snapshots (sometimes partial), cloning them, and deleting them - * - * It ensures that these operations should succeed via a system of shared/exclusive locks implemented via permits: acquiring a single - * permit is a shared lock, whereas acquiring all the permits is an exclusive lock. So for instance taking a snapshot acquires a shared - * lock on the repository (permitting other concurrent snapshots/clones/deletes) whereas deleting and recreating the repository requires - * an exclusive lock (ensuring that there are no ongoing operations on the repository, and preventing any new such operations from - * starting). - * - * None of the operations block. If the necessary locks aren't all available then the operation just releases the ones it has acquired - * and tries again later. - * - * The test completes after completing a certain number of snapshots (see {@link #completedSnapshotLatch}) or after a certain time has - * elapsed. - */ - private static class TrackedCluster { - - static final Logger logger = LogManager.getLogger(TrackedCluster.class); - static final String CLIENT = "client"; - static final String NODE_RESTARTER = "node_restarter"; - - private final ThreadPool threadPool = new TestThreadPool( - "TrackedCluster", - // a single thread for "client" activities, to limit the number of activities all starting at once - new ScalingExecutorBuilder(CLIENT, 1, 1, TimeValue.ZERO, true, CLIENT), - new ScalingExecutorBuilder(NODE_RESTARTER, 1, 5, TimeValue.ZERO, true, NODE_RESTARTER) - ); - private final Executor clientExecutor = threadPool.executor(CLIENT); - - private final AtomicBoolean shouldStop = new AtomicBoolean(); - private final InternalTestCluster cluster; - private final Map nodes = ConcurrentCollections.newConcurrentMap(); - private final Map repositories = ConcurrentCollections.newConcurrentMap(); - private final Map indices = ConcurrentCollections.newConcurrentMap(); - private final Map snapshots = ConcurrentCollections.newConcurrentMap(); - - /** - * If we acquire permits on nodes in a completely random order then we tend to block all possible restarts. Instead we always try - * the nodes in the same order, held in this field, so that nodes nearer the end of the list are more likely to be restartable. - * The elected master node is usually last in this list. - */ - private volatile List shuffledNodes; - - private final AtomicInteger snapshotCounter = new AtomicInteger(); - private final CountDownLatch completedSnapshotLatch = new CountDownLatch(30); - - TrackedCluster(InternalTestCluster cluster, Set masterNodeNames, Set dataNodeNames) { - this.cluster = cluster; - for (String nodeName : cluster.getNodeNames()) { - nodes.put(nodeName, new TrackedNode(nodeName, masterNodeNames.contains(nodeName), dataNodeNames.contains(nodeName))); - } - - final int repoCount = between(1, 3); - for (int i = 0; i < repoCount; i++) { - final String repositoryName = "repo-" + i; - repositories.put(repositoryName, new TrackedRepository(repositoryName, randomRepoPath())); - } - - final int indexCount = between(1, 10); - for (int i = 0; i < indexCount; i++) { - final String indexName = "index-" + i; - indices.put(indexName, new TrackedIndex(indexName)); - } - } - - void shuffleNodes() { - final List newNodes = new ArrayList<>(nodes.values()); - Randomness.shuffle(newNodes); - final String masterNodeName = Optional.ofNullable(cluster.getInstance(ClusterService.class).state().nodes().getMasterNode()) - .map(DiscoveryNode::getName) - .orElse(null); - newNodes.sort(Comparator.comparing(tn -> tn.nodeName.equals(masterNodeName))); - shuffledNodes = newNodes; - } - - public void run() throws InterruptedException { - shuffleNodes(); - - for (TrackedIndex trackedIndex : indices.values()) { - trackedIndex.start(); - } - - for (TrackedRepository trackedRepository : repositories.values()) { - trackedRepository.start(); - } - - final int nodeRestarterCount = between(1, 2); - for (int i = 0; i < nodeRestarterCount; i++) { - startNodeRestarter(); - } - - final int snapshotterCount = between(1, 5); - for (int i = 0; i < snapshotterCount; i++) { - startSnapshotter(); - } - - final int partialSnapshotterCount = between(1, 5); - for (int i = 0; i < partialSnapshotterCount; i++) { - startPartialSnapshotter(); - } - - final int clonerCount = between(0, 5); - for (int i = 0; i < clonerCount; i++) { - startCloner(); - } - - final int deleterCount = between(0, 3); - for (int i = 0; i < deleterCount; i++) { - startSnapshotDeleter(); - } - - final int restorerCount = between(0, 3); - for (int i = 0; i < restorerCount; i++) { - startRestorer(); - } - - final int cleanerCount = between(0, 2); - for (int i = 0; i < cleanerCount; i++) { - startCleaner(); - } - - if (randomBoolean()) { - startNodeShutdownMarker(); - } - - if (completedSnapshotLatch.await(30, TimeUnit.SECONDS)) { - logger.info("--> completed target snapshot count, finishing test"); - } else { - logger.info("--> did not complete target snapshot count in 30s, giving up"); - } - - assertTrue(shouldStop.compareAndSet(false, true)); - final long permitDeadlineMillis = threadPool.relativeTimeInMillis() + TimeUnit.MINUTES.toMillis(2); - - final List failedPermitAcquisitions = new ArrayList<>(); - acquirePermitsAtEnd( - repositories.values().stream().map(n -> Tuple.tuple(n.repositoryName, n.permits)), - failedPermitAcquisitions, - permitDeadlineMillis - ); - acquirePermitsAtEnd( - snapshots.values().stream().map(n -> Tuple.tuple(n.snapshotName, n.permits)), - failedPermitAcquisitions, - permitDeadlineMillis - ); - acquirePermitsAtEnd( - indices.values().stream().map(n -> Tuple.tuple(n.indexName, n.permits)), - failedPermitAcquisitions, - permitDeadlineMillis - ); - acquirePermitsAtEnd( - nodes.values().stream().map(n -> Tuple.tuple(n.nodeName, n.permits)), - failedPermitAcquisitions, - permitDeadlineMillis - ); - - if (failedPermitAcquisitions.isEmpty() == false) { - logger.warn("--> failed to acquire all permits: {}", failedPermitAcquisitions); - logger.info( - "--> current cluster state:\n{}", - Strings.toString(clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) - ); - fail("failed to acquire all permits: " + failedPermitAcquisitions); - } - logger.info("--> acquired all permits"); - - if (ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS) == false) { - logger.warn("--> threadpool termination timed out"); - logger.info( - "--> current cluster state:\n{}", - Strings.toString(clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) - ); - } - } - - private void acquirePermitsAtEnd( - Stream> labelledPermits, - List failedPermitAcquisitions, - long permitDeadlineMillis - ) { - labelledPermits.forEach(labelledPermit -> { - final long remainingMillis = Math.max(1L, permitDeadlineMillis - threadPool.relativeTimeInMillis()); - final String label = labelledPermit.v1(); - logger.info("--> acquiring permit [{}] with timeout of [{}ms]", label, remainingMillis); - try { - if (labelledPermit.v2().tryAcquire(Integer.MAX_VALUE, remainingMillis, TimeUnit.MILLISECONDS)) { - logger.info("--> acquired permit [{}]", label); - } else { - logger.warn("--> failed to acquire permit [{}]", label); - logger.info( - "--> current cluster state:\n{}", - Strings.toString(clusterAdmin().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) - ); - HotThreads.logLocalHotThreads( - logger, - Level.INFO, - "hot threads while failing to acquire permit [" + label + "]", - ReferenceDocs.LOGGING - ); - failedPermitAcquisitions.add(label); - } - } catch (InterruptedException e) { - logger.warn("--> interrupted while acquiring permit [{}]", label); - Thread.currentThread().interrupt(); - logAndFailTest(e); - } - }); - } - - private void enqueueAction(final CheckedRunnable action) { - if (shouldStop.get()) { - return; - } - - threadPool.scheduleUnlessShuttingDown(TimeValue.timeValueMillis(between(1, 500)), clientExecutor, mustSucceed(action)); - } - - private void startRestorer() { - enqueueAction(() -> { - boolean startedRestore = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - final List trackedSnapshots = new ArrayList<>(snapshots.values()); - if (trackedSnapshots.isEmpty()) { - return; - } - - if (localReleasables.add(blockNodeRestarts()) == null) { - return; - } - - final TrackedSnapshot trackedSnapshot = randomFrom(trackedSnapshots); - if (localReleasables.add(trackedSnapshot.tryAcquirePermit()) == null) { - return; - } - - if (snapshots.get(trackedSnapshot.snapshotName) != trackedSnapshot) { - // concurrently removed - return; - } - - final Releasable releaseAll = localReleasables.transfer(); - - logger.info( - "--> listing indices in [{}:{}] in preparation for restoring", - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName - ); - - trackedSnapshot.getSnapshotInfo(client(), mustSucceed(snapshotInfo -> restoreSnapshot(snapshotInfo, releaseAll))); - - startedRestore = true; - } finally { - if (startedRestore == false) { - startRestorer(); - } - } - }); - } - - private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePreviousStep) { - boolean startedRestore = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - localReleasables.add(releasePreviousStep); - - if (shouldStop.get()) { - return; - } - - boolean restoreSpecificIndicesTmp = randomBoolean(); - final List indicesToRestoreList = new ArrayList<>(snapshotInfo.indices().size()); - final List indicesToCloseList = new ArrayList<>(snapshotInfo.indices().size()); - final List indicesToDeleteList = new ArrayList<>(snapshotInfo.indices().size()); - for (String indexName : snapshotInfo.indices()) { - if (snapshotInfo.shardFailures() - .stream() - .anyMatch(snapshotShardFailure -> snapshotShardFailure.getShardId().getIndexName().equals(indexName))) { - - restoreSpecificIndicesTmp = true; - continue; - } - final var trackedIndex = indices.get(indexName); - if (randomBoolean() && localReleasables.add(tryAcquireAllPermits(trackedIndex.permits)) != null) { - - indicesToRestoreList.add(indexName); - - final int snapshotShardCount = snapshotInfo.indexSnapshotDetails().get(indexName).getShardCount(); - final int indexShardCount = trackedIndex.shardCount; - if (snapshotShardCount == indexShardCount - && randomBoolean() - && localReleasables.add(trackedIndex.tryAcquireClosingPermit()) != null) { - indicesToCloseList.add(indexName); - } else { - indicesToDeleteList.add(indexName); - trackedIndex.shardCount = snapshotShardCount; - } - } else { - restoreSpecificIndicesTmp = true; - } - } - final boolean restoreSpecificIndices = restoreSpecificIndicesTmp; - - if (indicesToRestoreList.isEmpty()) { - logger.info( - "--> could not obtain exclusive lock on any indices in [{}:{}] for restore", - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - - return; - } - - final Releasable releaseAll = localReleasables.transfer(); - - final String[] indicesToRestore = indicesToRestoreList.toArray(new String[0]); - final String[] indicesToClose = indicesToCloseList.toArray(new String[0]); - final String[] indicesToDelete = indicesToDeleteList.toArray(new String[0]); - final String indicesToRestoreDescription = (restoreSpecificIndices ? "" : "*=") + Arrays.toString(indicesToRestore); - - if (restoreSpecificIndices == false) { - assertEquals(Set.copyOf(snapshotInfo.indices()), Set.of(indicesToRestore)); - } - - final ListenableFuture closeIndicesStep = new ListenableFuture<>(); - final ListenableFuture deleteIndicesStep = new ListenableFuture<>(); - - if (indicesToClose.length > 0) { - logger.info( - "--> waiting for yellow health of [{}] before closing", - Strings.arrayToCommaDelimitedString(indicesToClose) - ); - - SubscribableListener.newForked( - l -> prepareClusterHealthRequest(indicesToClose).setWaitForYellowStatus().execute(l) - ).addListener(mustSucceed(clusterHealthResponse -> { - assertFalse( - "timed out waiting for yellow state of " + Strings.arrayToCommaDelimitedString(indicesToClose), - clusterHealthResponse.isTimedOut() - ); - - logger.info( - "--> closing indices {} in preparation for restoring {} from [{}:{}]", - indicesToClose, - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - indicesAdmin().prepareClose(indicesToClose).execute(mustSucceed(closeIndexResponse -> { - logger.info( - "--> finished closing indices {} in preparation for restoring {} from [{}:{}]", - indicesToClose, - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - assertTrue(closeIndexResponse.isAcknowledged()); - assertTrue(closeIndexResponse.isShardsAcknowledged()); - closeIndicesStep.onResponse(null); - })); - })); - } else { - closeIndicesStep.onResponse(null); - } - - if (indicesToDelete.length > 0) { - logger.info( - "--> deleting indices {} in preparation for restoring {} from [{}:{}]", - indicesToDelete, - indicesToRestore, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - indicesAdmin().prepareDelete(indicesToDelete).execute(mustSucceed(deleteIndicesResponse -> { - logger.info( - "--> finished deleting indices {} in preparation for restoring {} from [{}:{}]", - indicesToDelete, - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - assertTrue(deleteIndicesResponse.isAcknowledged()); - deleteIndicesStep.onResponse(null); - })); - } else { - deleteIndicesStep.onResponse(null); - } - - closeIndicesStep.addListener(mustSucceed(ignored1 -> deleteIndicesStep.addListener(mustSucceed(ignored2 -> { - - final RestoreSnapshotRequestBuilder restoreSnapshotRequestBuilder = clusterAdmin().prepareRestoreSnapshot( - TEST_REQUEST_TIMEOUT, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - - if (restoreSpecificIndices) { - restoreSnapshotRequestBuilder.setIndices(indicesToRestore); - } - - logger.info( - "--> restoring indices {} from [{}:{}]", - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - - restoreSnapshotRequestBuilder.execute(mustSucceed(restoreSnapshotResponse -> { - logger.info( - "--> triggered restore of indices {} from [{}:{}], waiting for green health", - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - - prepareClusterHealthRequest(indicesToRestore).setWaitForGreenStatus() - .setWaitForNoInitializingShards(true) - .execute(mustSucceed(clusterHealthResponse -> { - - logger.info( - "--> indices {} successfully restored from [{}:{}]", - indicesToRestoreDescription, - snapshotInfo.repository(), - snapshotInfo.snapshotId().getName() - ); - - Releasables.close(releaseAll); - assertFalse(clusterHealthResponse.isTimedOut()); - startRestorer(); - })); - })); - })))); - - startedRestore = true; - } finally { - if (startedRestore == false) { - startRestorer(); - } - } - } - - private void startCloner() { - enqueueAction(() -> { - boolean startedClone = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - final List trackedSnapshots = new ArrayList<>(snapshots.values()); - if (trackedSnapshots.isEmpty()) { - return; - } - - if (localReleasables.add(blockFullClusterRestart()) == null) { - return; - } - - final Client client = localReleasables.add(acquireClient()).getClient(); - - final TrackedSnapshot trackedSnapshot = randomFrom(trackedSnapshots); - if (localReleasables.add(trackedSnapshot.tryAcquirePermit()) == null) { - return; - } - - if (snapshots.get(trackedSnapshot.snapshotName) != trackedSnapshot) { - // concurrently removed - return; - } - - final Releasable releaseAll = localReleasables.transfer(); - - final ListenableFuture> getIndicesStep = new ListenableFuture<>(); - - logger.info( - "--> listing indices in [{}:{}] in preparation for cloning", - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName - ); - - trackedSnapshot.getSnapshotInfo(client, mustSucceed(snapshotInfo -> { - final Set failedShardIndices = snapshotInfo.shardFailures() - .stream() - .map(ShardOperationFailedException::index) - .collect(Collectors.toSet()); - final Set cloneableIndices = new HashSet<>(snapshotInfo.indices()); - cloneableIndices.removeAll(failedShardIndices); - - if (cloneableIndices.isEmpty()) { - getIndicesStep.onResponse(Collections.emptyList()); - return; - } - - if (failedShardIndices.isEmpty() && randomBoolean()) { - getIndicesStep.onResponse(Collections.singletonList("*")); - return; - } - - getIndicesStep.onResponse(randomSubsetOf(between(1, cloneableIndices.size()), cloneableIndices)); - })); - - getIndicesStep.addListener(mustSucceed(indexNames -> { - - if (indexNames.isEmpty()) { - logger.info( - "--> no successful indices in [{}:{}], skipping clone", - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName - ); - Releasables.close(releaseAll); - startCloner(); - return; - } - - final String cloneName = "snapshot-clone-" + snapshotCounter.incrementAndGet(); - - logger.info( - "--> starting clone of [{}:{}] as [{}:{}] with indices {}", - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName, - trackedSnapshot.trackedRepository.repositoryName, - cloneName, - indexNames - ); - - client.admin() - .cluster() - .prepareCloneSnapshot( - TEST_REQUEST_TIMEOUT, - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName, - cloneName - ) - .setIndices(indexNames.toArray(new String[0])) - .execute(mustSucceed(acknowledgedResponse -> { - Releasables.close(releaseAll); - assertTrue(acknowledgedResponse.isAcknowledged()); - completedSnapshotLatch.countDown(); - logger.info( - "--> completed clone of [{}:{}] as [{}:{}]", - trackedSnapshot.trackedRepository.repositoryName, - trackedSnapshot.snapshotName, - trackedSnapshot.trackedRepository.repositoryName, - cloneName - ); - startCloner(); - })); - })); - - startedClone = true; - } finally { - if (startedClone == false) { - startCloner(); - } - } - }); - } - - private void startSnapshotDeleter() { - enqueueAction(() -> { - - boolean startedDeletion = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - if (localReleasables.add(blockFullClusterRestart()) == null) { - return; - } - - final Client client = localReleasables.add(acquireClient()).getClient(); - - final List snapshotNames = new ArrayList<>(); - final TrackedRepository targetRepository = blockSnapshotsFromOneRepository(localReleasables, snapshotNames); - if (targetRepository == null) return; - - logger.info("--> starting deletion of [{}:{}]", targetRepository.repositoryName, snapshotNames); - - final Releasable releaseAll = localReleasables.transfer(); - - client.admin() - .cluster() - .prepareDeleteSnapshot(TEST_REQUEST_TIMEOUT, targetRepository.repositoryName, snapshotNames.toArray(new String[0])) - .execute(mustSucceed(acknowledgedResponse -> { - assertTrue(acknowledgedResponse.isAcknowledged()); - for (String snapshotName : snapshotNames) { - assertThat(snapshots.remove(snapshotName), notNullValue()); - } - Releasables.close(releaseAll); // must only release snapshot after removing it from snapshots map - logger.info("--> completed deletion of [{}:{}]", targetRepository.repositoryName, snapshotNames); - startSnapshotDeleter(); - })); - - startedDeletion = true; - - } finally { - if (startedDeletion == false) { - startSnapshotDeleter(); - } - } - }); - } - - @Nullable // if no blocks could be acquired - private TrackedRepository blockSnapshotsFromOneRepository(TransferableReleasables localReleasables, List snapshotNames) { - final List trackedSnapshots = new ArrayList<>(snapshots.values()); - TrackedRepository targetRepository = null; - Randomness.shuffle(trackedSnapshots); - for (TrackedSnapshot trackedSnapshot : trackedSnapshots) { - if ((targetRepository == null || trackedSnapshot.trackedRepository == targetRepository) - && (snapshotNames.isEmpty() || randomBoolean()) - && localReleasables.add(trackedSnapshot.tryAcquireAllPermits()) != null - && snapshots.get(trackedSnapshot.snapshotName) == trackedSnapshot) { - - targetRepository = trackedSnapshot.trackedRepository; - snapshotNames.add(trackedSnapshot.snapshotName); - } - } - - if (targetRepository != null) { - assertFalse(targetRepository.repositoryName, snapshotNames.isEmpty()); - } - return targetRepository; - } - - private void startCleaner() { - enqueueAction(() -> { - - boolean startedCleanup = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - if (localReleasables.add(blockFullClusterRestart()) == null) { - return; - } - - final Client client = localReleasables.add(acquireClient()).getClient(); - - for (TrackedRepository trackedRepository : repositories.values()) { - // cleanup forbids all concurrent snapshot activity - if (localReleasables.add(tryAcquireAllPermits(trackedRepository.permits)) == null) { - return; - } - } - - final TrackedRepository trackedRepository = randomFrom(repositories.values()); - - final Releasable releaseAll = localReleasables.transfer(); - - logger.info("--> starting cleanup of [{}]", trackedRepository.repositoryName); - client.admin() - .cluster() - .prepareCleanupRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) - .execute(mustSucceed(cleanupRepositoryResponse -> { - final RepositoryCleanupResult result = cleanupRepositoryResponse.result(); - if (result.bytes() > 0L || result.blobs() > 0L) { - // we could legitimately run into dangling blobs as the result of a shard snapshot failing half-way - // through the snapshot because of a concurrent index-close or -delete. The second round of cleanup on - // the same repository however should always find no more dangling blobs and be a no-op since we block all - // concurrent operations on the repository. - client.admin() - .cluster() - .prepareCleanupRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) - .execute(mustSucceed(secondCleanupRepositoryResponse -> { - final RepositoryCleanupResult secondCleanupResult = secondCleanupRepositoryResponse.result(); - if (secondCleanupResult.blobs() == 1) { - // The previous cleanup actually leaves behind a stale index-N blob, so this cleanup removes it - // and reports it in its response. When https://github.com/elastic/elasticsearch/pull/100718 is - // fixed the second cleanup will be a proper no-op and we can remove this lenience -- TODO - } else { - assertThat(Strings.toString(secondCleanupResult), secondCleanupResult.blobs(), equalTo(0L)); - assertThat(Strings.toString(secondCleanupResult), secondCleanupResult.bytes(), equalTo(0L)); - } - Releasables.close(releaseAll); - logger.info("--> completed second cleanup of [{}]", trackedRepository.repositoryName); - startCleaner(); - })); - } else { - Releasables.close(releaseAll); - logger.info("--> completed cleanup of [{}]", trackedRepository.repositoryName); - startCleaner(); - } - })); - - startedCleanup = true; - } finally { - if (startedCleanup == false) { - startCleaner(); - } - } - }); - } - - private void startSnapshotter() { - enqueueAction(() -> { - - boolean startedSnapshot = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - // separate TransferableReleasables for blocking node restarts & index deletion so we can release these blocks and - // permit data node restarts and index deletions as soon as the snapshot starts - final TransferableReleasables releasableAfterStart = new TransferableReleasables(); - localReleasables.add(releasableAfterStart); - - if (releasableAfterStart.add(blockNodeRestarts()) == null) { - return; - } - assertNotNull(localReleasables.add(blockFullClusterRestart())); - final Client client = localReleasables.add(acquireClient()).getClient(); - - final TrackedRepository trackedRepository = randomFrom(repositories.values()); - if (localReleasables.add(tryAcquirePermit(trackedRepository.permits)) == null) { - return; - } - - boolean snapshotSpecificIndicesTmp = randomBoolean(); - final List targetIndexNames = new ArrayList<>(indices.size()); - for (TrackedIndex trackedIndex : indices.values()) { - if (usually() && localReleasables.add(tryAcquirePermit(trackedIndex.permits)) != null) { - targetIndexNames.add(trackedIndex.indexName); - } else { - snapshotSpecificIndicesTmp = true; - } - } - final boolean snapshotSpecificIndices = snapshotSpecificIndicesTmp; - - if (snapshotSpecificIndices && targetIndexNames.isEmpty()) { - return; - } - - final Releasable releaseAll = localReleasables.transfer(); - - final ListenableFuture ensureYellowStep = new ListenableFuture<>(); - - final String snapshotName = "snapshot-" + snapshotCounter.incrementAndGet(); - - logger.info( - "--> waiting for yellow health of [{}] before creating snapshot [{}:{}]", - targetIndexNames, - trackedRepository.repositoryName, - snapshotName - ); - - prepareClusterHealthRequest(targetIndexNames.toArray(String[]::new)).setWaitForYellowStatus().execute(ensureYellowStep); - - ensureYellowStep.addListener(mustSucceed(clusterHealthResponse -> { - assertFalse("timed out waiting for yellow state of " + targetIndexNames, clusterHealthResponse.isTimedOut()); - - logger.info( - "--> take snapshot [{}:{}] with indices [{}{}]", - trackedRepository.repositoryName, - snapshotName, - snapshotSpecificIndices ? "" : "*=", - targetIndexNames - ); - - final CreateSnapshotRequestBuilder createSnapshotRequestBuilder = clusterAdmin().prepareCreateSnapshot( - TEST_REQUEST_TIMEOUT, - trackedRepository.repositoryName, - snapshotName - ); - - if (snapshotSpecificIndices) { - createSnapshotRequestBuilder.setIndices(targetIndexNames.toArray(new String[0])); - } - - if (randomBoolean()) { - createSnapshotRequestBuilder.setWaitForCompletion(true); - createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { - logger.info("--> completed snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); - final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); - assertThat(stringFromSnapshotInfo(snapshotInfo), snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); - Releasables.close(releaseAll); - completedSnapshotLatch.countDown(); - startSnapshotter(); - })); - } else { - createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { - logger.info("--> started snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); - Releasables.close(releasableAfterStart.transfer()); - pollForSnapshotCompletion(client, trackedRepository.repositoryName, snapshotName, releaseAll, () -> { - snapshots.put(snapshotName, new TrackedSnapshot(trackedRepository, snapshotName)); - completedSnapshotLatch.countDown(); - startSnapshotter(); - }); - })); - } - - })); - - startedSnapshot = true; - } finally { - if (startedSnapshot == false) { - startSnapshotter(); - } - } - }); - } - - private void startPartialSnapshotter() { - enqueueAction(() -> { - - boolean startedSnapshot = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - // separate TransferableReleasables for blocking node restarts & index deletion so we can release these blocks and - // permit data node restarts and index deletions as soon as the snapshot starts - final TransferableReleasables releasableAfterStart = new TransferableReleasables(); - localReleasables.add(releasableAfterStart); - - if (releasableAfterStart.add(blockNodeRestarts()) == null) { - return; - } - assertNotNull(localReleasables.add(blockFullClusterRestart())); - final Client client = localReleasables.add(acquireClient()).getClient(); - - final TrackedRepository trackedRepository = randomFrom(repositories.values()); - if (localReleasables.add(tryAcquirePermit(trackedRepository.permits)) == null) { - return; - } - - boolean snapshotSpecificIndicesTmp = randomBoolean(); - final List targetIndexNames = new ArrayList<>(indices.size()); - for (TrackedIndex trackedIndex : indices.values()) { - if (usually() - && releasableAfterStart.add(tryAcquirePermit(trackedIndex.permits)) != null - && localReleasables.add(trackedIndex.tryAcquirePartialSnapshottingPermit()) != null) { - targetIndexNames.add(trackedIndex.indexName); - } else { - snapshotSpecificIndicesTmp = true; - } - } - final boolean snapshotSpecificIndices = snapshotSpecificIndicesTmp; - - if (snapshotSpecificIndices && targetIndexNames.isEmpty()) { - return; - } - - final Releasable releaseAll = localReleasables.transfer(); - - final String snapshotName = "snapshot-partial-" + snapshotCounter.incrementAndGet(); - - logger.info( - "--> take partial snapshot [{}:{}] with indices [{}{}]", - trackedRepository.repositoryName, - snapshotName, - snapshotSpecificIndices ? "" : "*=", - targetIndexNames - ); - - final CreateSnapshotRequestBuilder createSnapshotRequestBuilder = clusterAdmin().prepareCreateSnapshot( - TEST_REQUEST_TIMEOUT, - trackedRepository.repositoryName, - snapshotName - ).setPartial(true); - - if (snapshotSpecificIndices) { - createSnapshotRequestBuilder.setIndices(targetIndexNames.toArray(new String[0])); - } - - final boolean abortSnapshot = randomBoolean(); - final Runnable abortRunnable; - if (abortSnapshot) { - try (TransferableReleasables abortReleasables = new TransferableReleasables()) { - - assertNotNull(abortReleasables.add(blockFullClusterRestart())); - final Client abortClient = abortReleasables.add(acquireClient()).getClient(); - - assertNotNull(abortReleasables.add(tryAcquirePermit(trackedRepository.permits))); - - final DeleteSnapshotRequestBuilder deleteSnapshotRequestBuilder = abortClient.admin() - .cluster() - .prepareDeleteSnapshot(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName, snapshotName); - - final Releasable abortReleasable = abortReleasables.transfer(); - - abortRunnable = mustSucceed(() -> { - logger.info("--> abort/delete snapshot [{}:{}] start", trackedRepository.repositoryName, snapshotName); - deleteSnapshotRequestBuilder.execute(new ActionListener<>() { - @Override - public void onResponse(AcknowledgedResponse acknowledgedResponse) { - logger.info( - "--> abort/delete snapshot [{}:{}] success", - trackedRepository.repositoryName, - snapshotName - ); - Releasables.close(abortReleasable); - assertTrue(acknowledgedResponse.isAcknowledged()); - } - - @Override - public void onFailure(Exception e) { - Releasables.close(abortReleasable); - if (ExceptionsHelper.unwrapCause(e) instanceof SnapshotMissingException) { - // processed before the snapshot even started - logger.info( - "--> abort/delete snapshot [{}:{}] got snapshot missing", - trackedRepository.repositoryName, - snapshotName - ); - } else { - logAndFailTest(e); - } - } - }); - }); - } - } else { - abortRunnable = () -> {}; - } - - createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { - logger.info("--> started partial snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); - Releasables.close(releasableAfterStart.transfer()); - pollForSnapshotCompletion(client, trackedRepository.repositoryName, snapshotName, releaseAll, () -> { - if (abortSnapshot == false) { - snapshots.put(snapshotName, new TrackedSnapshot(trackedRepository, snapshotName)); - completedSnapshotLatch.countDown(); - } - startPartialSnapshotter(); - }); - })); - - abortRunnable.run(); - - startedSnapshot = true; - } finally { - if (startedSnapshot == false) { - startPartialSnapshotter(); - } - } - }); - } - - private void pollForSnapshotCompletion( - Client client, - String repositoryName, - String snapshotName, - Releasable onCompletion, - Runnable onSuccess - ) { - clientExecutor.execute( - mustSucceed( - () -> client.admin() - .cluster() - .prepareGetSnapshots(TEST_REQUEST_TIMEOUT, repositoryName) - .setCurrentSnapshot() - .execute(mustSucceed(getSnapshotsResponse -> { - if (getSnapshotsResponse.getSnapshots() - .stream() - .noneMatch(snapshotInfo -> snapshotInfo.snapshotId().getName().equals(snapshotName))) { - - logger.info("--> snapshot [{}:{}] no longer running", repositoryName, snapshotName); - Releasables.close(onCompletion); - onSuccess.run(); - } else { - pollForSnapshotCompletion(client, repositoryName, snapshotName, onCompletion, onSuccess); - } - })) - ) - ); - } - - private void startNodeRestarter() { - enqueueAction(() -> { - boolean restarting = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - if (usually()) { - return; - } - - final ArrayList trackedNodes = new ArrayList<>(shuffledNodes); - Collections.reverse(trackedNodes); - - for (TrackedNode trackedNode : trackedNodes) { - if (localReleasables.add(tryAcquireAllPermits(trackedNode.permits)) != null) { - - final String nodeName = trackedNode.nodeName; - final Releasable releaseAll = localReleasables.transfer(); - - threadPool.executor(NODE_RESTARTER).execute(mustSucceed(() -> { - logger.info("--> restarting [{}]", nodeName); - cluster.restartNode(nodeName); - logger.info("--> finished restarting [{}]", nodeName); - shuffleNodes(); - Releasables.close(releaseAll); - startNodeRestarter(); - })); - - restarting = true; - return; - } - } - - } finally { - if (restarting == false) { - startNodeRestarter(); - } - } - }); - } - - private void startNodeShutdownMarker() { - enqueueAction(() -> { - boolean rerun = true; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - if (usually()) { - return; - } - if (localReleasables.add(blockFullClusterRestart()) == null) { - return; - } - - final var node = randomFrom(shuffledNodes); - - if (localReleasables.add(tryAcquirePermit(node.permits)) == null) { - return; - } - - final var clusterService = cluster.getCurrentMasterNodeInstance(ClusterService.class); - - if (node.nodeName.equals(clusterService.localNode().getName())) { - return; - } - - logger.info("--> marking [{}] for removal", node); - - SubscribableListener - - .newForked( - l -> clusterService.submitUnbatchedStateUpdateTask( - "mark [" + node + "] for removal", - new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - assertTrue( - Strings.toString(currentState), - currentState.metadata().nodeShutdowns().getAll().isEmpty() - ); - final var discoveryNode = currentState.nodes().resolveNode(node.nodeName); - return currentState.copyAndUpdateMetadata( - mdb -> mdb.putCustom( - NodesShutdownMetadata.TYPE, - new NodesShutdownMetadata( - Map.of( - discoveryNode.getId(), - SingleNodeShutdownMetadata.builder() - .setNodeId(discoveryNode.getId()) - .setNodeEphemeralId(discoveryNode.getEphemeralId()) - .setType(SingleNodeShutdownMetadata.Type.REMOVE) - .setStartedAtMillis(clusterService.threadPool().absoluteTimeInMillis()) - .setReason("test") - .build() - ) - ) - ) - ); - } - - @Override - public void onFailure(Exception e) { - l.onFailure(e); - } - - @Override - public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { - l.onResponse(null); - } - } - ) - ) - - .andThen( - l -> clusterService.submitUnbatchedStateUpdateTask( - "unmark [" + node + "] for removal", - new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - return currentState.copyAndUpdateMetadata( - mdb -> mdb.putCustom(NodesShutdownMetadata.TYPE, NodesShutdownMetadata.EMPTY) - ); - } - - @Override - public void onFailure(Exception e) { - l.onFailure(e); - } - - @Override - public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { - l.onResponse(null); - logger.info("--> unmarked [{}] for removal", node); - } - } - ) - ) - - .addListener( - ActionListener.releaseAfter(mustSucceed(ignored -> startNodeShutdownMarker()), localReleasables.transfer()) - ); - - rerun = false; - } finally { - if (rerun) { - startNodeShutdownMarker(); - } - } - }); - } - - @Nullable // if we couldn't block node restarts - private Releasable blockNodeRestarts() { - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - for (TrackedNode trackedNode : nodes.values()) { - if (localReleasables.add(tryAcquirePermit(trackedNode.getPermits())) == null) { - return null; - } - } - return localReleasables.transfer(); - } - } - - /** - * Try and block the restart of a majority of the master nodes, which therefore prevents a full-cluster restart from occurring. - */ - @Nullable // if we couldn't block enough master node restarts - private Releasable blockFullClusterRestart() { - // Today we block all master failovers to avoid things like TransportMasterNodeAction-led retries which might fail e.g. because - // the snapshot already exists). - - // TODO generalise this so that it succeeds as soon as it's acquired a permit on >1/2 of the master-eligible nodes - final List masterNodes = shuffledNodes.stream().filter(TrackedNode::isMasterNode).toList(); - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - for (TrackedNode trackedNode : masterNodes) { - if (localReleasables.add(tryAcquirePermit(trackedNode.getPermits())) == null) { - return null; - } - } - return localReleasables.transfer(); - } - } - - /** - * Acquire a client (i.e. block the client node from restarting) in a situation where we know that such a block can be obtained, - * since previous acquisitions mean that at least one node is already blocked from restarting. - */ - private ReleasableClient acquireClient() { - for (TrackedNode trackedNode : shuffledNodes) { - final Releasable permit = tryAcquirePermit(trackedNode.getPermits()); - if (permit != null) { - return new ReleasableClient(permit, client(trackedNode.nodeName)); - } - } - - final AssertionError assertionError = new AssertionError("could not acquire client"); - logger.error("acquireClient", assertionError); - throw assertionError; - } - - /** - * Tracks a repository in the cluster, and occasionally removes it and adds it back if no other activity holds any of its permits. - */ - private class TrackedRepository { - - private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); - private final String repositoryName; - private final Path location; - - private TrackedRepository(String repositoryName, Path location) { - this.repositoryName = repositoryName; - this.location = location; - } - - @Override - public String toString() { - return "TrackedRepository[" + repositoryName + "]"; - } - - public void start() { - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - assertNotNull(localReleasables.add(blockNodeRestarts())); - assertNotNull(localReleasables.add(tryAcquireAllPermits(permits))); - final Client client = localReleasables.add(acquireClient()).getClient(); - putRepositoryAndContinue(client, false, localReleasables.transfer()); - } - } - - private void putRepositoryAndContinue(Client client, boolean nodeMightRestart, Releasable releasable) { - logger.info("--> put repo [{}]", repositoryName); - client.admin() - .cluster() - .preparePutRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName) - .setType(FsRepository.TYPE) - .setSettings(Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), location)) - .setVerify(nodeMightRestart == false) - .execute(mustSucceed(acknowledgedResponse -> { - assertTrue(acknowledgedResponse.isAcknowledged()); - logger.info("--> finished put repo [{}]", repositoryName); - Releasables.close(releasable); - scheduleRemoveAndAdd(); - })); - } - - private void scheduleRemoveAndAdd() { - enqueueAction(() -> { - - boolean replacingRepo = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - if (usually()) { - return; - } - - if (localReleasables.add(tryAcquireAllPermits(permits)) == null) { - return; - } - - if (localReleasables.add(blockFullClusterRestart()) == null) { - return; - } - - final var nodeMightRestart = localReleasables.add(blockNodeRestarts()) == null; - - final Client client = localReleasables.add(acquireClient()).getClient(); - - final Releasable releaseAll = localReleasables.transfer(); - - logger.info("--> delete repo [{}]", repositoryName); - clusterAdmin().prepareDeleteRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName) - .execute(mustSucceed(acknowledgedResponse -> { - assertTrue(acknowledgedResponse.isAcknowledged()); - logger.info("--> finished delete repo [{}]", repositoryName); - putRepositoryAndContinue(client, nodeMightRestart, releaseAll); - })); - - replacingRepo = true; - } finally { - if (replacingRepo == false) { - scheduleRemoveAndAdd(); - } - } - }); - } - - } - - private class TrackedIndex { - - private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); - private final String indexName; - - // these fields are only changed when all permits held by the delete/recreate process: - private int shardCount; - private Semaphore docPermits; - - private TrackedIndex(String indexName) { - this.indexName = indexName; - } - - @Override - public String toString() { - return "TrackedIndex[" + indexName + "]"; - } - - public void start() { - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - assertNotNull(localReleasables.add(blockNodeRestarts())); - assertNotNull(localReleasables.add(tryAcquireAllPermits(permits))); - createIndexAndContinue(localReleasables.transfer()); - } - } - - private void createIndexAndContinue(Releasable releasable) { - shardCount = between(1, 5); - docPermits = new Semaphore(between(1000, 3000)); - logger.info("--> create index [{}] with max [{}] docs", indexName, docPermits.availablePermits()); - indicesAdmin().prepareCreate(indexName) - .setSettings(indexSettings(shardCount, between(0, cluster.numDataNodes() - 1))) - .execute(mustSucceed(response -> { - assertTrue(response.isAcknowledged()); - logger.info("--> finished create index [{}]", indexName); - Releasables.close(releasable); - scheduleIndexingAndPossibleDelete(); - })); - } - - private void scheduleIndexingAndPossibleDelete() { - enqueueAction(() -> { - - boolean forked = false; - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - - if (localReleasables.add(blockNodeRestarts()) == null) { - return; - } - - if (usually()) { - // index some more docs - - if (localReleasables.add(tryAcquirePermit(permits)) == null) { - return; - } - - final int maxDocCount = docPermits.drainPermits(); - assert maxDocCount >= 0 : maxDocCount; - if (maxDocCount == 0) { - return; - } - final int docCount = between(1, Math.min(maxDocCount, 200)); - docPermits.release(maxDocCount - docCount); - - final Releasable releaseAll = localReleasables.transfer(); - - final ListenableFuture ensureYellowStep = new ListenableFuture<>(); - - logger.info("--> waiting for yellow health of [{}] prior to indexing [{}] docs", indexName, docCount); - - prepareClusterHealthRequest(indexName).setWaitForYellowStatus().execute(ensureYellowStep); - - final ListenableFuture bulkStep = new ListenableFuture<>(); - - ensureYellowStep.addListener(mustSucceed(clusterHealthResponse -> { - - assertFalse( - "timed out waiting for yellow state of [" + indexName + "]", - clusterHealthResponse.isTimedOut() - ); - - final BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(indexName); - - logger.info("--> indexing [{}] docs into [{}]", docCount, indexName); - - for (int i = 0; i < docCount; i++) { - bulkRequestBuilder.add( - new IndexRequest().source( - jsonBuilder().startObject().field("field-" + between(1, 5), randomAlphaOfLength(10)).endObject() - ) - ); - } - - bulkRequestBuilder.execute(bulkStep); - })); - - bulkStep.addListener(mustSucceed(bulkItemResponses -> { - for (BulkItemResponse bulkItemResponse : bulkItemResponses.getItems()) { - assertNull(bulkItemResponse.getFailure()); - } - - logger.info("--> indexing into [{}] finished", indexName); - - Releasables.close(releaseAll); - scheduleIndexingAndPossibleDelete(); - - })); - - forked = true; - - } else if (localReleasables.add(tryAcquireAllPermits(permits)) != null) { - // delete the index and create a new one - - final Releasable releaseAll = localReleasables.transfer(); - - logger.info("--> deleting index [{}]", indexName); - - indicesAdmin().prepareDelete(indexName).execute(mustSucceed(acknowledgedResponse -> { - logger.info("--> deleting index [{}] finished", indexName); - assertTrue(acknowledgedResponse.isAcknowledged()); - createIndexAndContinue(releaseAll); - })); - - forked = true; - } - } finally { - if (forked == false) { - scheduleIndexingAndPossibleDelete(); - } - } - }); - } - - /** - * We must not close an index while it's being partially snapshotted; this counter tracks the number of ongoing - * close operations (positive) or partial snapshot operations (negative) in order to avoid them happening concurrently. - *

- * This is only a problem for partial snapshots because we release the index permit once a partial snapshot has started. With - * non-partial snapshots we retain the index permit until it completes which blocks other operations. - */ - private final AtomicInteger closingOrPartialSnapshottingCount = new AtomicInteger(); - - private static boolean closingPermitAvailable(int value) { - return value >= 0 && value != Integer.MAX_VALUE; - } - - private static boolean partialSnapshottingPermitAvailable(int value) { - return value <= 0 && value != Integer.MIN_VALUE; - } - - Releasable tryAcquireClosingPermit() { - final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> closingPermitAvailable(c) ? c + 1 : c); - if (closingPermitAvailable(previous)) { - return () -> assertThat(closingOrPartialSnapshottingCount.getAndDecrement(), greaterThan(0)); - } else { - return null; - } - } - - Releasable tryAcquirePartialSnapshottingPermit() { - final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> partialSnapshottingPermitAvailable(c) ? c - 1 : c); - if (partialSnapshottingPermitAvailable(previous)) { - return () -> assertThat(closingOrPartialSnapshottingCount.getAndIncrement(), lessThan(0)); - } else { - return null; - } - } - } - - } - - // Prepares a health request with twice the default (30s) timeout that waits for all cluster tasks to finish as well as all cluster - // nodes before returning - private static ClusterHealthRequestBuilder prepareClusterHealthRequest(String... targetIndexNames) { - return clusterAdmin().prepareHealth(TEST_REQUEST_TIMEOUT, targetIndexNames) - .setTimeout(TimeValue.timeValueSeconds(60)) - .setWaitForNodes(Integer.toString(internalCluster().getNodeNames().length)) - .setWaitForEvents(Priority.LANGUID); - } - - private static String stringFromSnapshotInfo(SnapshotInfo snapshotInfo) { - return Strings.toString((b, p) -> snapshotInfo.toXContent(b, SNAPSHOT_ONLY_FORMAT_PARAMS), true, false); - } - - /** - * A client to a node that is blocked from restarting; close this {@link Releasable} to release the block. - */ - private static class ReleasableClient implements Releasable { - private final Releasable releasable; - private final Client client; - - ReleasableClient(Releasable releasable, Client client) { - this.releasable = releasable; - this.client = client; - } - - @Override - public void close() { - releasable.close(); - } - - Client getClient() { - return client; - } - } - - /** - * Tracks a snapshot taken by the cluster. - */ - private static class TrackedSnapshot { - - private final TrackedCluster.TrackedRepository trackedRepository; - private final String snapshotName; - private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); - private final AtomicReference> snapshotInfoFutureRef = new AtomicReference<>(); - - TrackedSnapshot(TrackedCluster.TrackedRepository trackedRepository, String snapshotName) { - this.trackedRepository = trackedRepository; - this.snapshotName = snapshotName; - } - - /* - * Try and acquire a permit on this snapshot and the underlying repository - */ - Releasable tryAcquirePermit() { - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - if (localReleasables.add(SnapshotStressTestsIT.tryAcquirePermit(trackedRepository.permits)) == null) { - return null; - } - - if (localReleasables.add(SnapshotStressTestsIT.tryAcquirePermit(permits)) == null) { - return null; - } - - return localReleasables.transfer(); - } - } - - Releasable tryAcquireAllPermits() { - try (TransferableReleasables localReleasables = new TransferableReleasables()) { - if (localReleasables.add(SnapshotStressTestsIT.tryAcquirePermit(trackedRepository.permits)) == null) { - return null; - } - - if (localReleasables.add(SnapshotStressTestsIT.tryAcquireAllPermits(permits)) == null) { - return null; - } - - return localReleasables.transfer(); - } - } - - void getSnapshotInfo(Client client, ActionListener listener) { - final SubscribableListener newFuture = new SubscribableListener<>(); - - final boolean firstRunner = snapshotInfoFutureRef.compareAndSet(null, newFuture); - - if (firstRunner == false) { - if (usually()) { - snapshotInfoFutureRef.get().addListener(listener); - return; - } - // else (rarely) get it again, expecting it to be the same - - snapshotInfoFutureRef.get() - .addListener( - mustSucceed( - snapshotInfo1 -> newFuture.addListener( - mustSucceed(snapshotInfo2 -> assertThat(snapshotInfo1, equalTo(snapshotInfo2))) - ) - ) - ); - } - - newFuture.addListener(listener); - - TrackedCluster.logger.info( - "--> getting snapshot info{} for [{}:{}]", - firstRunner ? "" : " again", - trackedRepository.repositoryName, - snapshotName - ); - client.admin() - .cluster() - .prepareGetSnapshots(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) - .setSnapshots(snapshotName) - .execute(mustSucceed(getSnapshotsResponse -> { - assertThat(getSnapshotsResponse.getSnapshots(), hasSize(1)); - final SnapshotInfo snapshotInfo = getSnapshotsResponse.getSnapshots().get(0); - assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName)); - TrackedCluster.logger.info( - "--> got snapshot info for [{}:{}]{}", - trackedRepository.repositoryName, - snapshotName, - firstRunner ? ":\n" + stringFromSnapshotInfo(snapshotInfo) : " again" - ); - newFuture.onResponse(snapshotInfo); - })); - } - } - - /** - * Tracks a node in the cluster. - */ - private static class TrackedNode { - - private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); - private final String nodeName; - private final boolean isMasterNode; - private final boolean isDataNode; - - TrackedNode(String nodeName, boolean isMasterNode, boolean isDataNode) { - this.nodeName = nodeName; - this.isMasterNode = isMasterNode; - this.isDataNode = isDataNode; - } - - Semaphore getPermits() { - return permits; - } - - boolean isMasterNode() { - return isMasterNode; - } - - @Override - public String toString() { - return "TrackedNode{" + nodeName + "}{" + (isMasterNode ? "m" : "") + (isDataNode ? "d" : "") + "}"; - } - } - } diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java new file mode 100644 index 0000000000000..845792196358f --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java @@ -0,0 +1,1842 @@ +/* + * ELASTICSEARCH CONFIDENTIAL + * __________________ + * + * Copyright Elasticsearch B.V. All rights reserved. + * + * NOTICE: All information contained herein is, and remains + * the property of Elasticsearch B.V. and its suppliers, if any. + * The intellectual and technical concepts contained herein + * are proprietary to Elasticsearch B.V. and its suppliers and + * may be covered by U.S. and Foreign Patents, patents in + * process, and are protected by trade secret or copyright + * law. Dissemination of this information or reproduction of + * this material is strictly forbidden unless prior written + * permission is obtained from Elasticsearch B.V. + */ + +package org.elasticsearch.snapshots; + +import org.apache.logging.log4j.Level; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequestBuilder; +import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRequestBuilder; +import org.elasticsearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequestBuilder; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequestBuilder; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.SubscribableListener; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.metadata.NodesShutdownMetadata; +import org.elasticsearch.cluster.metadata.SingleNodeShutdownMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.Randomness; +import org.elasticsearch.common.ReferenceDocs; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.ListenableFuture; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.CheckedRunnable; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; +import org.elasticsearch.monitor.jvm.HotThreads; +import org.elasticsearch.repositories.RepositoryCleanupResult; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.threadpool.ScalingExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; + +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executor; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.lucene.tests.util.LuceneTestCase.usually; +import static org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.SNAPSHOT_ONLY_FORMAT_PARAMS; +import static org.elasticsearch.test.ESIntegTestCase.client; +import static org.elasticsearch.test.ESIntegTestCase.internalCluster; +import static org.elasticsearch.test.ESIntegTestCase.randomRepoPath; +import static org.elasticsearch.test.ESTestCase.TEST_REQUEST_TIMEOUT; +import static org.elasticsearch.test.ESTestCase.assertThat; +import static org.elasticsearch.test.ESTestCase.between; +import static org.elasticsearch.test.ESTestCase.indexSettings; +import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; +import static org.elasticsearch.test.ESTestCase.randomBoolean; +import static org.elasticsearch.test.ESTestCase.randomFrom; +import static org.elasticsearch.test.ESTestCase.randomSubsetOf; +import static org.elasticsearch.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class SnapshotStressTestsHelper { + public static Set nodeNames(Map nodesMap) { + return nodesMap.values().stream().map(DiscoveryNode::getName).collect(Collectors.toSet()); + } + + /** + * Encapsulates a common pattern of trying to acquire a bunch of resources and then transferring ownership elsewhere on success, + * but releasing them on failure. + */ + public static class TransferableReleasables implements Releasable { + + private boolean transferred = false; + private final List releasables = new ArrayList<>(); + + T add(T releasable) { + assert transferred == false : "already transferred"; + releasables.add(releasable); + return releasable; + } + + Releasable transfer() { + assert transferred == false : "already transferred"; + transferred = true; + Collections.reverse(releasables); + return () -> Releasables.close(releasables); + } + + @Override + public void close() { + if (transferred == false) { + Releasables.close(releasables); + } + } + } + + @Nullable // if no permit was acquired + public static Releasable tryAcquirePermit(Semaphore permits) { + if (permits.tryAcquire()) { + return Releasables.releaseOnce(permits::release); + } else { + return null; + } + } + + @Nullable // if not all permits were acquired + public static Releasable tryAcquireAllPermits(Semaphore permits) { + if (permits.tryAcquire(Integer.MAX_VALUE)) { + return Releasables.releaseOnce(() -> permits.release(Integer.MAX_VALUE)); + } else { + return null; + } + } + + public static AbstractRunnable mustSucceed(CheckedRunnable runnable) { + return new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + logAndFailTest(e); + } + + @Override + protected void doRun() throws Exception { + runnable.run(); + } + + @Override + public void onRejection(Exception e) { + // ok, shutting down + } + }; + } + + public static ActionListener mustSucceed(CheckedConsumer consumer) { + return new ActionListener<>() { + @Override + public void onResponse(T t) { + try { + consumer.accept(t); + } catch (Exception e) { + logAndFailTest(e); + } + } + + @Override + public void onFailure(Exception e) { + logAndFailTest(e); + } + }; + } + + public static void logAndFailTest(Exception e) { + final AssertionError assertionError = new AssertionError("unexpected", e); + TrackedCluster.logger.error("test failed", assertionError); + throw assertionError; + } + + /** + * Test harness for snapshot stress tests. + * + * The test performs random operations on the cluster, as if from an external client: + * + * - indexing docs, deleting and re-creating the indices + * - restarting nodes + * - removing and adding repositories + * - taking snapshots (sometimes partial), cloning them, and deleting them + * + * It ensures that these operations should succeed via a system of shared/exclusive locks implemented via permits: acquiring a single + * permit is a shared lock, whereas acquiring all the permits is an exclusive lock. So for instance taking a snapshot acquires a shared + * lock on the repository (permitting other concurrent snapshots/clones/deletes) whereas deleting and recreating the repository requires + * an exclusive lock (ensuring that there are no ongoing operations on the repository, and preventing any new such operations from + * starting). + * + * None of the operations block. If the necessary locks aren't all available then the operation just releases the ones it has acquired + * and tries again later. + * + * The test completes after completing a certain number of snapshots (see {@link #completedSnapshotLatch}) or after a certain time has + * elapsed. + */ + public static class TrackedCluster { + + static final Logger logger = LogManager.getLogger(TrackedCluster.class); + static final String CLIENT = "client"; + static final String NODE_RESTARTER = "node_restarter"; + + private final ThreadPool threadPool = new TestThreadPool( + "TrackedCluster", + // a single thread for "client" activities, to limit the number of activities all starting at once + new ScalingExecutorBuilder(CLIENT, 1, 1, TimeValue.ZERO, true, CLIENT), + new ScalingExecutorBuilder(NODE_RESTARTER, 1, 5, TimeValue.ZERO, true, NODE_RESTARTER) + ); + private final Executor clientExecutor = threadPool.executor(CLIENT); + + private final AtomicBoolean shouldStop = new AtomicBoolean(); + private final InternalTestCluster cluster; + private final Map nodes = ConcurrentCollections.newConcurrentMap(); + private final Map repositories = ConcurrentCollections.newConcurrentMap(); + private final Map indices = ConcurrentCollections.newConcurrentMap(); + private final Map snapshots = ConcurrentCollections.newConcurrentMap(); + + /** + * If we acquire permits on nodes in a completely random order then we tend to block all possible restarts. Instead we always try + * the nodes in the same order, held in this field, so that nodes nearer the end of the list are more likely to be restartable. + * The elected master node is usually last in this list. + */ + private volatile List shuffledNodes; + + private final AtomicInteger snapshotCounter = new AtomicInteger(); + private final CountDownLatch completedSnapshotLatch = new CountDownLatch(30); + + public TrackedCluster(InternalTestCluster cluster, Set masterNodeNames, Set dataNodeNames) { + this.cluster = cluster; + for (String nodeName : cluster.getNodeNames()) { + nodes.put(nodeName, new TrackedNode(nodeName, masterNodeNames.contains(nodeName), dataNodeNames.contains(nodeName))); + } + + final int repoCount = between(1, 3); + for (int i = 0; i < repoCount; i++) { + final String repositoryName = "repo-" + i; + repositories.put(repositoryName, new TrackedRepository(repositoryName, randomRepoPath())); + } + + final int indexCount = between(1, 10); + for (int i = 0; i < indexCount; i++) { + final String indexName = "index-" + i; + indices.put(indexName, new TrackedIndex(indexName)); + } + } + + void shuffleNodes() { + final List newNodes = new ArrayList<>(nodes.values()); + Randomness.shuffle(newNodes); + final String masterNodeName = Optional.ofNullable(cluster.getInstance(ClusterService.class).state().nodes().getMasterNode()) + .map(DiscoveryNode::getName) + .orElse(null); + newNodes.sort(Comparator.comparing(tn -> tn.nodeName.equals(masterNodeName))); + shuffledNodes = newNodes; + } + + public void run() throws InterruptedException { + shuffleNodes(); + + for (TrackedIndex trackedIndex : indices.values()) { + trackedIndex.start(); + } + + for (TrackedRepository trackedRepository : repositories.values()) { + trackedRepository.start(); + } + + final int nodeRestarterCount = between(1, 2); + for (int i = 0; i < nodeRestarterCount; i++) { + startNodeRestarter(); + } + + final int snapshotterCount = between(1, 5); + for (int i = 0; i < snapshotterCount; i++) { + startSnapshotter(); + } + + final int partialSnapshotterCount = between(1, 5); + for (int i = 0; i < partialSnapshotterCount; i++) { + startPartialSnapshotter(); + } + + final int clonerCount = between(0, 5); + for (int i = 0; i < clonerCount; i++) { + startCloner(); + } + + final int deleterCount = between(0, 3); + for (int i = 0; i < deleterCount; i++) { + startSnapshotDeleter(); + } + + final int restorerCount = between(0, 3); + for (int i = 0; i < restorerCount; i++) { + startRestorer(); + } + + final int cleanerCount = between(0, 2); + for (int i = 0; i < cleanerCount; i++) { + startCleaner(); + } + + if (randomBoolean()) { + startNodeShutdownMarker(); + } + + if (completedSnapshotLatch.await(30, TimeUnit.SECONDS)) { + logger.info("--> completed target snapshot count, finishing test"); + } else { + logger.info("--> did not complete target snapshot count in 30s, giving up"); + } + + assertTrue(shouldStop.compareAndSet(false, true)); + final long permitDeadlineMillis = threadPool.relativeTimeInMillis() + TimeUnit.MINUTES.toMillis(2); + + final List failedPermitAcquisitions = new ArrayList<>(); + acquirePermitsAtEnd( + repositories.values().stream().map(n -> Tuple.tuple(n.repositoryName, n.permits)), + failedPermitAcquisitions, + permitDeadlineMillis + ); + acquirePermitsAtEnd( + snapshots.values().stream().map(n -> Tuple.tuple(n.snapshotName, n.permits)), + failedPermitAcquisitions, + permitDeadlineMillis + ); + acquirePermitsAtEnd( + indices.values().stream().map(n -> Tuple.tuple(n.indexName, n.permits)), + failedPermitAcquisitions, + permitDeadlineMillis + ); + acquirePermitsAtEnd( + nodes.values().stream().map(n -> Tuple.tuple(n.nodeName, n.permits)), + failedPermitAcquisitions, + permitDeadlineMillis + ); + + if (failedPermitAcquisitions.isEmpty() == false) { + logger.warn("--> failed to acquire all permits: {}", failedPermitAcquisitions); + logger.info( + "--> current cluster state:\n{}", + Strings.toString(client().admin().cluster().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) + ); + fail("failed to acquire all permits: " + failedPermitAcquisitions); + } + logger.info("--> acquired all permits"); + + if (ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS) == false) { + logger.warn("--> threadpool termination timed out"); + logger.info( + "--> current cluster state:\n{}", + Strings.toString(client().admin().cluster().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) + ); + } + } + + private void acquirePermitsAtEnd( + Stream> labelledPermits, + List failedPermitAcquisitions, + long permitDeadlineMillis + ) { + labelledPermits.forEach(labelledPermit -> { + final long remainingMillis = Math.max(1L, permitDeadlineMillis - threadPool.relativeTimeInMillis()); + final String label = labelledPermit.v1(); + logger.info("--> acquiring permit [{}] with timeout of [{}ms]", label, remainingMillis); + try { + if (labelledPermit.v2().tryAcquire(Integer.MAX_VALUE, remainingMillis, TimeUnit.MILLISECONDS)) { + logger.info("--> acquired permit [{}]", label); + } else { + logger.warn("--> failed to acquire permit [{}]", label); + logger.info( + "--> current cluster state:\n{}", + Strings.toString(client().admin().cluster().prepareState(TEST_REQUEST_TIMEOUT).get().getState(), true, true) + ); + HotThreads.logLocalHotThreads( + logger, + Level.INFO, + "hot threads while failing to acquire permit [" + label + "]", + ReferenceDocs.LOGGING + ); + failedPermitAcquisitions.add(label); + } + } catch (InterruptedException e) { + logger.warn("--> interrupted while acquiring permit [{}]", label); + Thread.currentThread().interrupt(); + logAndFailTest(e); + } + }); + } + + private void enqueueAction(final CheckedRunnable action) { + if (shouldStop.get()) { + return; + } + + threadPool.scheduleUnlessShuttingDown(TimeValue.timeValueMillis(between(1, 500)), clientExecutor, mustSucceed(action)); + } + + private void startRestorer() { + enqueueAction(() -> { + boolean startedRestore = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + final List trackedSnapshots = new ArrayList<>(snapshots.values()); + if (trackedSnapshots.isEmpty()) { + return; + } + + if (localReleasables.add(blockNodeRestarts()) == null) { + return; + } + + final TrackedSnapshot trackedSnapshot = randomFrom(trackedSnapshots); + if (localReleasables.add(trackedSnapshot.tryAcquirePermit()) == null) { + return; + } + + if (snapshots.get(trackedSnapshot.snapshotName) != trackedSnapshot) { + // concurrently removed + return; + } + + final Releasable releaseAll = localReleasables.transfer(); + + logger.info( + "--> listing indices in [{}:{}] in preparation for restoring", + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName + ); + + trackedSnapshot.getSnapshotInfo(client(), mustSucceed(snapshotInfo -> restoreSnapshot(snapshotInfo, releaseAll))); + + startedRestore = true; + } finally { + if (startedRestore == false) { + startRestorer(); + } + } + }); + } + + private void restoreSnapshot(SnapshotInfo snapshotInfo, Releasable releasePreviousStep) { + boolean startedRestore = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = new SnapshotStressTestsHelper.TransferableReleasables() + ) { + localReleasables.add(releasePreviousStep); + + if (shouldStop.get()) { + return; + } + + boolean restoreSpecificIndicesTmp = randomBoolean(); + final List indicesToRestoreList = new ArrayList<>(snapshotInfo.indices().size()); + final List indicesToCloseList = new ArrayList<>(snapshotInfo.indices().size()); + final List indicesToDeleteList = new ArrayList<>(snapshotInfo.indices().size()); + for (String indexName : snapshotInfo.indices()) { + if (snapshotInfo.shardFailures() + .stream() + .anyMatch(snapshotShardFailure -> snapshotShardFailure.getShardId().getIndexName().equals(indexName))) { + + restoreSpecificIndicesTmp = true; + continue; + } + final var trackedIndex = indices.get(indexName); + if (randomBoolean() && localReleasables.add(tryAcquireAllPermits(trackedIndex.permits)) != null) { + + indicesToRestoreList.add(indexName); + + final int snapshotShardCount = snapshotInfo.indexSnapshotDetails().get(indexName).getShardCount(); + final int indexShardCount = trackedIndex.shardCount; + if (snapshotShardCount == indexShardCount + && randomBoolean() + && localReleasables.add(trackedIndex.tryAcquireClosingPermit()) != null) { + indicesToCloseList.add(indexName); + } else { + indicesToDeleteList.add(indexName); + trackedIndex.shardCount = snapshotShardCount; + } + } else { + restoreSpecificIndicesTmp = true; + } + } + final boolean restoreSpecificIndices = restoreSpecificIndicesTmp; + + if (indicesToRestoreList.isEmpty()) { + logger.info( + "--> could not obtain exclusive lock on any indices in [{}:{}] for restore", + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + + return; + } + + final Releasable releaseAll = localReleasables.transfer(); + + final String[] indicesToRestore = indicesToRestoreList.toArray(new String[0]); + final String[] indicesToClose = indicesToCloseList.toArray(new String[0]); + final String[] indicesToDelete = indicesToDeleteList.toArray(new String[0]); + final String indicesToRestoreDescription = (restoreSpecificIndices ? "" : "*=") + Arrays.toString(indicesToRestore); + + if (restoreSpecificIndices == false) { + assertEquals(Set.copyOf(snapshotInfo.indices()), Set.of(indicesToRestore)); + } + + final ListenableFuture closeIndicesStep = new ListenableFuture<>(); + final ListenableFuture deleteIndicesStep = new ListenableFuture<>(); + + if (indicesToClose.length > 0) { + logger.info( + "--> waiting for yellow health of [{}] before closing", + Strings.arrayToCommaDelimitedString(indicesToClose) + ); + + SubscribableListener.newForked( + l -> prepareClusterHealthRequest(indicesToClose).setWaitForYellowStatus().execute(l) + ).addListener(mustSucceed(clusterHealthResponse -> { + assertFalse( + "timed out waiting for yellow state of " + Strings.arrayToCommaDelimitedString(indicesToClose), + clusterHealthResponse.isTimedOut() + ); + + logger.info( + "--> closing indices {} in preparation for restoring {} from [{}:{}]", + indicesToClose, + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + client().admin().indices().prepareClose(indicesToClose).execute(mustSucceed(closeIndexResponse -> { + logger.info( + "--> finished closing indices {} in preparation for restoring {} from [{}:{}]", + indicesToClose, + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + assertTrue(closeIndexResponse.isAcknowledged()); + assertTrue(closeIndexResponse.isShardsAcknowledged()); + closeIndicesStep.onResponse(null); + })); + })); + } else { + closeIndicesStep.onResponse(null); + } + + if (indicesToDelete.length > 0) { + logger.info( + "--> deleting indices {} in preparation for restoring {} from [{}:{}]", + indicesToDelete, + indicesToRestore, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + client().admin().indices().prepareDelete(indicesToDelete).execute(mustSucceed(deleteIndicesResponse -> { + logger.info( + "--> finished deleting indices {} in preparation for restoring {} from [{}:{}]", + indicesToDelete, + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + assertTrue(deleteIndicesResponse.isAcknowledged()); + deleteIndicesStep.onResponse(null); + })); + } else { + deleteIndicesStep.onResponse(null); + } + + closeIndicesStep.addListener(mustSucceed(ignored1 -> deleteIndicesStep.addListener(mustSucceed(ignored2 -> { + + final RestoreSnapshotRequestBuilder restoreSnapshotRequestBuilder = client().admin() + .cluster() + .prepareRestoreSnapshot(TEST_REQUEST_TIMEOUT, snapshotInfo.repository(), snapshotInfo.snapshotId().getName()); + + if (restoreSpecificIndices) { + restoreSnapshotRequestBuilder.setIndices(indicesToRestore); + } + + logger.info( + "--> restoring indices {} from [{}:{}]", + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + + restoreSnapshotRequestBuilder.execute(mustSucceed(restoreSnapshotResponse -> { + logger.info( + "--> triggered restore of indices {} from [{}:{}], waiting for green health", + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + + prepareClusterHealthRequest(indicesToRestore).setWaitForGreenStatus() + .setWaitForNoInitializingShards(true) + .execute(mustSucceed(clusterHealthResponse -> { + + logger.info( + "--> indices {} successfully restored from [{}:{}]", + indicesToRestoreDescription, + snapshotInfo.repository(), + snapshotInfo.snapshotId().getName() + ); + + Releasables.close(releaseAll); + assertFalse(clusterHealthResponse.isTimedOut()); + startRestorer(); + })); + })); + })))); + + startedRestore = true; + } finally { + if (startedRestore == false) { + startRestorer(); + } + } + } + + private void startCloner() { + enqueueAction(() -> { + boolean startedClone = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + final List trackedSnapshots = new ArrayList<>(snapshots.values()); + if (trackedSnapshots.isEmpty()) { + return; + } + + if (localReleasables.add(blockFullClusterRestart()) == null) { + return; + } + + final Client client = localReleasables.add(acquireClient()).getClient(); + + final TrackedSnapshot trackedSnapshot = randomFrom(trackedSnapshots); + if (localReleasables.add(trackedSnapshot.tryAcquirePermit()) == null) { + return; + } + + if (snapshots.get(trackedSnapshot.snapshotName) != trackedSnapshot) { + // concurrently removed + return; + } + + final Releasable releaseAll = localReleasables.transfer(); + + final ListenableFuture> getIndicesStep = new ListenableFuture<>(); + + logger.info( + "--> listing indices in [{}:{}] in preparation for cloning", + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName + ); + + trackedSnapshot.getSnapshotInfo(client, mustSucceed(snapshotInfo -> { + final Set failedShardIndices = snapshotInfo.shardFailures() + .stream() + .map(ShardOperationFailedException::index) + .collect(Collectors.toSet()); + final Set cloneableIndices = new HashSet<>(snapshotInfo.indices()); + cloneableIndices.removeAll(failedShardIndices); + + if (cloneableIndices.isEmpty()) { + getIndicesStep.onResponse(Collections.emptyList()); + return; + } + + if (failedShardIndices.isEmpty() && randomBoolean()) { + getIndicesStep.onResponse(Collections.singletonList("*")); + return; + } + + getIndicesStep.onResponse(randomSubsetOf(between(1, cloneableIndices.size()), cloneableIndices)); + })); + + getIndicesStep.addListener(mustSucceed(indexNames -> { + + if (indexNames.isEmpty()) { + logger.info( + "--> no successful indices in [{}:{}], skipping clone", + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName + ); + Releasables.close(releaseAll); + startCloner(); + return; + } + + final String cloneName = "snapshot-clone-" + snapshotCounter.incrementAndGet(); + + logger.info( + "--> starting clone of [{}:{}] as [{}:{}] with indices {}", + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName, + trackedSnapshot.trackedRepository.repositoryName, + cloneName, + indexNames + ); + + client.admin() + .cluster() + .prepareCloneSnapshot( + TEST_REQUEST_TIMEOUT, + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName, + cloneName + ) + .setIndices(indexNames.toArray(new String[0])) + .execute(mustSucceed(acknowledgedResponse -> { + Releasables.close(releaseAll); + assertTrue(acknowledgedResponse.isAcknowledged()); + completedSnapshotLatch.countDown(); + logger.info( + "--> completed clone of [{}:{}] as [{}:{}]", + trackedSnapshot.trackedRepository.repositoryName, + trackedSnapshot.snapshotName, + trackedSnapshot.trackedRepository.repositoryName, + cloneName + ); + startCloner(); + })); + })); + + startedClone = true; + } finally { + if (startedClone == false) { + startCloner(); + } + } + }); + } + + private void startSnapshotDeleter() { + enqueueAction(() -> { + + boolean startedDeletion = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + if (localReleasables.add(blockFullClusterRestart()) == null) { + return; + } + + final Client client = localReleasables.add(acquireClient()).getClient(); + + final List snapshotNames = new ArrayList<>(); + final TrackedRepository targetRepository = blockSnapshotsFromOneRepository(localReleasables, snapshotNames); + if (targetRepository == null) return; + + logger.info("--> starting deletion of [{}:{}]", targetRepository.repositoryName, snapshotNames); + + final Releasable releaseAll = localReleasables.transfer(); + + client.admin() + .cluster() + .prepareDeleteSnapshot(TEST_REQUEST_TIMEOUT, targetRepository.repositoryName, snapshotNames.toArray(new String[0])) + .execute(mustSucceed(acknowledgedResponse -> { + assertTrue(acknowledgedResponse.isAcknowledged()); + for (String snapshotName : snapshotNames) { + assertThat(snapshots.remove(snapshotName), notNullValue()); + } + Releasables.close(releaseAll); // must only release snapshot after removing it from snapshots map + logger.info("--> completed deletion of [{}:{}]", targetRepository.repositoryName, snapshotNames); + startSnapshotDeleter(); + })); + + startedDeletion = true; + + } finally { + if (startedDeletion == false) { + startSnapshotDeleter(); + } + } + }); + } + + @Nullable // if no blocks could be acquired + private TrackedRepository blockSnapshotsFromOneRepository( + SnapshotStressTestsHelper.TransferableReleasables localReleasables, + List snapshotNames + ) { + final List trackedSnapshots = new ArrayList<>(snapshots.values()); + TrackedRepository targetRepository = null; + Randomness.shuffle(trackedSnapshots); + for (TrackedSnapshot trackedSnapshot : trackedSnapshots) { + if ((targetRepository == null || trackedSnapshot.trackedRepository == targetRepository) + && (snapshotNames.isEmpty() || randomBoolean()) + && localReleasables.add(trackedSnapshot.tryAcquireAllPermits()) != null + && snapshots.get(trackedSnapshot.snapshotName) == trackedSnapshot) { + + targetRepository = trackedSnapshot.trackedRepository; + snapshotNames.add(trackedSnapshot.snapshotName); + } + } + + if (targetRepository != null) { + assertFalse(targetRepository.repositoryName, snapshotNames.isEmpty()); + } + return targetRepository; + } + + private void startCleaner() { + enqueueAction(() -> { + + boolean startedCleanup = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + if (localReleasables.add(blockFullClusterRestart()) == null) { + return; + } + + final Client client = localReleasables.add(acquireClient()).getClient(); + + for (TrackedRepository trackedRepository : repositories.values()) { + // cleanup forbids all concurrent snapshot activity + if (localReleasables.add(tryAcquireAllPermits(trackedRepository.permits)) == null) { + return; + } + } + + final TrackedRepository trackedRepository = randomFrom(repositories.values()); + + final Releasable releaseAll = localReleasables.transfer(); + + logger.info("--> starting cleanup of [{}]", trackedRepository.repositoryName); + client.admin() + .cluster() + .prepareCleanupRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) + .execute(mustSucceed(cleanupRepositoryResponse -> { + final RepositoryCleanupResult result = cleanupRepositoryResponse.result(); + if (result.bytes() > 0L || result.blobs() > 0L) { + // we could legitimately run into dangling blobs as the result of a shard snapshot failing half-way + // through the snapshot because of a concurrent index-close or -delete. The second round of cleanup on + // the same repository however should always find no more dangling blobs and be a no-op since we block all + // concurrent operations on the repository. + client.admin() + .cluster() + .prepareCleanupRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) + .execute(mustSucceed(secondCleanupRepositoryResponse -> { + final RepositoryCleanupResult secondCleanupResult = secondCleanupRepositoryResponse.result(); + if (secondCleanupResult.blobs() == 1) { + // The previous cleanup actually leaves behind a stale index-N blob, so this cleanup removes it + // and reports it in its response. When https://github.com/elastic/elasticsearch/pull/100718 is + // fixed the second cleanup will be a proper no-op and we can remove this lenience -- TODO + } else { + assertThat(Strings.toString(secondCleanupResult), secondCleanupResult.blobs(), equalTo(0L)); + assertThat(Strings.toString(secondCleanupResult), secondCleanupResult.bytes(), equalTo(0L)); + } + Releasables.close(releaseAll); + logger.info("--> completed second cleanup of [{}]", trackedRepository.repositoryName); + startCleaner(); + })); + } else { + Releasables.close(releaseAll); + logger.info("--> completed cleanup of [{}]", trackedRepository.repositoryName); + startCleaner(); + } + })); + + startedCleanup = true; + } finally { + if (startedCleanup == false) { + startCleaner(); + } + } + }); + } + + private void startSnapshotter() { + enqueueAction(() -> { + + boolean startedSnapshot = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + // separate TransferableReleasables for blocking node restarts & index deletion so we can release these blocks and + // permit data node restarts and index deletions as soon as the snapshot starts + final SnapshotStressTestsHelper.TransferableReleasables releasableAfterStart = + new SnapshotStressTestsHelper.TransferableReleasables(); + localReleasables.add(releasableAfterStart); + + if (releasableAfterStart.add(blockNodeRestarts()) == null) { + return; + } + assertNotNull(localReleasables.add(blockFullClusterRestart())); + final Client client = localReleasables.add(acquireClient()).getClient(); + + final TrackedRepository trackedRepository = randomFrom(repositories.values()); + if (localReleasables.add(tryAcquirePermit(trackedRepository.permits)) == null) { + return; + } + + boolean snapshotSpecificIndicesTmp = randomBoolean(); + final List targetIndexNames = new ArrayList<>(indices.size()); + for (TrackedIndex trackedIndex : indices.values()) { + if (usually() && localReleasables.add(tryAcquirePermit(trackedIndex.permits)) != null) { + targetIndexNames.add(trackedIndex.indexName); + } else { + snapshotSpecificIndicesTmp = true; + } + } + final boolean snapshotSpecificIndices = snapshotSpecificIndicesTmp; + + if (snapshotSpecificIndices && targetIndexNames.isEmpty()) { + return; + } + + final Releasable releaseAll = localReleasables.transfer(); + + final ListenableFuture ensureYellowStep = new ListenableFuture<>(); + + final String snapshotName = "snapshot-" + snapshotCounter.incrementAndGet(); + + logger.info( + "--> waiting for yellow health of [{}] before creating snapshot [{}:{}]", + targetIndexNames, + trackedRepository.repositoryName, + snapshotName + ); + + prepareClusterHealthRequest(targetIndexNames.toArray(String[]::new)).setWaitForYellowStatus().execute(ensureYellowStep); + + ensureYellowStep.addListener(mustSucceed(clusterHealthResponse -> { + assertFalse("timed out waiting for yellow state of " + targetIndexNames, clusterHealthResponse.isTimedOut()); + + logger.info( + "--> take snapshot [{}:{}] with indices [{}{}]", + trackedRepository.repositoryName, + snapshotName, + snapshotSpecificIndices ? "" : "*=", + targetIndexNames + ); + + final CreateSnapshotRequestBuilder createSnapshotRequestBuilder = client().admin() + .cluster() + .prepareCreateSnapshot(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName, snapshotName); + + if (snapshotSpecificIndices) { + createSnapshotRequestBuilder.setIndices(targetIndexNames.toArray(new String[0])); + } + + if (randomBoolean()) { + createSnapshotRequestBuilder.setWaitForCompletion(true); + createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { + logger.info("--> completed snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); + final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); + assertThat(stringFromSnapshotInfo(snapshotInfo), snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + Releasables.close(releaseAll); + completedSnapshotLatch.countDown(); + startSnapshotter(); + })); + } else { + createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { + logger.info("--> started snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); + Releasables.close(releasableAfterStart.transfer()); + pollForSnapshotCompletion(client, trackedRepository.repositoryName, snapshotName, releaseAll, () -> { + snapshots.put(snapshotName, new TrackedSnapshot(trackedRepository, snapshotName)); + completedSnapshotLatch.countDown(); + startSnapshotter(); + }); + })); + } + + })); + + startedSnapshot = true; + } finally { + if (startedSnapshot == false) { + startSnapshotter(); + } + } + }); + } + + private void startPartialSnapshotter() { + enqueueAction(() -> { + + boolean startedSnapshot = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + // separate TransferableReleasables for blocking node restarts & index deletion so we can release these blocks and + // permit data node restarts and index deletions as soon as the snapshot starts + final SnapshotStressTestsHelper.TransferableReleasables releasableAfterStart = + new SnapshotStressTestsHelper.TransferableReleasables(); + localReleasables.add(releasableAfterStart); + + if (releasableAfterStart.add(blockNodeRestarts()) == null) { + return; + } + assertNotNull(localReleasables.add(blockFullClusterRestart())); + final Client client = localReleasables.add(acquireClient()).getClient(); + + final TrackedRepository trackedRepository = randomFrom(repositories.values()); + if (localReleasables.add(tryAcquirePermit(trackedRepository.permits)) == null) { + return; + } + + boolean snapshotSpecificIndicesTmp = randomBoolean(); + final List targetIndexNames = new ArrayList<>(indices.size()); + for (TrackedIndex trackedIndex : indices.values()) { + if (usually() + && releasableAfterStart.add(tryAcquirePermit(trackedIndex.permits)) != null + && localReleasables.add(trackedIndex.tryAcquirePartialSnapshottingPermit()) != null) { + targetIndexNames.add(trackedIndex.indexName); + } else { + snapshotSpecificIndicesTmp = true; + } + } + final boolean snapshotSpecificIndices = snapshotSpecificIndicesTmp; + + if (snapshotSpecificIndices && targetIndexNames.isEmpty()) { + return; + } + + final Releasable releaseAll = localReleasables.transfer(); + + final String snapshotName = "snapshot-partial-" + snapshotCounter.incrementAndGet(); + + logger.info( + "--> take partial snapshot [{}:{}] with indices [{}{}]", + trackedRepository.repositoryName, + snapshotName, + snapshotSpecificIndices ? "" : "*=", + targetIndexNames + ); + + final CreateSnapshotRequestBuilder createSnapshotRequestBuilder = client().admin() + .cluster() + .prepareCreateSnapshot(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName, snapshotName) + .setPartial(true); + + if (snapshotSpecificIndices) { + createSnapshotRequestBuilder.setIndices(targetIndexNames.toArray(new String[0])); + } + + final boolean abortSnapshot = randomBoolean(); + final Runnable abortRunnable; + if (abortSnapshot) { + try ( + SnapshotStressTestsHelper.TransferableReleasables abortReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + assertNotNull(abortReleasables.add(blockFullClusterRestart())); + final Client abortClient = abortReleasables.add(acquireClient()).getClient(); + + assertNotNull(abortReleasables.add(tryAcquirePermit(trackedRepository.permits))); + + final DeleteSnapshotRequestBuilder deleteSnapshotRequestBuilder = abortClient.admin() + .cluster() + .prepareDeleteSnapshot(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName, snapshotName); + + final Releasable abortReleasable = abortReleasables.transfer(); + + abortRunnable = mustSucceed(() -> { + logger.info("--> abort/delete snapshot [{}:{}] start", trackedRepository.repositoryName, snapshotName); + deleteSnapshotRequestBuilder.execute(new ActionListener<>() { + @Override + public void onResponse(AcknowledgedResponse acknowledgedResponse) { + logger.info( + "--> abort/delete snapshot [{}:{}] success", + trackedRepository.repositoryName, + snapshotName + ); + Releasables.close(abortReleasable); + assertTrue(acknowledgedResponse.isAcknowledged()); + } + + @Override + public void onFailure(Exception e) { + Releasables.close(abortReleasable); + if (ExceptionsHelper.unwrapCause(e) instanceof SnapshotMissingException) { + // processed before the snapshot even started + logger.info( + "--> abort/delete snapshot [{}:{}] got snapshot missing", + trackedRepository.repositoryName, + snapshotName + ); + } else { + logAndFailTest(e); + } + } + }); + }); + } + } else { + abortRunnable = () -> {}; + } + + createSnapshotRequestBuilder.execute(mustSucceed(createSnapshotResponse -> { + logger.info("--> started partial snapshot [{}:{}]", trackedRepository.repositoryName, snapshotName); + Releasables.close(releasableAfterStart.transfer()); + pollForSnapshotCompletion(client, trackedRepository.repositoryName, snapshotName, releaseAll, () -> { + if (abortSnapshot == false) { + snapshots.put(snapshotName, new TrackedSnapshot(trackedRepository, snapshotName)); + completedSnapshotLatch.countDown(); + } + startPartialSnapshotter(); + }); + })); + + abortRunnable.run(); + + startedSnapshot = true; + } finally { + if (startedSnapshot == false) { + startPartialSnapshotter(); + } + } + }); + } + + private void pollForSnapshotCompletion( + Client client, + String repositoryName, + String snapshotName, + Releasable onCompletion, + Runnable onSuccess + ) { + clientExecutor.execute( + mustSucceed( + () -> client.admin() + .cluster() + .prepareGetSnapshots(TEST_REQUEST_TIMEOUT, repositoryName) + .setCurrentSnapshot() + .execute(mustSucceed(getSnapshotsResponse -> { + if (getSnapshotsResponse.getSnapshots() + .stream() + .noneMatch(snapshotInfo -> snapshotInfo.snapshotId().getName().equals(snapshotName))) { + + logger.info("--> snapshot [{}:{}] no longer running", repositoryName, snapshotName); + Releasables.close(onCompletion); + onSuccess.run(); + } else { + pollForSnapshotCompletion(client, repositoryName, snapshotName, onCompletion, onSuccess); + } + })) + ) + ); + } + + private void startNodeRestarter() { + enqueueAction(() -> { + boolean restarting = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + if (usually()) { + return; + } + + final ArrayList trackedNodes = new ArrayList<>(shuffledNodes); + Collections.reverse(trackedNodes); + + for (TrackedNode trackedNode : trackedNodes) { + if (localReleasables.add(tryAcquireAllPermits(trackedNode.permits)) != null) { + + final String nodeName = trackedNode.nodeName; + final Releasable releaseAll = localReleasables.transfer(); + + threadPool.executor(NODE_RESTARTER).execute(mustSucceed(() -> { + logger.info("--> restarting [{}]", nodeName); + cluster.restartNode(nodeName); + logger.info("--> finished restarting [{}]", nodeName); + shuffleNodes(); + Releasables.close(releaseAll); + startNodeRestarter(); + })); + + restarting = true; + return; + } + } + + } finally { + if (restarting == false) { + startNodeRestarter(); + } + } + }); + } + + private void startNodeShutdownMarker() { + enqueueAction(() -> { + boolean rerun = true; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + if (usually()) { + return; + } + if (localReleasables.add(blockFullClusterRestart()) == null) { + return; + } + + final var node = randomFrom(shuffledNodes); + + if (localReleasables.add(tryAcquirePermit(node.permits)) == null) { + return; + } + + final var clusterService = cluster.getCurrentMasterNodeInstance(ClusterService.class); + + if (node.nodeName.equals(clusterService.localNode().getName())) { + return; + } + + logger.info("--> marking [{}] for removal", node); + + SubscribableListener + + .newForked( + l -> clusterService.submitUnbatchedStateUpdateTask( + "mark [" + node + "] for removal", + new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + assertTrue( + Strings.toString(currentState), + currentState.metadata().nodeShutdowns().getAll().isEmpty() + ); + final var discoveryNode = currentState.nodes().resolveNode(node.nodeName); + return currentState.copyAndUpdateMetadata( + mdb -> mdb.putCustom( + NodesShutdownMetadata.TYPE, + new NodesShutdownMetadata( + Map.of( + discoveryNode.getId(), + SingleNodeShutdownMetadata.builder() + .setNodeId(discoveryNode.getId()) + .setNodeEphemeralId(discoveryNode.getEphemeralId()) + .setType(SingleNodeShutdownMetadata.Type.REMOVE) + .setStartedAtMillis(clusterService.threadPool().absoluteTimeInMillis()) + .setReason("test") + .build() + ) + ) + ) + ); + } + + @Override + public void onFailure(Exception e) { + l.onFailure(e); + } + + @Override + public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { + l.onResponse(null); + } + } + ) + ) + + .andThen( + l -> clusterService.submitUnbatchedStateUpdateTask( + "unmark [" + node + "] for removal", + new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + return currentState.copyAndUpdateMetadata( + mdb -> mdb.putCustom(NodesShutdownMetadata.TYPE, NodesShutdownMetadata.EMPTY) + ); + } + + @Override + public void onFailure(Exception e) { + l.onFailure(e); + } + + @Override + public void clusterStateProcessed(ClusterState initialState, ClusterState newState) { + l.onResponse(null); + logger.info("--> unmarked [{}] for removal", node); + } + } + ) + ) + + .addListener( + ActionListener.releaseAfter(mustSucceed(ignored -> startNodeShutdownMarker()), localReleasables.transfer()) + ); + + rerun = false; + } finally { + if (rerun) { + startNodeShutdownMarker(); + } + } + }); + } + + @Nullable // if we couldn't block node restarts + private Releasable blockNodeRestarts() { + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = new SnapshotStressTestsHelper.TransferableReleasables() + ) { + for (TrackedNode trackedNode : nodes.values()) { + if (localReleasables.add(tryAcquirePermit(trackedNode.getPermits())) == null) { + return null; + } + } + return localReleasables.transfer(); + } + } + + /** + * Try and block the restart of a majority of the master nodes, which therefore prevents a full-cluster restart from occurring. + */ + @Nullable // if we couldn't block enough master node restarts + private Releasable blockFullClusterRestart() { + // Today we block all master failovers to avoid things like TransportMasterNodeAction-led retries which might fail e.g. because + // the snapshot already exists). + + // TODO generalise this so that it succeeds as soon as it's acquired a permit on >1/2 of the master-eligible nodes + final List masterNodes = shuffledNodes.stream().filter(TrackedNode::isMasterNode).toList(); + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = new SnapshotStressTestsHelper.TransferableReleasables() + ) { + for (TrackedNode trackedNode : masterNodes) { + if (localReleasables.add(tryAcquirePermit(trackedNode.getPermits())) == null) { + return null; + } + } + return localReleasables.transfer(); + } + } + + /** + * Acquire a client (i.e. block the client node from restarting) in a situation where we know that such a block can be obtained, + * since previous acquisitions mean that at least one node is already blocked from restarting. + */ + private ReleasableClient acquireClient() { + for (TrackedNode trackedNode : shuffledNodes) { + final Releasable permit = tryAcquirePermit(trackedNode.getPermits()); + if (permit != null) { + return new ReleasableClient(permit, client(trackedNode.nodeName)); + } + } + + final AssertionError assertionError = new AssertionError("could not acquire client"); + logger.error("acquireClient", assertionError); + throw assertionError; + } + + /** + * Tracks a repository in the cluster, and occasionally removes it and adds it back if no other activity holds any of its permits. + */ + private class TrackedRepository { + + private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); + private final String repositoryName; + private final Path location; + + private TrackedRepository(String repositoryName, Path location) { + this.repositoryName = repositoryName; + this.location = location; + } + + @Override + public String toString() { + return "TrackedRepository[" + repositoryName + "]"; + } + + public void start() { + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + assertNotNull(localReleasables.add(blockNodeRestarts())); + assertNotNull(localReleasables.add(tryAcquireAllPermits(permits))); + final Client client = localReleasables.add(acquireClient()).getClient(); + putRepositoryAndContinue(client, false, localReleasables.transfer()); + } + } + + private void putRepositoryAndContinue(Client client, boolean nodeMightRestart, Releasable releasable) { + logger.info("--> put repo [{}]", repositoryName); + client.admin() + .cluster() + .preparePutRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName) + .setType(FsRepository.TYPE) + .setSettings(Settings.builder().put(FsRepository.LOCATION_SETTING.getKey(), location)) + .setVerify(nodeMightRestart == false) + .execute(mustSucceed(acknowledgedResponse -> { + assertTrue(acknowledgedResponse.isAcknowledged()); + logger.info("--> finished put repo [{}]", repositoryName); + Releasables.close(releasable); + scheduleRemoveAndAdd(); + })); + } + + private void scheduleRemoveAndAdd() { + enqueueAction(() -> { + + boolean replacingRepo = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + if (usually()) { + return; + } + + if (localReleasables.add(tryAcquireAllPermits(permits)) == null) { + return; + } + + if (localReleasables.add(blockFullClusterRestart()) == null) { + return; + } + + final var nodeMightRestart = localReleasables.add(blockNodeRestarts()) == null; + + final Client client = localReleasables.add(acquireClient()).getClient(); + + final Releasable releaseAll = localReleasables.transfer(); + + logger.info("--> delete repo [{}]", repositoryName); + client().admin() + .cluster() + .prepareDeleteRepository(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT, repositoryName) + .execute(mustSucceed(acknowledgedResponse -> { + assertTrue(acknowledgedResponse.isAcknowledged()); + logger.info("--> finished delete repo [{}]", repositoryName); + putRepositoryAndContinue(client, nodeMightRestart, releaseAll); + })); + + replacingRepo = true; + } finally { + if (replacingRepo == false) { + scheduleRemoveAndAdd(); + } + } + }); + } + + } + + private class TrackedIndex { + + private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); + private final String indexName; + + // these fields are only changed when all permits held by the delete/recreate process: + private int shardCount; + private Semaphore docPermits; + + private TrackedIndex(String indexName) { + this.indexName = indexName; + } + + @Override + public String toString() { + return "TrackedIndex[" + indexName + "]"; + } + + public void start() { + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + assertNotNull(localReleasables.add(blockNodeRestarts())); + assertNotNull(localReleasables.add(tryAcquireAllPermits(permits))); + createIndexAndContinue(localReleasables.transfer()); + } + } + + private void createIndexAndContinue(Releasable releasable) { + shardCount = between(1, 5); + docPermits = new Semaphore(between(1000, 3000)); + logger.info("--> create index [{}] with max [{}] docs", indexName, docPermits.availablePermits()); + client().admin() + .indices() + .prepareCreate(indexName) + .setSettings(indexSettings(shardCount, between(0, cluster.numDataNodes() - 1))) + .execute(mustSucceed(response -> { + assertTrue(response.isAcknowledged()); + logger.info("--> finished create index [{}]", indexName); + Releasables.close(releasable); + scheduleIndexingAndPossibleDelete(); + })); + } + + private void scheduleIndexingAndPossibleDelete() { + enqueueAction(() -> { + + boolean forked = false; + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = + new SnapshotStressTestsHelper.TransferableReleasables() + ) { + + if (localReleasables.add(blockNodeRestarts()) == null) { + return; + } + + if (usually()) { + // index some more docs + + if (localReleasables.add(tryAcquirePermit(permits)) == null) { + return; + } + + final int maxDocCount = docPermits.drainPermits(); + assert maxDocCount >= 0 : maxDocCount; + if (maxDocCount == 0) { + return; + } + final int docCount = between(1, Math.min(maxDocCount, 200)); + docPermits.release(maxDocCount - docCount); + + final Releasable releaseAll = localReleasables.transfer(); + + final ListenableFuture ensureYellowStep = new ListenableFuture<>(); + + logger.info("--> waiting for yellow health of [{}] prior to indexing [{}] docs", indexName, docCount); + + prepareClusterHealthRequest(indexName).setWaitForYellowStatus().execute(ensureYellowStep); + + final ListenableFuture bulkStep = new ListenableFuture<>(); + + ensureYellowStep.addListener(mustSucceed(clusterHealthResponse -> { + + assertFalse( + "timed out waiting for yellow state of [" + indexName + "]", + clusterHealthResponse.isTimedOut() + ); + + final BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(indexName); + + logger.info("--> indexing [{}] docs into [{}]", docCount, indexName); + + for (int i = 0; i < docCount; i++) { + bulkRequestBuilder.add( + new IndexRequest().source( + jsonBuilder().startObject().field("field-" + between(1, 5), randomAlphaOfLength(10)).endObject() + ) + ); + } + + bulkRequestBuilder.execute(bulkStep); + })); + + bulkStep.addListener(mustSucceed(bulkItemResponses -> { + for (BulkItemResponse bulkItemResponse : bulkItemResponses.getItems()) { + assertNull(bulkItemResponse.getFailure()); + } + + logger.info("--> indexing into [{}] finished", indexName); + + Releasables.close(releaseAll); + scheduleIndexingAndPossibleDelete(); + + })); + + forked = true; + + } else if (localReleasables.add(tryAcquireAllPermits(permits)) != null) { + // delete the index and create a new one + + final Releasable releaseAll = localReleasables.transfer(); + + logger.info("--> deleting index [{}]", indexName); + + client().admin().indices().prepareDelete(indexName).execute(mustSucceed(acknowledgedResponse -> { + logger.info("--> deleting index [{}] finished", indexName); + assertTrue(acknowledgedResponse.isAcknowledged()); + createIndexAndContinue(releaseAll); + })); + + forked = true; + } + } finally { + if (forked == false) { + scheduleIndexingAndPossibleDelete(); + } + } + }); + } + + /** + * We must not close an index while it's being partially snapshotted; this counter tracks the number of ongoing + * close operations (positive) or partial snapshot operations (negative) in order to avoid them happening concurrently. + *

+ * This is only a problem for partial snapshots because we release the index permit once a partial snapshot has started. With + * non-partial snapshots we retain the index permit until it completes which blocks other operations. + */ + private final AtomicInteger closingOrPartialSnapshottingCount = new AtomicInteger(); + + public static boolean closingPermitAvailable(int value) { + return value >= 0 && value != Integer.MAX_VALUE; + } + + public static boolean partialSnapshottingPermitAvailable(int value) { + return value <= 0 && value != Integer.MIN_VALUE; + } + + Releasable tryAcquireClosingPermit() { + final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> closingPermitAvailable(c) ? c + 1 : c); + if (closingPermitAvailable(previous)) { + return () -> assertThat(closingOrPartialSnapshottingCount.getAndDecrement(), greaterThan(0)); + } else { + return null; + } + } + + Releasable tryAcquirePartialSnapshottingPermit() { + final var previous = closingOrPartialSnapshottingCount.getAndUpdate(c -> partialSnapshottingPermitAvailable(c) ? c - 1 : c); + if (partialSnapshottingPermitAvailable(previous)) { + return () -> assertThat(closingOrPartialSnapshottingCount.getAndIncrement(), lessThan(0)); + } else { + return null; + } + } + } + + } + + // Prepares a health request with twice the default (30s) timeout that waits for all cluster tasks to finish as well as all cluster + // nodes before returning + public static ClusterHealthRequestBuilder prepareClusterHealthRequest(String... targetIndexNames) { + return client().admin() + .cluster() + .prepareHealth(TEST_REQUEST_TIMEOUT, targetIndexNames) + .setTimeout(TimeValue.timeValueSeconds(60)) + .setWaitForNodes(Integer.toString(internalCluster().getNodeNames().length)) + .setWaitForEvents(Priority.LANGUID); + } + + public static String stringFromSnapshotInfo(SnapshotInfo snapshotInfo) { + return Strings.toString((b, p) -> snapshotInfo.toXContent(b, SNAPSHOT_ONLY_FORMAT_PARAMS), true, false); + } + + /** + * A client to a node that is blocked from restarting; close this {@link Releasable} to release the block. + */ + public static class ReleasableClient implements Releasable { + private final Releasable releasable; + private final Client client; + + ReleasableClient(Releasable releasable, Client client) { + this.releasable = releasable; + this.client = client; + } + + @Override + public void close() { + releasable.close(); + } + + Client getClient() { + return client; + } + } + + /** + * Tracks a snapshot taken by the cluster. + */ + public static class TrackedSnapshot { + + private final TrackedCluster.TrackedRepository trackedRepository; + private final String snapshotName; + private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); + private final AtomicReference> snapshotInfoFutureRef = new AtomicReference<>(); + + TrackedSnapshot(TrackedCluster.TrackedRepository trackedRepository, String snapshotName) { + this.trackedRepository = trackedRepository; + this.snapshotName = snapshotName; + } + + /* + * Try and acquire a permit on this snapshot and the underlying repository + */ + Releasable tryAcquirePermit() { + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = new SnapshotStressTestsHelper.TransferableReleasables() + ) { + if (localReleasables.add(SnapshotStressTestsHelper.tryAcquirePermit(trackedRepository.permits)) == null) { + return null; + } + + if (localReleasables.add(SnapshotStressTestsHelper.tryAcquirePermit(permits)) == null) { + return null; + } + + return localReleasables.transfer(); + } + } + + Releasable tryAcquireAllPermits() { + try ( + SnapshotStressTestsHelper.TransferableReleasables localReleasables = new SnapshotStressTestsHelper.TransferableReleasables() + ) { + if (localReleasables.add(SnapshotStressTestsHelper.tryAcquirePermit(trackedRepository.permits)) == null) { + return null; + } + + if (localReleasables.add(SnapshotStressTestsHelper.tryAcquireAllPermits(permits)) == null) { + return null; + } + + return localReleasables.transfer(); + } + } + + void getSnapshotInfo(Client client, ActionListener listener) { + final SubscribableListener newFuture = new SubscribableListener<>(); + + final boolean firstRunner = snapshotInfoFutureRef.compareAndSet(null, newFuture); + + if (firstRunner == false) { + if (usually()) { + snapshotInfoFutureRef.get().addListener(listener); + return; + } + // else (rarely) get it again, expecting it to be the same + + snapshotInfoFutureRef.get() + .addListener( + mustSucceed( + snapshotInfo1 -> newFuture.addListener( + mustSucceed(snapshotInfo2 -> assertThat(snapshotInfo1, equalTo(snapshotInfo2))) + ) + ) + ); + } + + newFuture.addListener(listener); + + TrackedCluster.logger.info( + "--> getting snapshot info{} for [{}:{}]", + firstRunner ? "" : " again", + trackedRepository.repositoryName, + snapshotName + ); + client.admin() + .cluster() + .prepareGetSnapshots(TEST_REQUEST_TIMEOUT, trackedRepository.repositoryName) + .setSnapshots(snapshotName) + .execute(mustSucceed(getSnapshotsResponse -> { + assertThat(getSnapshotsResponse.getSnapshots(), hasSize(1)); + final SnapshotInfo snapshotInfo = getSnapshotsResponse.getSnapshots().get(0); + assertThat(snapshotInfo.snapshotId().getName(), equalTo(snapshotName)); + TrackedCluster.logger.info( + "--> got snapshot info for [{}:{}]{}", + trackedRepository.repositoryName, + snapshotName, + firstRunner ? ":\n" + stringFromSnapshotInfo(snapshotInfo) : " again" + ); + newFuture.onResponse(snapshotInfo); + })); + } + } + + /** + * Tracks a node in the cluster. + */ + public static class TrackedNode { + + private final Semaphore permits = new Semaphore(Integer.MAX_VALUE); + private final String nodeName; + private final boolean isMasterNode; + private final boolean isDataNode; + + TrackedNode(String nodeName, boolean isMasterNode, boolean isDataNode) { + this.nodeName = nodeName; + this.isMasterNode = isMasterNode; + this.isDataNode = isDataNode; + } + + Semaphore getPermits() { + return permits; + } + + boolean isMasterNode() { + return isMasterNode; + } + + @Override + public String toString() { + return "TrackedNode{" + nodeName + "}{" + (isMasterNode ? "m" : "") + (isDataNode ? "d" : "") + "}"; + } + } +} From 4a25f3ea42299d9eb905d338aef002c20017cfb7 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Mon, 27 Oct 2025 20:35:04 +1100 Subject: [PATCH 4/8] wire factory interface --- .../elasticsearch/node/NodeConstruction.java | 7 ++- ...calPrimarySnapshotShardContextFactory.java | 49 ++++++++++++++++--- 2 files changed, 47 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java index 964a8636f7f07..50ce3787a0f73 100644 --- a/server/src/main/java/org/elasticsearch/node/NodeConstruction.java +++ b/server/src/main/java/org/elasticsearch/node/NodeConstruction.java @@ -195,6 +195,7 @@ import org.elasticsearch.repositories.RepositoriesModule; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.SnapshotMetrics; +import org.elasticsearch.repositories.SnapshotShardContextFactory; import org.elasticsearch.reservedstate.ReservedClusterStateHandler; import org.elasticsearch.reservedstate.ReservedProjectStateHandler; import org.elasticsearch.reservedstate.ReservedStateHandlerProvider; @@ -1179,13 +1180,17 @@ public Map queryFields() { projectResolver.supportsMultipleProjects(), snapshotMetrics ); + SnapshotShardsService snapshotShardsService = new SnapshotShardsService( settings, clusterService, repositoriesService, transportService, indicesService, - new LocalPrimarySnapshotShardContextFactory(indicesService) + pluginsService.loadSingletonServiceProvider( + SnapshotShardContextFactory.class, + () -> new LocalPrimarySnapshotShardContextFactory(indicesService) + ) ); final CachingSnapshotAndShardByStateMetricsService cachingSnapshotAndShardByStateMetricsService = new CachingSnapshotAndShardByStateMetricsService(clusterService); diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java index 01df5e136b13f..c75aed2acbe22 100644 --- a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java @@ -34,6 +34,10 @@ public class LocalPrimarySnapshotShardContextFactory implements SnapshotShardCon private final IndicesService indicesService; + public LocalPrimarySnapshotShardContextFactory() { + throw new IllegalStateException("This no arg constructor only exists for SPI validation"); + } + public LocalPrimarySnapshotShardContextFactory(IndicesService indicesService) { this.indicesService = indicesService; } @@ -72,20 +76,21 @@ public SubscribableListener asyncCreate( final var shardStateId = getShardStateId(indexShard, snapshotIndexCommit.indexCommit()); // not aborted so indexCommit() ok snapshotStatus.addAbortListener(makeAbortListener(indexShard.shardId(), snapshot, snapshotIndexCommit)); snapshotStatus.ensureNotAborted(); - final var snapshotShardContext = new LocalPrimarySnapshotShardContext( - indexShard.store(), - indexShard.mapperService(), - snapshot.getSnapshotId(), + + final var snapshotShardContextListener = doAsyncCreate( + shardId, + snapshot, indexId, - snapshotIndexCommit, - shardStateId, snapshotStatus, repositoryMetaVersion, snapshotStartTime, - listener + listener, + indexShard, + snapshotIndexCommit, + shardStateId ); snapshotIndexCommit = null; - return SubscribableListener.newSucceeded(snapshotShardContext); + return snapshotShardContextListener; } finally { if (snapshotIndexCommit != null) { snapshotIndexCommit.closingBefore(new ActionListener() { @@ -102,6 +107,34 @@ public void onFailure(Exception e) { } } + protected SubscribableListener doAsyncCreate( + ShardId shardId, + Snapshot snapshot, + IndexId indexId, + IndexShardSnapshotStatus snapshotStatus, + IndexVersion repositoryMetaVersion, + long snapshotStartTime, + ActionListener listener, + IndexShard indexShard, + SnapshotIndexCommit snapshotIndexCommit, + String shardStateId + ) { + return SubscribableListener.newSucceeded( + new LocalPrimarySnapshotShardContext( + indexShard.store(), + indexShard.mapperService(), + snapshot.getSnapshotId(), + indexId, + snapshotIndexCommit, + shardStateId, + snapshotStatus, + repositoryMetaVersion, + snapshotStartTime, + listener + ) + ); + } + static ActionListener makeAbortListener( ShardId shardId, Snapshot snapshot, From 484ff89191e9ba3f3ebc135821a70d8fcf56b760 Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 6 Nov 2025 00:29:32 +1100 Subject: [PATCH 5/8] fix replicas cap --- .../elasticsearch/snapshots/SnapshotStressTestsHelper.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java index 845792196358f..b56a2bb572deb 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java @@ -288,6 +288,10 @@ void shuffleNodes() { shuffledNodes = newNodes; } + protected int numberOfReplicasUpperBound() { + return cluster.numDataNodes() - 1; + } + public void run() throws InterruptedException { shuffleNodes(); @@ -1530,7 +1534,7 @@ private void createIndexAndContinue(Releasable releasable) { client().admin() .indices() .prepareCreate(indexName) - .setSettings(indexSettings(shardCount, between(0, cluster.numDataNodes() - 1))) + .setSettings(indexSettings(shardCount, between(0, numberOfReplicasUpperBound()))) .execute(mustSucceed(response -> { assertTrue(response.isAcknowledged()); logger.info("--> finished create index [{}]", indexName); From b0b18ac79a526f91c9b556261dc63a54f8d7fe8b Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Mon, 17 Nov 2025 12:53:12 +1100 Subject: [PATCH 6/8] option to disable restart and shutdown --- .../snapshots/SnapshotStressTestsHelper.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java index b56a2bb572deb..7abcc8b06a9d6 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java @@ -293,6 +293,10 @@ protected int numberOfReplicasUpperBound() { } public void run() throws InterruptedException { + run(true, true); + } + + public void run(boolean restartEnabled, boolean shutdownEnabled) throws InterruptedException { shuffleNodes(); for (TrackedIndex trackedIndex : indices.values()) { @@ -303,7 +307,7 @@ public void run() throws InterruptedException { trackedRepository.start(); } - final int nodeRestarterCount = between(1, 2); + final int nodeRestarterCount = restartEnabled ? between(1, 2) : 0; for (int i = 0; i < nodeRestarterCount; i++) { startNodeRestarter(); } @@ -338,7 +342,7 @@ public void run() throws InterruptedException { startCleaner(); } - if (randomBoolean()) { + if (shutdownEnabled && randomBoolean()) { startNodeShutdownMarker(); } From c59a41de1e5d04c22c941f2dac17276572c0ab6d Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 20 Nov 2025 17:18:10 +1100 Subject: [PATCH 7/8] tweak and comments --- .../LocalPrimarySnapshotShardContext.java | 3 ++ ...calPrimarySnapshotShardContextFactory.java | 4 +++ .../SnapshotShardContextFactory.java | 29 ++++++++++++++++--- 3 files changed, 32 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java index 49c6028a82b29..6ca01de401a05 100644 --- a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContext.java @@ -35,6 +35,9 @@ import java.io.IOException; import java.util.Collection; +/** + * A {@link SnapshotShardContext} implementation that reads data from a local primary shard for snapshotting. + */ public final class LocalPrimarySnapshotShardContext extends SnapshotShardContext { private static final Logger logger = LogManager.getLogger(LocalPrimarySnapshotShardContext.class); diff --git a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java index c75aed2acbe22..4b972847c8bf7 100644 --- a/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java +++ b/server/src/main/java/org/elasticsearch/repositories/LocalPrimarySnapshotShardContextFactory.java @@ -28,6 +28,10 @@ import static org.elasticsearch.snapshots.SnapshotShardsService.getShardStateId; +/** + * A factory implementation for creating {@link LocalPrimarySnapshotShardContext} instance from the primary shard + * running on the local node. + */ public class LocalPrimarySnapshotShardContextFactory implements SnapshotShardContextFactory { private static final Logger logger = LogManager.getLogger(LocalPrimarySnapshotShardContextFactory.class); diff --git a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java index 653ab3d5505b8..cef08a8978b5b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java +++ b/server/src/main/java/org/elasticsearch/repositories/SnapshotShardContextFactory.java @@ -18,12 +18,25 @@ import java.io.IOException; +/** + * Factory interface to create {@link SnapshotShardContext} instances. + */ public interface SnapshotShardContextFactory { - default boolean ignoreShardCloseEvent() { - return false; - } - + /** + * Asynchronously creates a {@link SnapshotShardContext} for the given shard and snapshot. The passed-in listener is + * notified once the shard snapshot completes, either successfully or with a failure. + * @param shardId Shard ID of the shard to snapshot + * @param snapshot The overall snapshot information + * @param indexId The index ID of the index in the snapshot + * @param snapshotStatus Status of the shard snapshot + * @param repositoryMetaVersion The repository metadata version this snapshot uses + * @param snapshotStartTime Start time of the overall snapshot + * @param listener Listener to be invoked once the shard snapshot completes + * @return A subscribable listener that provides the created {@link SnapshotShardContext} or an exception if creation failed. + * @throws IOException Exception that may throw before even the returning subscribable listener is created. When this happens, + * the passed-in listener will NOT be notified. Caller is responsible to handle this situation. + */ SubscribableListener asyncCreate( ShardId shardId, Snapshot snapshot, @@ -33,4 +46,12 @@ SubscribableListener asyncCreate( long snapshotStartTime, ActionListener listener ) throws IOException; + + /** + * Indicates whether this factory wants to ignore shard close events while a shard snapshot is running. + * Defaults to false. It should return false unless the shard snapshot allows the shard to relocate. + */ + default boolean ignoreShardCloseEvent() { + return false; + } } From c0e875b480432b7f5010262dba69810ce0b442ca Mon Sep 17 00:00:00 2001 From: Yang Wang Date: Thu, 20 Nov 2025 18:12:18 +1100 Subject: [PATCH 8/8] license header --- .../snapshots/SnapshotStressTestsHelper.java | 20 ++++++------------- 1 file changed, 6 insertions(+), 14 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java index 7abcc8b06a9d6..5a1141d6f4d8f 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/SnapshotStressTestsHelper.java @@ -1,18 +1,10 @@ /* - * ELASTICSEARCH CONFIDENTIAL - * __________________ - * - * Copyright Elasticsearch B.V. All rights reserved. - * - * NOTICE: All information contained herein is, and remains - * the property of Elasticsearch B.V. and its suppliers, if any. - * The intellectual and technical concepts contained herein - * are proprietary to Elasticsearch B.V. and its suppliers and - * may be covered by U.S. and Foreign Patents, patents in - * process, and are protected by trade secret or copyright - * law. Dissemination of this information or reproduction of - * this material is strictly forbidden unless prior written - * permission is obtained from Elasticsearch B.V. + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". */ package org.elasticsearch.snapshots;