From f861f21f9f4cfa4d4cad92458bb04481dbd97a05 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 15:52:40 +0000 Subject: [PATCH 01/10] Add repository metadata integrity check API Relates #52622 --- .../BlobStoreMetadataIntegrityIT.java | 178 ++++++ server/src/main/java/module-info.java | 1 + .../elasticsearch/action/ActionModule.java | 4 + .../VerifyRepositoryIntegrityAction.java | 260 ++++++++ .../common/xcontent/ChunkedToXContent.java | 5 + .../BlobStoreIndexShardSnapshot.java | 18 +- .../BlobStoreIndexShardSnapshots.java | 8 +- .../repositories/Repository.java | 10 + .../blobstore/BlobStoreRepository.java | 39 ++ .../blobstore/MetadataVerifier.java | 584 ++++++++++++++++++ .../action/RestChunkedToXContentListener.java | 5 +- .../RestVerifyRepositoryIntegrityAction.java | 50 ++ ...ndexShardSnapshotsIntegritySuppressor.java | 23 + .../xpack/security/operator/Constants.java | 1 + 14 files changed, 1182 insertions(+), 4 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java create mode 100644 server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java create mode 100644 server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java create mode 100644 server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java create mode 100644 test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java new file mode 100644 index 0000000000000..945e360312f7a --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java @@ -0,0 +1,178 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.repositories.blobstore; + +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshotsIntegritySuppressor; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.RepositoryException; +import org.elasticsearch.repositories.RepositoryVerificationException; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.test.CorruptionUtils; +import org.junit.After; +import org.junit.Before; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; + +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; + +public class BlobStoreMetadataIntegrityIT extends AbstractSnapshotIntegTestCase { + + private static final String REPOSITORY_NAME = "test-repo"; + + private Releasable integrityCheckSuppressor; + + @Before + public void suppressIntegrityChecks() { + disableRepoConsistencyCheck("testing integrity checks involves breaking the repo"); + assertNull(integrityCheckSuppressor); + integrityCheckSuppressor = new BlobStoreIndexShardSnapshotsIntegritySuppressor(); + } + + @After + public void enableIntegrityChecks() { + Releasables.closeExpectNoException(integrityCheckSuppressor); + integrityCheckSuppressor = null; + } + + public void testIntegrityCheck() throws Exception { + final var repoPath = randomRepoPath(); + createRepository( + REPOSITORY_NAME, + "fs", + Settings.builder().put(BlobStoreRepository.SUPPORT_URL_REPO.getKey(), false).put("location", repoPath) + ); + final var repository = internalCluster().getCurrentMasterNodeInstance(RepositoriesService.class).repository(REPOSITORY_NAME); + + final var indexCount = between(1, 3); + for (int i = 0; i < indexCount; i++) { + createIndexWithRandomDocs("test-index-" + i, between(1, 1000)); + } + + for (int snapshotIndex = 0; snapshotIndex < 4; snapshotIndex++) { + final var indexRequests = new ArrayList(); + for (int i = 0; i < indexCount; i++) { + if (randomBoolean()) { + final var indexName = "test-index-" + i; + if (randomBoolean()) { + assertAcked(client().admin().indices().prepareDelete(indexName)); + createIndexWithRandomDocs(indexName, between(1, 1000)); + } + final var numDocs = between(1, 1000); + for (int doc = 0; doc < numDocs; doc++) { + indexRequests.add(client().prepareIndex(indexName).setSource("field1", "bar " + doc)); + } + } + } + indexRandom(true, indexRequests); + assertEquals(0, client().admin().indices().prepareFlush().get().getFailedShards()); + createFullSnapshot(REPOSITORY_NAME, "test-snapshot-" + snapshotIndex); + } + + final var request = new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, 5, 5, 5, 5, 10000, false); + + final var response = PlainActionFuture.get( + listener -> client().execute(VerifyRepositoryIntegrityAction.INSTANCE, request, listener), + 30, + TimeUnit.SECONDS + ); + assertThat(response.getRestStatus(), equalTo(RestStatus.OK)); + assertThat(response.getExceptions(), empty()); + + final var tempDir = createTempDir(); + + final List blobs; + try (var paths = Files.walk(repoPath)) { + blobs = paths.filter(Files::isRegularFile).sorted().toList(); + } + for (final var blob : blobs) { + logger.info("repo contents: {}", blob); + } + + final var repositoryDataFuture = new PlainActionFuture(); + repository.getRepositoryData(repositoryDataFuture); + final var repositoryData = repositoryDataFuture.get(); + final var repositoryDataBlob = repoPath.resolve("index-" + repositoryData.getGenId()); + + for (int i = 0; i < 2000; i++) { + final var blobToDamage = randomFrom(blobs); + final var isDataBlob = blobToDamage.getFileName().toString().startsWith(BlobStoreRepository.UPLOADED_DATA_BLOB_PREFIX); + final var isIndexBlob = blobToDamage.equals(repositoryDataBlob); + if (isDataBlob || isIndexBlob || randomBoolean()) { + logger.info("--> deleting {}", blobToDamage); + Files.move(blobToDamage, tempDir.resolve("tmp")); + } else { + logger.info("--> corrupting {}", blobToDamage); + Files.copy(blobToDamage, tempDir.resolve("tmp")); + CorruptionUtils.corruptFile(random(), blobToDamage); + } + try { + final var isCancelled = new AtomicBoolean(); + + final var verificationResponse = PlainActionFuture.get( + (PlainActionFuture> listener) -> repository.verifyMetadataIntegrity( + request, + listener, + () -> { + if (rarely() && rarely()) { + isCancelled.set(true); + return true; + } + return isCancelled.get(); + } + ), + 30, + TimeUnit.SECONDS + ); + assertThat(verificationResponse, not(empty())); + final var responseString = verificationResponse.stream().map(Throwable::getMessage).collect(Collectors.joining("\n")); + if (isCancelled.get()) { + assertThat(responseString, containsString("verification task cancelled before completion")); + } + if (isDataBlob && isCancelled.get() == false) { + assertThat( + responseString, + allOf(containsString(blobToDamage.getFileName().toString()), containsString("missing blob")) + ); + } + } catch (RepositoryException e) { + // ok, this means e.g. we couldn't even read the index blob + } finally { + Files.deleteIfExists(blobToDamage); + Files.move(tempDir.resolve("tmp"), blobToDamage); + } + + final var repairResponse = PlainActionFuture.get( + listener -> client().execute(VerifyRepositoryIntegrityAction.INSTANCE, request, listener), + 30, + TimeUnit.SECONDS + ); + assertThat(repairResponse.getRestStatus(), equalTo(RestStatus.OK)); + assertThat(repairResponse.getExceptions(), empty()); + } + } +} diff --git a/server/src/main/java/module-info.java b/server/src/main/java/module-info.java index 0ced8611f293c..0124059eb756a 100644 --- a/server/src/main/java/module-info.java +++ b/server/src/main/java/module-info.java @@ -70,6 +70,7 @@ exports org.elasticsearch.action.admin.cluster.repositories.cleanup; exports org.elasticsearch.action.admin.cluster.repositories.delete; exports org.elasticsearch.action.admin.cluster.repositories.get; + exports org.elasticsearch.action.admin.cluster.repositories.integrity; exports org.elasticsearch.action.admin.cluster.repositories.put; exports org.elasticsearch.action.admin.cluster.repositories.verify; exports org.elasticsearch.action.admin.cluster.reroute; diff --git a/server/src/main/java/org/elasticsearch/action/ActionModule.java b/server/src/main/java/org/elasticsearch/action/ActionModule.java index 59d055e27415c..12d43d119cb44 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionModule.java +++ b/server/src/main/java/org/elasticsearch/action/ActionModule.java @@ -60,6 +60,7 @@ import org.elasticsearch.action.admin.cluster.repositories.delete.TransportDeleteRepositoryAction; import org.elasticsearch.action.admin.cluster.repositories.get.GetRepositoriesAction; import org.elasticsearch.action.admin.cluster.repositories.get.TransportGetRepositoriesAction; +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryAction; import org.elasticsearch.action.admin.cluster.repositories.put.TransportPutRepositoryAction; import org.elasticsearch.action.admin.cluster.repositories.verify.TransportVerifyRepositoryAction; @@ -336,6 +337,7 @@ import org.elasticsearch.rest.action.admin.cluster.RestSnapshottableFeaturesAction; import org.elasticsearch.rest.action.admin.cluster.RestUpdateDesiredNodesAction; import org.elasticsearch.rest.action.admin.cluster.RestVerifyRepositoryAction; +import org.elasticsearch.rest.action.admin.cluster.RestVerifyRepositoryIntegrityAction; import org.elasticsearch.rest.action.admin.cluster.dangling.RestDeleteDanglingIndexAction; import org.elasticsearch.rest.action.admin.cluster.dangling.RestImportDanglingIndexAction; import org.elasticsearch.rest.action.admin.cluster.dangling.RestListDanglingIndicesAction; @@ -588,6 +590,7 @@ public void reg actions.register(GetRepositoriesAction.INSTANCE, TransportGetRepositoriesAction.class); actions.register(DeleteRepositoryAction.INSTANCE, TransportDeleteRepositoryAction.class); actions.register(VerifyRepositoryAction.INSTANCE, TransportVerifyRepositoryAction.class); + actions.register(VerifyRepositoryIntegrityAction.INSTANCE, VerifyRepositoryIntegrityAction.TransportAction.class); actions.register(CleanupRepositoryAction.INSTANCE, TransportCleanupRepositoryAction.class); actions.register(GetSnapshotsAction.INSTANCE, TransportGetSnapshotsAction.class); actions.register(DeleteSnapshotAction.INSTANCE, TransportDeleteSnapshotAction.class); @@ -757,6 +760,7 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestGetRepositoriesAction(settingsFilter)); registerHandler.accept(new RestDeleteRepositoryAction()); registerHandler.accept(new RestVerifyRepositoryAction()); + registerHandler.accept(new RestVerifyRepositoryIntegrityAction()); registerHandler.accept(new RestCleanupRepositoryAction()); registerHandler.accept(new RestGetSnapshotsAction()); registerHandler.accept(new RestCreateSnapshotAction()); diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java new file mode 100644 index 0000000000000..6599c1e2f8e35 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java @@ -0,0 +1,260 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.action.admin.cluster.repositories.integrity; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.ActionType; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.MasterNodeReadRequest; +import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.collect.Iterators; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.xcontent.ChunkedToXContent; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.RepositoryVerificationException; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xcontent.ToXContent; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +public class VerifyRepositoryIntegrityAction extends ActionType { + + public static final VerifyRepositoryIntegrityAction INSTANCE = new VerifyRepositoryIntegrityAction(); + public static final String NAME = "cluster:admin/repository/verify_integrity"; + + private VerifyRepositoryIntegrityAction() { + super(NAME, VerifyRepositoryIntegrityAction.Response::new); + } + + public static class Request extends MasterNodeReadRequest { + + private final String repository; + private final int threadpoolConcurrency; + private final int snapshotVerificationConcurrency; + private final int indexVerificationConcurrency; + private final int indexSnapshotVerificationConcurrency; + private final int maxFailures; + private final boolean permitMissingSnapshotDetails; + + public Request( + String repository, + int threadpoolConcurrency, + int snapshotVerificationConcurrency, + int indexVerificationConcurrency, + int indexSnapshotVerificationConcurrency, + int maxFailures, + boolean permitMissingSnapshotDetails + ) { + this.repository = repository; + this.threadpoolConcurrency = requireMin("threadpoolConcurrency", 0, threadpoolConcurrency); + this.snapshotVerificationConcurrency = requireMin("snapshotVerificationConcurrency", 1, snapshotVerificationConcurrency); + this.indexVerificationConcurrency = requireMin("indexVerificationConcurrency", 1, indexVerificationConcurrency); + this.indexSnapshotVerificationConcurrency = requireMin( + "indexSnapshotVerificationConcurrency", + 1, + indexSnapshotVerificationConcurrency + ); + this.maxFailures = requireMin("maxFailure", 1, maxFailures); + this.permitMissingSnapshotDetails = permitMissingSnapshotDetails; + } + + private static int requireMin(String name, int min, int value) { + if (value < min) { + throw new IllegalArgumentException("argument [" + name + "] must be at least [" + min + "]"); + } + return value; + } + + public Request(StreamInput in) throws IOException { + super(in); + this.repository = in.readString(); + this.threadpoolConcurrency = in.readVInt(); + this.snapshotVerificationConcurrency = in.readVInt(); + this.indexVerificationConcurrency = in.readVInt(); + this.indexSnapshotVerificationConcurrency = in.readVInt(); + this.maxFailures = in.readVInt(); + this.permitMissingSnapshotDetails = in.readBoolean(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(repository); + out.writeVInt(threadpoolConcurrency); + out.writeVInt(snapshotVerificationConcurrency); + out.writeVInt(indexVerificationConcurrency); + out.writeVInt(indexSnapshotVerificationConcurrency); + out.writeVInt(maxFailures); + out.writeBoolean(permitMissingSnapshotDetails); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers); + } + + public int getThreadpoolConcurrency() { + return threadpoolConcurrency; + } + + public int getSnapshotVerificationConcurrency() { + return snapshotVerificationConcurrency; + } + + public int getIndexVerificationConcurrency() { + return indexVerificationConcurrency; + } + + public int getIndexSnapshotVerificationConcurrency() { + return indexSnapshotVerificationConcurrency; + } + + public int getMaxFailures() { + return maxFailures; + } + + public boolean permitMissingSnapshotDetails() { + return permitMissingSnapshotDetails; + } + + public Request withDefaultThreadpoolConcurrency(Settings settings) { + if (threadpoolConcurrency == 0) { + final var request = new Request( + repository, + Math.max(1, EsExecutors.allocatedProcessors(settings) / 2), + snapshotVerificationConcurrency, + indexVerificationConcurrency, + indexSnapshotVerificationConcurrency, + maxFailures, + permitMissingSnapshotDetails + ); + request.masterNodeTimeout(masterNodeTimeout()); + return request; + } else { + return this; + } + } + } + + public static class Response extends ActionResponse implements ChunkedToXContent { + + private final List exceptions; + + public Response(List exceptions) { + this.exceptions = exceptions; + } + + public Response(StreamInput in) throws IOException { + super(in); + this.exceptions = in.readList(RepositoryVerificationException::new); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeList(exceptions); + } + + @Override + public Iterator toXContentChunked(ToXContent.Params outerParams) { + return Iterators.concat( + Iterators.single((builder, params) -> builder.startObject().startArray("errors")), + exceptions.stream().map(e -> (builder, params) -> { + builder.startObject(); + ElasticsearchException.generateFailureXContent(builder, params, e, true); + return builder.endObject(); + }).iterator(), + Iterators.single((builder, params) -> builder.endArray().endObject()) + ); + } + + @Override + public RestStatus getRestStatus() { + if (exceptions.isEmpty()) { + return RestStatus.OK; + } else { + return RestStatus.INTERNAL_SERVER_ERROR; + } + } + + public List getExceptions() { + return exceptions; + } + } + + public static class TransportAction extends TransportMasterNodeReadAction { + + private final RepositoriesService repositoriesService; + + @Inject + public TransportAction( + TransportService transportService, + ClusterService clusterService, + RepositoriesService repositoriesService, + ThreadPool threadPool, + ActionFilters actionFilters, + IndexNameExpressionResolver indexNameExpressionResolver + ) { + super( + NAME, + true, + transportService, + clusterService, + threadPool, + actionFilters, + Request::new, + indexNameExpressionResolver, + Response::new, + ThreadPool.Names.SNAPSHOT_META + ); + this.repositoriesService = repositoriesService; + } + + @Override + protected ClusterBlockException checkBlock(Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } + + @Override + protected void masterOperation(Task task, Request request, ClusterState state, ActionListener listener) throws Exception { + // TODO add mechanism to block blob deletions while this is running + final var cancellableTask = (CancellableTask) task; + repositoriesService.repository(request.repository) + .verifyMetadataIntegrity( + request.withDefaultThreadpoolConcurrency(clusterService.getSettings()), + listener.map(Response::new), + cancellableTask::isCancelled + ); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java index 9fe5eefd45889..a93ff4853879a 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java @@ -8,6 +8,7 @@ package org.elasticsearch.common.xcontent; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; @@ -42,4 +43,8 @@ static ToXContentObject wrapAsXContentObject(ChunkedToXContent chunkedToXContent return builder; }; } + + default RestStatus getRestStatus() { + return RestStatus.OK; + } } diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index baf0c9ad08059..7f5c50ceaf421 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -17,6 +17,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentParserUtils; import org.elasticsearch.core.Nullable; +import org.elasticsearch.gateway.CorruptStateException; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.xcontent.ParseField; import org.elasticsearch.xcontent.ToXContentFragment; @@ -318,7 +319,10 @@ public static FileInfo fromXContent(XContentParser parser) throws IOException { } case WRITER_UUID -> { writerUuid = new BytesRef(parser.binaryValue()); - assert writerUuid.length > 0; + assert BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED == false || writerUuid.length > 0; + if (writerUuid.length == 0) { + throw new ElasticsearchParseException("invalid (empty) writer uuid"); + } } default -> XContentParserUtils.throwUnknownField(currentFieldName, parser); } @@ -336,6 +340,11 @@ public static FileInfo fromXContent(XContentParser parser) throws IOException { } else if (checksum == null) { throw new ElasticsearchParseException("missing checksum for name [" + name + "]"); } + try { + org.apache.lucene.util.Version.parse(writtenBy); + } catch (Exception e) { + throw new ElasticsearchParseException("invalid written_by [" + writtenBy + "]"); + } return new FileInfo(name, new StoreFileMetadata(physicalName, length, checksum, writtenBy, metaHash, writerUuid), partSize); } @@ -571,6 +580,13 @@ public static BlobStoreIndexShardSnapshot fromXContent(XContentParser parser) th } } + if (snapshot == null) { + throw new CorruptStateException("snapshot missing"); + } + if (indexVersion < 0) { + throw new CorruptStateException("index version missing or corrupt"); + } + return new BlobStoreIndexShardSnapshot( snapshot, indexVersion, diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java index 113d3c8f28a19..2f022e1c34394 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshots.java @@ -256,6 +256,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + static volatile boolean INTEGRITY_ASSERTIONS_ENABLED = true; + public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) throws IOException { XContentParser.Token token = parser.currentToken(); if (token == null) { // New parser @@ -309,7 +311,11 @@ public static BlobStoreIndexShardSnapshots fromXContent(XContentParser parser) t List fileInfosBuilder = new ArrayList<>(); for (String file : entry.v2()) { FileInfo fileInfo = files.get(file); - assert fileInfo != null; + if (fileInfo == null) { + final var exception = new IllegalStateException("shard index inconsistent at file [" + file + "]"); + assert INTEGRITY_ASSERTIONS_ENABLED == false : exception; + throw exception; + } fileInfosBuilder.add(fileInfo); } snapshots.add(new SnapshotFiles(entry.v1(), Collections.unmodifiableList(fileInfosBuilder), historyUUIDs.get(entry.v1()))); diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index d818476c68c2c..4f9c00b959a57 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -9,6 +9,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -31,6 +32,7 @@ import java.util.Collection; import java.util.List; import java.util.Set; +import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; @@ -309,4 +311,12 @@ void cloneShardSnapshot( static boolean assertSnapshotMetaThread() { return ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SNAPSHOT_META); } + + default void verifyMetadataIntegrity( + VerifyRepositoryIntegrityAction.Request request, + ActionListener> listener, + BooleanSupplier isCancelledSupplier + ) { + listener.onFailure(new UnsupportedOperationException("this repository type does not support metadata integrity verification")); + } } 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 3c536328522cf..0e7e73b2de242 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.SingleResultDeduplicator; import org.elasticsearch.action.StepListener; +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.ListenableActionFuture; import org.elasticsearch.action.support.PlainActionFuture; @@ -131,6 +132,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -142,6 +144,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Supplier; @@ -3539,6 +3542,42 @@ private static void failStoreIfCorrupted(Store store, Exception e) { } } + @Override + public void verifyMetadataIntegrity( + VerifyRepositoryIntegrityAction.Request request, + ActionListener> listener, + BooleanSupplier isCancelledSupplier + ) { + getRepositoryData(listener.delegateFailure((l, repositoryData) -> { + logger.info( + "[{}] verifying metadata integrity for index generation [{}]: repo UUID [{}], cluster UUID [{}]", + metadata.name(), + repositoryData.getGenId(), + repositoryData.getUuid(), + repositoryData.getClusterUUID() + ); + + threadPool.executor(ThreadPool.Names.SNAPSHOT_META) + .execute(ActionRunnable.supply(l.delegateFailure((l2, loadedRepositoryData) -> { + // really just checking that the repo data can be loaded, but may as well check a little consistency too + if (loadedRepositoryData.getGenId() != repositoryData.getGenId()) { + throw new IllegalStateException( + String.format( + Locale.ROOT, + "[%s] has repository data generation [%d], expected [%d]", + metadata.name(), + loadedRepositoryData.getGenId(), + repositoryData.getGenId() + ) + ); + } + try (var metadataVerifier = new MetadataVerifier(this, request, repositoryData, isCancelledSupplier, l2)) { + metadataVerifier.run(); + } + }), () -> getRepositoryData(repositoryData.getGenId()))); + })); + } + public boolean supportURLRepo() { return supportURLRepo; } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java new file mode 100644 index 0000000000000..f26418389d749 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -0,0 +1,584 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.repositories.blobstore; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; +import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.common.CheckedSupplier; +import org.elasticsearch.common.blobstore.support.BlobMetadata; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.CheckedConsumer; +import org.elasticsearch.core.CheckedRunnable; +import org.elasticsearch.core.RefCounted; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.RepositoryVerificationException; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiConsumer; +import java.util.function.BooleanSupplier; +import java.util.stream.Collectors; + +import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; +import static org.elasticsearch.core.Strings.format; + +class MetadataVerifier implements Releasable { + private static final Logger logger = LogManager.getLogger(MetadataVerifier.class); + + private final BlobStoreRepository blobStoreRepository; + private final ActionListener> finalListener; + private final RefCounted finalRefs = AbstractRefCounted.of(this::onCompletion); + private final String repositoryName; + private final VerifyRepositoryIntegrityAction.Request verifyRequest; + private final RepositoryData repositoryData; + private final BooleanSupplier isCancelledSupplier; + private final Queue failures = new ConcurrentLinkedQueue<>(); + private final AtomicLong failureCount = new AtomicLong(); + private final Map> snapshotsByIndex; + private final Semaphore threadPoolPermits; + private final Queue executorQueue = new ConcurrentLinkedQueue<>(); + + MetadataVerifier( + BlobStoreRepository blobStoreRepository, + VerifyRepositoryIntegrityAction.Request verifyRequest, + RepositoryData repositoryData, + BooleanSupplier isCancelledSupplier, + ActionListener> finalListener + ) { + this.blobStoreRepository = blobStoreRepository; + this.repositoryName = blobStoreRepository.metadata.name(); + this.verifyRequest = verifyRequest; + this.repositoryData = repositoryData; + this.isCancelledSupplier = isCancelledSupplier; + this.finalListener = finalListener; + this.snapshotsByIndex = this.repositoryData.getIndices() + .values() + .stream() + .collect(Collectors.toMap(IndexId::getName, indexId -> Set.copyOf(this.repositoryData.getSnapshots(indexId)))); + + this.threadPoolPermits = new Semaphore(Math.max(1, verifyRequest.getThreadpoolConcurrency())); + } + + @Override + public void close() { + finalRefs.decRef(); + } + + private void addFailure(String format, Object... args) { + if (failureCount.incrementAndGet() <= verifyRequest.getMaxFailures()) { + final var failure = format(format, args); + logger.debug("[{}] found metadata verification failure: {}", repositoryName, failure); + failures.add(new RepositoryVerificationException(repositoryName, failure)); + } + } + + private void addFailure(Exception exception) { + if (isCancelledSupplier.getAsBoolean() && exception instanceof TaskCancelledException) { + return; + } + if (failureCount.incrementAndGet() <= verifyRequest.getMaxFailures()) { + logger.debug(() -> format("[%s] exception during metadata verification: {}", repositoryName), exception); + failures.add( + exception instanceof RepositoryVerificationException rve + ? rve + : new RepositoryVerificationException(repositoryName, "exception during metadata verification", exception) + ); + } + } + + public void run() { + logger.info( + "[{}] verifying metadata integrity for index generation [{}]: repo UUID [{}], cluster UUID [{}]", + repositoryName, + repositoryData.getGenId(), + repositoryData.getUuid(), + repositoryData.getClusterUUID() + ); + + verifySnapshots(); + } + + private void verifySnapshots() { + runThrottled( + makeVoidListener(finalRefs, this::verifyIndices), + repositoryData.getSnapshotIds().iterator(), + this::verifySnapshot, + verifyRequest.getSnapshotVerificationConcurrency() + ); + } + + private void verifySnapshot(RefCounted snapshotRefs, SnapshotId snapshotId) { + if (verifyRequest.permitMissingSnapshotDetails() == false && repositoryData.hasMissingDetails(snapshotId)) { + addFailure("snapshot [%s] has missing snapshot details", snapshotId); + } + + if (isCancelledSupplier.getAsBoolean()) { + // getSnapshotInfo does its own forking so we must check for cancellation here + return; + } + + blobStoreRepository.getSnapshotInfo(snapshotId, makeListener(snapshotRefs, snapshotInfo -> { + if (snapshotInfo.snapshotId().equals(snapshotId) == false) { + addFailure("snapshot [%s] has unexpected ID in info blob: [%s]", snapshotId, snapshotInfo.snapshotId()); + } + for (final var index : snapshotInfo.indices()) { + if (snapshotsByIndex.get(index).contains(snapshotId) == false) { + addFailure("snapshot [%s] contains unexpected index [%s]", snapshotId, index); + } + } + })); + + forkSupply(snapshotRefs, () -> blobStoreRepository.getSnapshotGlobalMetadata(snapshotId), metadata -> { + if (metadata.indices().isEmpty() == false) { + addFailure("snapshot [%s] contains unexpected index metadata within global metadata", snapshotId); + } + }); + } + + private void verifyIndices() { + final var indicesMap = repositoryData.getIndices(); + + for (final var indicesEntry : indicesMap.entrySet()) { + final var name = indicesEntry.getKey(); + final var indexId = indicesEntry.getValue(); + if (name.equals(indexId.getName()) == false) { + addFailure("index name [%s] has mismatched name in [%s]", name, indexId); + } + } + + runThrottled( + makeVoidListener(finalRefs, () -> {}), + indicesMap.values().iterator(), + (refCounted, indexId) -> new IndexVerifier(refCounted, indexId).run(), + verifyRequest.getIndexVerificationConcurrency() + ); + } + + private class IndexVerifier { + private final RefCounted indexRefs; + private final IndexId indexId; + private final Set expectedSnapshots; + private final Map>> shardBlobsListenersByShard = newConcurrentMap(); + private final Map> shardCountListenersByBlobId = new HashMap<>(); + + IndexVerifier(RefCounted indexRefs, IndexId indexId) { + this.indexRefs = indexRefs; + this.indexId = indexId; + this.expectedSnapshots = snapshotsByIndex.get(this.indexId.getName()); + } + + void run() { + + // TODO consider distributing the workload, giving each node a subset of indices to process + + final var indexSnapshots = repositoryData.getSnapshots(indexId); + + try (var indexMetadataChecksRef = wrap(makeVoidListener(indexRefs, this::onIndexMetadataChecksComplete))) { + runThrottled( + makeVoidListener(indexMetadataChecksRef, () -> {}), + indexSnapshots.iterator(), + this::verifyIndexSnapshot, + verifyRequest.getIndexSnapshotVerificationConcurrency() + ); + } + } + + private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapshotId) { + if (expectedSnapshots.contains(snapshotId) == false) { + addFailure("index [%s] has mismatched snapshot [%s]", indexId, snapshotId); + } + + final var indexMetaBlobId = repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, indexId); + shardCountListenersByBlobId.computeIfAbsent(indexMetaBlobId, ignored -> { + final var shardCountFuture = new ListenableActionFuture(); + forkSupply(() -> { + final var shardCount = blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId) + .getNumberOfShards(); + for (int i = 0; i < shardCount; i++) { + shardBlobsListenersByShard.computeIfAbsent(i, shardId -> { + final var shardBlobsFuture = new ListenableActionFuture>(); + forkSupply(() -> blobStoreRepository.shardContainer(indexId, shardId).listBlobs(), shardBlobsFuture); + return shardBlobsFuture; + }); + } + return shardCount; + }, shardCountFuture); + return shardCountFuture; + }).addListener(makeListener(indexSnapshotRefs, shardCount -> { + for (int i = 0; i < shardCount; i++) { + final var shardId = i; + shardBlobsListenersByShard.get(i) + .addListener( + makeListener( + indexSnapshotRefs, + shardBlobs -> forkSupply( + indexSnapshotRefs, + () -> blobStoreRepository.loadShardSnapshot( + blobStoreRepository.shardContainer(indexId, shardId), + snapshotId + ), + shardSnapshot -> verifyShardSnapshot(snapshotId, shardId, shardBlobs, shardSnapshot) + ) + ) + ); + } + })); + } + + private void verifyShardSnapshot( + SnapshotId snapshotId, + int shardId, + Map shardBlobs, + BlobStoreIndexShardSnapshot shardSnapshot + ) { + if (shardSnapshot.snapshot().equals(snapshotId.getName()) == false) { + addFailure( + "snapshot [%s] for shard [%s/%d] has mismatched name [%s]", + snapshotId, + indexId, + shardId, + shardSnapshot.snapshot() + ); + } + + for (final var fileInfo : shardSnapshot.indexFiles()) { + verifyFileInfo(snapshotId.toString(), shardId, shardBlobs, fileInfo); + } + } + + private static String formatExact(ByteSizeValue byteSizeValue) { + if (byteSizeValue.getBytes() >= ByteSizeUnit.KB.toBytes(1)) { + return format("%s/%dB", byteSizeValue.toString(), byteSizeValue.getBytes()); + } else { + return byteSizeValue.toString(); + } + } + + private void verifyFileInfo( + String snapshot, + int shardId, + Map shardBlobs, + BlobStoreIndexShardSnapshot.FileInfo fileInfo + ) { + final var fileLength = ByteSizeValue.ofBytes(fileInfo.length()); + if (fileInfo.metadata().hashEqualsContents()) { + final var actualLength = ByteSizeValue.ofBytes(fileInfo.metadata().hash().length); + if (fileLength.getBytes() != actualLength.getBytes()) { + addFailure( + "snapshot [%s] for shard [%s/%d] has virtual blob [%s] for [%s] with length [%s] instead of [%s]", + snapshot, + indexId, + shardId, + fileInfo.name(), + fileInfo.physicalName(), + formatExact(actualLength), + formatExact(fileLength) + ); + } + } else { + for (int part = 0; part < fileInfo.numberOfParts(); part++) { + final var blobName = fileInfo.partName(part); + final var blobInfo = shardBlobs.get(blobName); + final var partLength = ByteSizeValue.ofBytes(fileInfo.partBytes(part)); + if (blobInfo == null) { + addFailure( + "snapshot [%s] for shard [%s/%d] has missing blob [%s] for [%s] part [%d/%d]; " + + "file length [%s], part length [%s]", + snapshot, + indexId, + shardId, + blobName, + fileInfo.physicalName(), + part, + fileInfo.numberOfParts(), + formatExact(fileLength), + formatExact(partLength) + ); + + } else if (blobInfo.length() != partLength.getBytes()) { + addFailure( + "snapshot [%s] for shard [%s/%d] has blob [%s] for [%s] part [%d/%d] with length [%s] instead of [%s]; " + + "file length [%s]", + snapshot, + indexId, + shardId, + blobName, + fileInfo.physicalName(), + part, + fileInfo.numberOfParts(), + formatExact(ByteSizeValue.ofBytes(blobInfo.length())), + formatExact(partLength), + formatExact(fileLength) + ); + } + } + } + } + + private void onIndexMetadataChecksComplete() { + if (shardCountListenersByBlobId.isEmpty()) { + throw new IllegalStateException(format("index [%s] has no metadata", indexId)); + } + + try (var shardGenerationChecksRef = wrap(makeVoidListener(indexRefs, () -> {}))) { + for (final var shardEntry : shardBlobsListenersByShard.entrySet()) { + verifyShardGenerations(shardGenerationChecksRef, shardEntry); + } + } + } + + private void verifyShardGenerations( + RefCounted shardGenerationChecksRef, + Map.Entry>> shardEntry + ) { + final int shardId = shardEntry.getKey(); + shardEntry.getValue() + .addListener( + makeListener( + shardGenerationChecksRef, + shardBlobs -> forkSupply( + shardGenerationChecksRef, + () -> blobStoreRepository.getBlobStoreIndexShardSnapshots( + indexId, + shardId, + Objects.requireNonNull( + repositoryData.shardGenerations().getShardGen(indexId, shardId), + "shard generations for " + indexId + "/" + shardId + ) + ), + blobStoreIndexShardSnapshots -> { + for (final var snapshotFiles : blobStoreIndexShardSnapshots.snapshots()) { + snapshotFiles.snapshot(); // TODO validate + snapshotFiles.shardStateIdentifier(); // TODO validate + for (final var fileInfo : snapshotFiles.indexFiles()) { + verifyFileInfo(snapshotFiles.snapshot(), shardId, shardBlobs, fileInfo); + } + } + } + ) + ) + ); + } + } + + private ActionListener makeListener(RefCounted refCounted, CheckedConsumer consumer) { + refCounted.incRef(); + return ActionListener.runAfter(ActionListener.wrap(consumer, this::addFailure), refCounted::decRef); + } + + private ActionListener makeVoidListener(RefCounted refCounted, CheckedRunnable runnable) { + return makeListener(refCounted, ignored -> runnable.run()); + } + + private void forkSupply(CheckedSupplier supplier, ActionListener listener) { + fork(ActionRunnable.supply(listener, supplier)); + } + + private void fork(AbstractRunnable runnable) { + executorQueue.add(runnable); + tryProcessQueue(); + } + + private void tryProcessQueue() { + while (threadPoolPermits.tryAcquire()) { + final var runnable = executorQueue.poll(); + if (runnable == null) { + threadPoolPermits.release(); + return; + } + + if (isCancelledSupplier.getAsBoolean()) { + try { + runnable.onFailure(new TaskCancelledException("task cancelled")); + continue; + } finally { + threadPoolPermits.release(); + } + } + + blobStoreRepository.threadPool().executor(ThreadPool.Names.SNAPSHOT_META).execute(new AbstractRunnable() { + @Override + public void onRejection(Exception e) { + try { + runnable.onRejection(e); + } finally { + onCompletion(); + } + } + + @Override + public void onFailure(Exception e) { + try { + runnable.onFailure(e); + } finally { + onCompletion(); + } + } + + @Override + protected void doRun() { + runnable.run(); + onCompletion(); + } + + @Override + public String toString() { + return runnable.toString(); + } + + private void onCompletion() { + threadPoolPermits.release(); + tryProcessQueue(); + } + }); + } + } + + private void forkSupply(RefCounted refCounted, CheckedSupplier supplier, CheckedConsumer consumer) { + forkSupply(supplier, makeListener(refCounted, consumer)); + } + + private void onCompletion() { + final var finalFailureCount = failureCount.get(); + if (finalFailureCount > verifyRequest.getMaxFailures()) { + failures.add( + new RepositoryVerificationException( + repositoryName, + format( + "found %d verification failures in total, %d suppressed", + finalFailureCount, + finalFailureCount - verifyRequest.getMaxFailures() + ) + ) + ); + } + + if (isCancelledSupplier.getAsBoolean()) { + failures.add(new RepositoryVerificationException(repositoryName, "verification task cancelled before completion")); + } + + finalListener.onResponse(failures.stream().toList()); + } + + private interface RefCountedListenerWrapper extends Releasable, RefCounted {} + + private static RefCountedListenerWrapper wrap(ActionListener listener) { + final var refCounted = AbstractRefCounted.of(() -> listener.onResponse(null)); + return new RefCountedListenerWrapper() { + @Override + public void incRef() { + refCounted.incRef(); + } + + @Override + public boolean tryIncRef() { + return refCounted.tryIncRef(); + } + + @Override + public boolean decRef() { + return refCounted.decRef(); + } + + @Override + public boolean hasReferences() { + return refCounted.hasReferences(); + } + + @Override + public void close() { + decRef(); + } + }; + } + + private static void runThrottled( + ActionListener completionListener, + Iterator iterator, + BiConsumer consumer, + int maxConcurrency + ) { + try (var throttledIterator = new ThrottledIterator<>(completionListener, iterator, consumer, maxConcurrency)) { + throttledIterator.run(); + } + } + + private static class ThrottledIterator implements Releasable { + private final RefCountedListenerWrapper refCounted; + private final Iterator iterator; + private final BiConsumer consumer; + private final Semaphore permits; + + ThrottledIterator( + ActionListener completionListener, + Iterator iterator, + BiConsumer consumer, + int maxConcurrency + ) { + this.refCounted = wrap(completionListener); + this.iterator = iterator; + this.consumer = consumer; + this.permits = new Semaphore(maxConcurrency); + } + + void run() { + while (permits.tryAcquire()) { + final T item; + synchronized (iterator) { + if (iterator.hasNext()) { + item = iterator.next(); + } else { + permits.release(); + return; + } + } + refCounted.incRef(); + final var itemRefCount = AbstractRefCounted.of(() -> { + permits.release(); + try { + run(); + } finally { + refCounted.decRef(); + } + }); + try { + consumer.accept(itemRefCount, item); + } finally { + itemRefCount.decRef(); + } + } + } + + @Override + public void close() { + refCounted.close(); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java b/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java index ece3a432f059b..01b3febe31a2c 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java @@ -12,7 +12,6 @@ import org.elasticsearch.rest.ChunkedRestResponseBody; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestResponse; -import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -28,6 +27,8 @@ public RestChunkedToXContentListener(RestChannel channel) { @Override protected void processResponse(Response response) throws IOException { - channel.sendResponse(new RestResponse(RestStatus.OK, ChunkedRestResponseBody.fromXContent(response, channel.request(), channel))); + channel.sendResponse( + new RestResponse(response.getRestStatus(), ChunkedRestResponseBody.fromXContent(response, channel.request(), channel)) + ); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java new file mode 100644 index 0000000000000..c6fda490d40e4 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java @@ -0,0 +1,50 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.rest.action.admin.cluster; + +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; +import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.action.RestCancellableNodeClient; +import org.elasticsearch.rest.action.RestChunkedToXContentListener; + +import java.io.IOException; +import java.util.List; + +import static org.elasticsearch.rest.RestRequest.Method.POST; + +public class RestVerifyRepositoryIntegrityAction extends BaseRestHandler { + @Override + public List routes() { + return List.of(new Route(POST, "/_snapshot/{repository}/_verify_integrity")); + } + + @Override + public String getName() { + return "verify_repository_integrity_action"; + } + + @Override + public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { + final var verifyRequest = new VerifyRepositoryIntegrityAction.Request( + request.param("repository"), + request.paramAsInt("threadpool_concurrency", 0), + request.paramAsInt("snapshot_verification_concurrency", 5), + request.paramAsInt("index_verification_concurrency", 5), + request.paramAsInt("index_snapshot_verification_concurrency", 5), + request.paramAsInt("max_failures", 10000), + request.paramAsBoolean("permit_missing_snapshot_details", false) + ); + verifyRequest.masterNodeTimeout(request.paramAsTime("master_timeout", verifyRequest.masterNodeTimeout())); + return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).admin() + .cluster() + .execute(VerifyRepositoryIntegrityAction.INSTANCE, verifyRequest, new RestChunkedToXContentListener<>(channel)); + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java b/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java new file mode 100644 index 0000000000000..d22db399a2f34 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshotsIntegritySuppressor.java @@ -0,0 +1,23 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.index.snapshots.blobstore; + +import org.elasticsearch.core.Releasable; + +public class BlobStoreIndexShardSnapshotsIntegritySuppressor implements Releasable { + + public BlobStoreIndexShardSnapshotsIntegritySuppressor() { + BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED = false; + } + + @Override + public void close() { + BlobStoreIndexShardSnapshots.INTEGRITY_ASSERTIONS_ENABLED = true; + } +} diff --git a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java index 65037651cf4cb..19c42a77337db 100644 --- a/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java +++ b/x-pack/plugin/security/qa/operator-privileges-tests/src/javaRestTest/java/org/elasticsearch/xpack/security/operator/Constants.java @@ -53,6 +53,7 @@ public class Constants { "cluster:admin/repository/get", "cluster:admin/repository/put", "cluster:admin/repository/verify", + "cluster:admin/repository/verify_integrity", "cluster:admin/reroute", "cluster:admin/script/delete", "cluster:admin/script/get", From 3dd40155a3aa1f0fdd055230b59ccd3b03e99149 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 16:54:11 +0000 Subject: [PATCH 02/10] Add progress logging --- .../repositories/RepositoryData.java | 4 ++ .../blobstore/MetadataVerifier.java | 62 ++++++++++++++++--- 2 files changed, 56 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java index 089b5a6e639ba..93cc618236c6d 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java @@ -275,6 +275,10 @@ public Collection getSnapshotIds() { return snapshotIds.values(); } + public long getIndexSnapshotCount() { + return indexSnapshots.values().stream().mapToLong(List::size).sum(); + } + /** * @return whether some of the {@link SnapshotDetails} of the given snapshot are missing, due to BwC, so that they must be loaded from * the {@link SnapshotInfo} blob instead. diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index f26418389d749..b2745d8876796 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -32,7 +32,6 @@ import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.threadpool.ThreadPool; -import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -64,6 +63,9 @@ class MetadataVerifier implements Releasable { private final Map> snapshotsByIndex; private final Semaphore threadPoolPermits; private final Queue executorQueue = new ConcurrentLinkedQueue<>(); + private final ProgressLogger snapshotProgressLogger; + private final ProgressLogger indexProgressLogger; + private final ProgressLogger indexSnapshotProgressLogger; MetadataVerifier( BlobStoreRepository blobStoreRepository, @@ -84,6 +86,9 @@ class MetadataVerifier implements Releasable { .collect(Collectors.toMap(IndexId::getName, indexId -> Set.copyOf(this.repositoryData.getSnapshots(indexId)))); this.threadPoolPermits = new Semaphore(Math.max(1, verifyRequest.getThreadpoolConcurrency())); + this.snapshotProgressLogger = new ProgressLogger("snapshots", repositoryData.getSnapshotIds().size(), 100); + this.indexProgressLogger = new ProgressLogger("indices", repositoryData.getIndices().size(), 100); + this.indexSnapshotProgressLogger = new ProgressLogger("index snapshots", repositoryData.getIndexSnapshotCount(), 10000); } @Override @@ -115,11 +120,15 @@ private void addFailure(Exception exception) { public void run() { logger.info( - "[{}] verifying metadata integrity for index generation [{}]: repo UUID [{}], cluster UUID [{}]", + "[{}] verifying metadata integrity for index generation [{}]: " + + "repo UUID [{}], cluster UUID [{}], snapshots [{}], indices [{}], index snapshots [{}]", repositoryName, repositoryData.getGenId(), repositoryData.getUuid(), - repositoryData.getClusterUUID() + repositoryData.getClusterUUID(), + snapshotProgressLogger.getExpectedMax(), + indexProgressLogger.getExpectedMax(), + indexSnapshotProgressLogger.getExpectedMax() ); verifySnapshots(); @@ -130,7 +139,8 @@ private void verifySnapshots() { makeVoidListener(finalRefs, this::verifyIndices), repositoryData.getSnapshotIds().iterator(), this::verifySnapshot, - verifyRequest.getSnapshotVerificationConcurrency() + verifyRequest.getSnapshotVerificationConcurrency(), + snapshotProgressLogger ); } @@ -177,7 +187,8 @@ private void verifyIndices() { makeVoidListener(finalRefs, () -> {}), indicesMap.values().iterator(), (refCounted, indexId) -> new IndexVerifier(refCounted, indexId).run(), - verifyRequest.getIndexVerificationConcurrency() + verifyRequest.getIndexVerificationConcurrency(), + indexProgressLogger ); } @@ -186,7 +197,7 @@ private class IndexVerifier { private final IndexId indexId; private final Set expectedSnapshots; private final Map>> shardBlobsListenersByShard = newConcurrentMap(); - private final Map> shardCountListenersByBlobId = new HashMap<>(); + private final Map> shardCountListenersByBlobId = newConcurrentMap(); IndexVerifier(RefCounted indexRefs, IndexId indexId) { this.indexRefs = indexRefs; @@ -205,7 +216,8 @@ void run() { makeVoidListener(indexMetadataChecksRef, () -> {}), indexSnapshots.iterator(), this::verifyIndexSnapshot, - verifyRequest.getIndexSnapshotVerificationConcurrency() + verifyRequest.getIndexSnapshotVerificationConcurrency(), + indexSnapshotProgressLogger ); } } @@ -523,9 +535,10 @@ private static void runThrottled( ActionListener completionListener, Iterator iterator, BiConsumer consumer, - int maxConcurrency + int maxConcurrency, + ProgressLogger progressLogger ) { - try (var throttledIterator = new ThrottledIterator<>(completionListener, iterator, consumer, maxConcurrency)) { + try (var throttledIterator = new ThrottledIterator<>(completionListener, iterator, consumer, maxConcurrency, progressLogger)) { throttledIterator.run(); } } @@ -535,17 +548,20 @@ private static class ThrottledIterator implements Releasable { private final Iterator iterator; private final BiConsumer consumer; private final Semaphore permits; + private final ProgressLogger progressLogger; ThrottledIterator( ActionListener completionListener, Iterator iterator, BiConsumer consumer, - int maxConcurrency + int maxConcurrency, + ProgressLogger progressLogger ) { this.refCounted = wrap(completionListener); this.iterator = iterator; this.consumer = consumer; this.permits = new Semaphore(maxConcurrency); + this.progressLogger = progressLogger; } void run() { @@ -566,6 +582,7 @@ void run() { run(); } finally { refCounted.decRef(); + progressLogger.maybeLogProgress(); } }); try { @@ -581,4 +598,29 @@ public void close() { refCounted.close(); } } + + private class ProgressLogger { + private final String type; + private final long expectedMax; + private final long logFrequency; + private final AtomicLong currentCount = new AtomicLong(); + + ProgressLogger(String type, long expectedMax, long logFrequency) { + this.type = type; + this.expectedMax = expectedMax; + this.logFrequency = logFrequency; + } + + long getExpectedMax() { + return expectedMax; + } + + void maybeLogProgress() { + final var count = currentCount.incrementAndGet(); + if (count % logFrequency == 0) { + logger.info("[{}] processed [{}] of [{}] {}", repositoryName, count, expectedMax, type); + } + } + + } } From 24c2b6af5db1f928a2c8c8fc38d5d0f2de749049 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 18:55:53 +0000 Subject: [PATCH 03/10] Verify shard-level summary consistent with each snapshot --- .../blobstore/MetadataVerifier.java | 167 ++++++++++-------- 1 file changed, 98 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index b2745d8876796..5ca53e8266e2f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -25,6 +25,8 @@ import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots; +import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryVerificationException; @@ -43,6 +45,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; +import java.util.function.Function; import java.util.stream.Collectors; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; @@ -192,11 +195,16 @@ private void verifyIndices() { ); } + private record ShardContainerContents( + Map blobsByName, + BlobStoreIndexShardSnapshots blobStoreIndexShardSnapshots + ) {} + private class IndexVerifier { private final RefCounted indexRefs; private final IndexId indexId; private final Set expectedSnapshots; - private final Map>> shardBlobsListenersByShard = newConcurrentMap(); + private final Map> shardContainerContentsListener = newConcurrentMap(); private final Map> shardCountListenersByBlobId = newConcurrentMap(); IndexVerifier(RefCounted indexRefs, IndexId indexId) { @@ -206,20 +214,13 @@ private class IndexVerifier { } void run() { - - // TODO consider distributing the workload, giving each node a subset of indices to process - - final var indexSnapshots = repositoryData.getSnapshots(indexId); - - try (var indexMetadataChecksRef = wrap(makeVoidListener(indexRefs, this::onIndexMetadataChecksComplete))) { - runThrottled( - makeVoidListener(indexMetadataChecksRef, () -> {}), - indexSnapshots.iterator(), - this::verifyIndexSnapshot, - verifyRequest.getIndexSnapshotVerificationConcurrency(), - indexSnapshotProgressLogger - ); - } + runThrottled( + makeVoidListener(indexRefs, () -> {}), + repositoryData.getSnapshots(indexId).iterator(), + this::verifyIndexSnapshot, + verifyRequest.getIndexSnapshotVerificationConcurrency(), + indexSnapshotProgressLogger + ); } private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapshotId) { @@ -234,10 +235,23 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh final var shardCount = blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId) .getNumberOfShards(); for (int i = 0; i < shardCount; i++) { - shardBlobsListenersByShard.computeIfAbsent(i, shardId -> { - final var shardBlobsFuture = new ListenableActionFuture>(); - forkSupply(() -> blobStoreRepository.shardContainer(indexId, shardId).listBlobs(), shardBlobsFuture); - return shardBlobsFuture; + shardContainerContentsListener.computeIfAbsent(i, shardId -> { + final var shardContainerContentsFuture = new ListenableActionFuture(); + forkSupply( + () -> new ShardContainerContents( + blobStoreRepository.shardContainer(indexId, shardId).listBlobs(), + blobStoreRepository.getBlobStoreIndexShardSnapshots( + indexId, + shardId, + Objects.requireNonNull( + repositoryData.shardGenerations().getShardGen(indexId, shardId), + "shard generations for " + indexId + "/" + shardId + ) + ) + ), + shardContainerContentsFuture + ); + return shardContainerContentsFuture; }); } return shardCount; @@ -246,17 +260,17 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh }).addListener(makeListener(indexSnapshotRefs, shardCount -> { for (int i = 0; i < shardCount; i++) { final var shardId = i; - shardBlobsListenersByShard.get(i) + shardContainerContentsListener.get(i) .addListener( makeListener( indexSnapshotRefs, - shardBlobs -> forkSupply( + shardContainerContents -> forkSupply( indexSnapshotRefs, () -> blobStoreRepository.loadShardSnapshot( blobStoreRepository.shardContainer(indexId, shardId), snapshotId ), - shardSnapshot -> verifyShardSnapshot(snapshotId, shardId, shardBlobs, shardSnapshot) + shardSnapshot -> verifyShardSnapshot(snapshotId, shardId, shardContainerContents, shardSnapshot) ) ) ); @@ -267,7 +281,7 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh private void verifyShardSnapshot( SnapshotId snapshotId, int shardId, - Map shardBlobs, + ShardContainerContents shardContainerContents, BlobStoreIndexShardSnapshot shardSnapshot ) { if (shardSnapshot.snapshot().equals(snapshotId.getName()) == false) { @@ -281,7 +295,67 @@ private void verifyShardSnapshot( } for (final var fileInfo : shardSnapshot.indexFiles()) { - verifyFileInfo(snapshotId.toString(), shardId, shardBlobs, fileInfo); + verifyFileInfo(snapshotId.toString(), shardId, shardContainerContents.blobsByName(), fileInfo); + } + + boolean foundSnapshot = false; + for (SnapshotFiles summary : shardContainerContents.blobStoreIndexShardSnapshots().snapshots()) { + if (summary.snapshot().equals(snapshotId.getName())) { + foundSnapshot = true; + verifyConsistentShardFiles(snapshotId, shardId, shardSnapshot, summary); + break; + } + } + + if (foundSnapshot == false) { + addFailure("snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); + } + } + + private void verifyConsistentShardFiles( + SnapshotId snapshotId, + int shardId, + BlobStoreIndexShardSnapshot shardSnapshot, + SnapshotFiles summary + ) { + final var snapshotFiles = shardSnapshot.indexFiles() + .stream() + .collect(Collectors.toMap(BlobStoreIndexShardSnapshot.FileInfo::physicalName, Function.identity())); + + for (final var summaryFile : summary.indexFiles()) { + final var snapshotFile = snapshotFiles.get(summaryFile.physicalName()); + if (snapshotFile == null) { + addFailure( + "snapshot [%s] for shard [%s/%d] has no entry for file [%s] found in summary", + snapshotId, + indexId, + shardId, + summaryFile.physicalName() + ); + } else if (summaryFile.isSame(snapshotFile) == false) { + addFailure( + "snapshot [%s] for shard [%s/%d] has a mismatched entry for file [%s]", + snapshotId, + indexId, + shardId, + summaryFile.physicalName() + ); + } + } + + final var summaryFiles = summary.indexFiles() + .stream() + .collect(Collectors.toMap(BlobStoreIndexShardSnapshot.FileInfo::physicalName, Function.identity())); + for (final var snapshotFile : shardSnapshot.indexFiles()) { + if (summaryFiles.get(snapshotFile.physicalName()) == null) { + addFailure( + "snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary for file [%s]", + snapshotId, + indexId, + shardId, + snapshotFile.physicalName() + ); + } } } @@ -353,51 +427,6 @@ private void verifyFileInfo( } } } - - private void onIndexMetadataChecksComplete() { - if (shardCountListenersByBlobId.isEmpty()) { - throw new IllegalStateException(format("index [%s] has no metadata", indexId)); - } - - try (var shardGenerationChecksRef = wrap(makeVoidListener(indexRefs, () -> {}))) { - for (final var shardEntry : shardBlobsListenersByShard.entrySet()) { - verifyShardGenerations(shardGenerationChecksRef, shardEntry); - } - } - } - - private void verifyShardGenerations( - RefCounted shardGenerationChecksRef, - Map.Entry>> shardEntry - ) { - final int shardId = shardEntry.getKey(); - shardEntry.getValue() - .addListener( - makeListener( - shardGenerationChecksRef, - shardBlobs -> forkSupply( - shardGenerationChecksRef, - () -> blobStoreRepository.getBlobStoreIndexShardSnapshots( - indexId, - shardId, - Objects.requireNonNull( - repositoryData.shardGenerations().getShardGen(indexId, shardId), - "shard generations for " + indexId + "/" + shardId - ) - ), - blobStoreIndexShardSnapshots -> { - for (final var snapshotFiles : blobStoreIndexShardSnapshots.snapshots()) { - snapshotFiles.snapshot(); // TODO validate - snapshotFiles.shardStateIdentifier(); // TODO validate - for (final var fileInfo : snapshotFiles.indexFiles()) { - verifyFileInfo(snapshotFiles.snapshot(), shardId, shardBlobs, fileInfo); - } - } - } - ) - ) - ); - } } private ActionListener makeListener(RefCounted refCounted, CheckedConsumer consumer) { From 320747beca7739222fff2783c51fabfdc7b2fdb5 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 19:52:31 +0000 Subject: [PATCH 04/10] No point in double-checking the repo data if readonly, we read it from source every time --- .../repositories/blobstore/BlobStoreRepository.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0e7e73b2de242..c384ca7e64e33 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -3574,7 +3574,7 @@ public void verifyMetadataIntegrity( try (var metadataVerifier = new MetadataVerifier(this, request, repositoryData, isCancelledSupplier, l2)) { metadataVerifier.run(); } - }), () -> getRepositoryData(repositoryData.getGenId()))); + }), () -> readOnly ? repositoryData : getRepositoryData(repositoryData.getGenId()))); })); } From 7d23476e4f333869e20003520f62743eb20f1c8b Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 19:54:12 +0000 Subject: [PATCH 05/10] More logs --- .../repositories/blobstore/MetadataVerifier.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 5ca53e8266e2f..01c88d9cd7cfa 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -90,8 +90,8 @@ class MetadataVerifier implements Releasable { this.threadPoolPermits = new Semaphore(Math.max(1, verifyRequest.getThreadpoolConcurrency())); this.snapshotProgressLogger = new ProgressLogger("snapshots", repositoryData.getSnapshotIds().size(), 100); - this.indexProgressLogger = new ProgressLogger("indices", repositoryData.getIndices().size(), 100); - this.indexSnapshotProgressLogger = new ProgressLogger("index snapshots", repositoryData.getIndexSnapshotCount(), 10000); + this.indexProgressLogger = new ProgressLogger("indices", repositoryData.getIndices().size(), 20); + this.indexSnapshotProgressLogger = new ProgressLogger("index snapshots", repositoryData.getIndexSnapshotCount(), 1000); } @Override @@ -646,10 +646,9 @@ long getExpectedMax() { void maybeLogProgress() { final var count = currentCount.incrementAndGet(); - if (count % logFrequency == 0) { + if (count == expectedMax || count % logFrequency == 0) { logger.info("[{}] processed [{}] of [{}] {}", repositoryName, count, expectedMax, type); } } - } } From 7f259d720fc2a000b8738f4bda1b39ba64d4b90f Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 21:28:50 +0000 Subject: [PATCH 06/10] More info on failures --- .../blobstore/MetadataVerifier.java | 117 +++++++++++++----- 1 file changed, 87 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 01c88d9cd7cfa..485d436ba3b64 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -14,6 +14,7 @@ import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.blobstore.support.BlobMetadata; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -37,7 +38,6 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Queue; import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; @@ -100,9 +100,10 @@ public void close() { } private void addFailure(String format, Object... args) { - if (failureCount.incrementAndGet() <= verifyRequest.getMaxFailures()) { + final var failureNumber = failureCount.incrementAndGet(); + if (failureNumber <= verifyRequest.getMaxFailures()) { final var failure = format(format, args); - logger.debug("[{}] found metadata verification failure: {}", repositoryName, failure); + logger.debug("[{}] found metadata verification failure [{}]: {}", repositoryName, failureNumber, failure); failures.add(new RepositoryVerificationException(repositoryName, failure)); } } @@ -111,8 +112,9 @@ private void addFailure(Exception exception) { if (isCancelledSupplier.getAsBoolean() && exception instanceof TaskCancelledException) { return; } - if (failureCount.incrementAndGet() <= verifyRequest.getMaxFailures()) { - logger.debug(() -> format("[%s] exception during metadata verification: {}", repositoryName), exception); + final var failureNumber = failureCount.incrementAndGet(); + if (failureNumber <= verifyRequest.getMaxFailures()) { + logger.debug(() -> format("[%s] exception [%d] during metadata verification", repositoryName, failureNumber), exception); failures.add( exception instanceof RepositoryVerificationException rve ? rve @@ -168,13 +170,28 @@ private void verifySnapshot(RefCounted snapshotRefs, SnapshotId snapshotId) { } })); - forkSupply(snapshotRefs, () -> blobStoreRepository.getSnapshotGlobalMetadata(snapshotId), metadata -> { - if (metadata.indices().isEmpty() == false) { + forkSupply(snapshotRefs, () -> getSnapshotGlobalMetadata(snapshotId), metadata -> { + if (metadata != null && metadata.indices().isEmpty() == false) { addFailure("snapshot [%s] contains unexpected index metadata within global metadata", snapshotId); } }); } + private Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) { + try { + return blobStoreRepository.getSnapshotGlobalMetadata(snapshotId); + } catch (Exception e) { + addFailure( + new RepositoryVerificationException( + repositoryName, + format("failed to get snapshot global metadata for [%s]", snapshotId), + e + ) + ); + return null; + } + } + private void verifyIndices() { final var indicesMap = repositoryData.getIndices(); @@ -232,22 +249,14 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh shardCountListenersByBlobId.computeIfAbsent(indexMetaBlobId, ignored -> { final var shardCountFuture = new ListenableActionFuture(); forkSupply(() -> { - final var shardCount = blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId) - .getNumberOfShards(); + final var shardCount = getNumberOfShards(indexMetaBlobId, snapshotId); for (int i = 0; i < shardCount; i++) { shardContainerContentsListener.computeIfAbsent(i, shardId -> { final var shardContainerContentsFuture = new ListenableActionFuture(); forkSupply( () -> new ShardContainerContents( blobStoreRepository.shardContainer(indexId, shardId).listBlobs(), - blobStoreRepository.getBlobStoreIndexShardSnapshots( - indexId, - shardId, - Objects.requireNonNull( - repositoryData.shardGenerations().getShardGen(indexId, shardId), - "shard generations for " + indexId + "/" + shardId - ) - ) + getBlobStoreIndexShardSnapshots(shardId) ), shardContainerContentsFuture ); @@ -266,10 +275,7 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh indexSnapshotRefs, shardContainerContents -> forkSupply( indexSnapshotRefs, - () -> blobStoreRepository.loadShardSnapshot( - blobStoreRepository.shardContainer(indexId, shardId), - snapshotId - ), + () -> getBlobStoreIndexShardSnapshot(snapshotId, shardId), shardSnapshot -> verifyShardSnapshot(snapshotId, shardId, shardContainerContents, shardSnapshot) ) ) @@ -278,12 +284,60 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh })); } + private BlobStoreIndexShardSnapshot getBlobStoreIndexShardSnapshot(SnapshotId snapshotId, int shardId) { + try { + return blobStoreRepository.loadShardSnapshot(blobStoreRepository.shardContainer(indexId, shardId), snapshotId); + } catch (Exception e) { + addFailure( + new RepositoryVerificationException( + repositoryName, + format("failed to load shard %s/%d snapshot for %s", indexId, shardId, snapshotId), + e + ) + ); + return null; + } + } + + private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { + try { + return blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId).getNumberOfShards(); + } catch (Exception e) { + addFailure( + new RepositoryVerificationException( + repositoryName, + format("failed to load index %s metadata for %s from blob [%s]", indexId, snapshotId, indexMetaBlobId), + e + ) + ); + return 0; + } + } + + private BlobStoreIndexShardSnapshots getBlobStoreIndexShardSnapshots(int shardId) { + final var shardGen = repositoryData.shardGenerations().getShardGen(indexId, shardId); + if (shardGen == null) { + addFailure("unknown shard generation for %s/%d", indexId, shardId); + return null; + } + try { + return blobStoreRepository.getBlobStoreIndexShardSnapshots(indexId, shardId, shardGen); + } catch (Exception e) { + addFailure(e); + return null; + } + } + private void verifyShardSnapshot( SnapshotId snapshotId, int shardId, ShardContainerContents shardContainerContents, BlobStoreIndexShardSnapshot shardSnapshot ) { + if (shardSnapshot == null) { + return; + } + if (shardSnapshot.snapshot().equals(snapshotId.getName()) == false) { addFailure( "snapshot [%s] for shard [%s/%d] has mismatched name [%s]", @@ -298,17 +352,20 @@ private void verifyShardSnapshot( verifyFileInfo(snapshotId.toString(), shardId, shardContainerContents.blobsByName(), fileInfo); } - boolean foundSnapshot = false; - for (SnapshotFiles summary : shardContainerContents.blobStoreIndexShardSnapshots().snapshots()) { - if (summary.snapshot().equals(snapshotId.getName())) { - foundSnapshot = true; - verifyConsistentShardFiles(snapshotId, shardId, shardSnapshot, summary); - break; + final var blobStoreIndexShardSnapshots = shardContainerContents.blobStoreIndexShardSnapshots(); + if (blobStoreIndexShardSnapshots != null) { + boolean foundSnapshot = false; + for (SnapshotFiles summary : blobStoreIndexShardSnapshots.snapshots()) { + if (summary.snapshot().equals(snapshotId.getName())) { + foundSnapshot = true; + verifyConsistentShardFiles(snapshotId, shardId, shardSnapshot, summary); + break; + } } - } - if (foundSnapshot == false) { - addFailure("snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); + if (foundSnapshot == false) { + addFailure("snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); + } } } From 7a88397e9acc7b8792e0957c4eb1a8e29ffbc4c1 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 14 Dec 2022 21:31:54 +0000 Subject: [PATCH 07/10] Better formatting --- .../blobstore/MetadataVerifier.java | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 485d436ba3b64..2d3a7583910c6 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -199,7 +199,7 @@ private void verifyIndices() { final var name = indicesEntry.getKey(); final var indexId = indicesEntry.getValue(); if (name.equals(indexId.getName()) == false) { - addFailure("index name [%s] has mismatched name in [%s]", name, indexId); + addFailure("index name [%s] has mismatched name in %s", name, indexId); } } @@ -242,7 +242,7 @@ void run() { private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapshotId) { if (expectedSnapshots.contains(snapshotId) == false) { - addFailure("index [%s] has mismatched snapshot [%s]", indexId, snapshotId); + addFailure("index %s has mismatched snapshot [%s]", indexId, snapshotId); } final var indexMetaBlobId = repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, indexId); @@ -291,7 +291,7 @@ private BlobStoreIndexShardSnapshot getBlobStoreIndexShardSnapshot(SnapshotId sn addFailure( new RepositoryVerificationException( repositoryName, - format("failed to load shard %s/%d snapshot for %s", indexId, shardId, snapshotId), + format("failed to load shard %s[%d] snapshot for [%s]", indexId, shardId, snapshotId), e ) ); @@ -306,7 +306,7 @@ private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { addFailure( new RepositoryVerificationException( repositoryName, - format("failed to load index %s metadata for %s from blob [%s]", indexId, snapshotId, indexMetaBlobId), + format("failed to load index %s metadata for [%s] from blob [%s]", indexId, snapshotId, indexMetaBlobId), e ) ); @@ -317,7 +317,7 @@ private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { private BlobStoreIndexShardSnapshots getBlobStoreIndexShardSnapshots(int shardId) { final var shardGen = repositoryData.shardGenerations().getShardGen(indexId, shardId); if (shardGen == null) { - addFailure("unknown shard generation for %s/%d", indexId, shardId); + addFailure("unknown shard generation for %s[%d]", indexId, shardId); return null; } try { @@ -340,7 +340,7 @@ private void verifyShardSnapshot( if (shardSnapshot.snapshot().equals(snapshotId.getName()) == false) { addFailure( - "snapshot [%s] for shard [%s/%d] has mismatched name [%s]", + "snapshot [%s] for shard %s[%d] has mismatched name [%s]", snapshotId, indexId, shardId, @@ -364,7 +364,7 @@ private void verifyShardSnapshot( } if (foundSnapshot == false) { - addFailure("snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); + addFailure("snapshot [%s] for shard %s[%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); } } } @@ -383,7 +383,7 @@ private void verifyConsistentShardFiles( final var snapshotFile = snapshotFiles.get(summaryFile.physicalName()); if (snapshotFile == null) { addFailure( - "snapshot [%s] for shard [%s/%d] has no entry for file [%s] found in summary", + "snapshot [%s] for shard %s[%d] has no entry for file [%s] found in summary", snapshotId, indexId, shardId, @@ -391,7 +391,7 @@ private void verifyConsistentShardFiles( ); } else if (summaryFile.isSame(snapshotFile) == false) { addFailure( - "snapshot [%s] for shard [%s/%d] has a mismatched entry for file [%s]", + "snapshot [%s] for shard %s[%d] has a mismatched entry for file [%s]", snapshotId, indexId, shardId, @@ -406,7 +406,7 @@ private void verifyConsistentShardFiles( for (final var snapshotFile : shardSnapshot.indexFiles()) { if (summaryFiles.get(snapshotFile.physicalName()) == null) { addFailure( - "snapshot [%s] for shard [%s/%d] has no entry in the shard-level summary for file [%s]", + "snapshot [%s] for shard %s[%d] has no entry in the shard-level summary for file [%s]", snapshotId, indexId, shardId, @@ -435,7 +435,7 @@ private void verifyFileInfo( final var actualLength = ByteSizeValue.ofBytes(fileInfo.metadata().hash().length); if (fileLength.getBytes() != actualLength.getBytes()) { addFailure( - "snapshot [%s] for shard [%s/%d] has virtual blob [%s] for [%s] with length [%s] instead of [%s]", + "snapshot [%s] for shard %s[%d] has virtual blob [%s] for [%s] with length [%s] instead of [%s]", snapshot, indexId, shardId, @@ -452,7 +452,7 @@ private void verifyFileInfo( final var partLength = ByteSizeValue.ofBytes(fileInfo.partBytes(part)); if (blobInfo == null) { addFailure( - "snapshot [%s] for shard [%s/%d] has missing blob [%s] for [%s] part [%d/%d]; " + "snapshot [%s] for shard %s[%d] has missing blob [%s] for [%s] part [%d/%d]; " + "file length [%s], part length [%s]", snapshot, indexId, @@ -467,7 +467,7 @@ private void verifyFileInfo( } else if (blobInfo.length() != partLength.getBytes()) { addFailure( - "snapshot [%s] for shard [%s/%d] has blob [%s] for [%s] part [%d/%d] with length [%s] instead of [%s]; " + "snapshot [%s] for shard %s[%d] has blob [%s] for [%s] part [%d/%d] with length [%s] instead of [%s]; " + "file length [%s]", snapshot, indexId, From f9024eb88617dc989235b8ff6899fb761074c18c Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 15 Dec 2022 10:20:32 +0000 Subject: [PATCH 08/10] Allow single-index analysis --- .../blobstore/BlobStoreMetadataIntegrityIT.java | 3 ++- .../VerifyRepositoryIntegrityAction.java | 17 ++++++++++++++++- .../blobstore/MetadataVerifier.java | 13 ++++++++++++- .../RestVerifyRepositoryIntegrityAction.java | 2 ++ 4 files changed, 32 insertions(+), 3 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java index 945e360312f7a..720e372a078f7 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -93,7 +94,7 @@ public void testIntegrityCheck() throws Exception { createFullSnapshot(REPOSITORY_NAME, "test-snapshot-" + snapshotIndex); } - final var request = new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, 5, 5, 5, 5, 10000, false); + final var request = new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, Strings.EMPTY_ARRAY, 5, 5, 5, 5, 10000, false); final var response = PlainActionFuture.get( listener -> client().execute(VerifyRepositoryIntegrityAction.INSTANCE, request, listener), diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java index 6599c1e2f8e35..c815c562713b0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java @@ -42,6 +42,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; public class VerifyRepositoryIntegrityAction extends ActionType { @@ -55,6 +56,7 @@ private VerifyRepositoryIntegrityAction() { public static class Request extends MasterNodeReadRequest { private final String repository; + private final String[] indices; private final int threadpoolConcurrency; private final int snapshotVerificationConcurrency; private final int indexVerificationConcurrency; @@ -64,6 +66,7 @@ public static class Request extends MasterNodeReadRequest { public Request( String repository, + String[] indices, int threadpoolConcurrency, int snapshotVerificationConcurrency, int indexVerificationConcurrency, @@ -72,6 +75,7 @@ public Request( boolean permitMissingSnapshotDetails ) { this.repository = repository; + this.indices = Objects.requireNonNull(indices, "indices"); this.threadpoolConcurrency = requireMin("threadpoolConcurrency", 0, threadpoolConcurrency); this.snapshotVerificationConcurrency = requireMin("snapshotVerificationConcurrency", 1, snapshotVerificationConcurrency); this.indexVerificationConcurrency = requireMin("indexVerificationConcurrency", 1, indexVerificationConcurrency); @@ -94,6 +98,7 @@ private static int requireMin(String name, int min, int value) { public Request(StreamInput in) throws IOException { super(in); this.repository = in.readString(); + this.indices = in.readStringArray(); this.threadpoolConcurrency = in.readVInt(); this.snapshotVerificationConcurrency = in.readVInt(); this.indexVerificationConcurrency = in.readVInt(); @@ -106,6 +111,7 @@ public Request(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(repository); + out.writeStringArray(indices); out.writeVInt(threadpoolConcurrency); out.writeVInt(snapshotVerificationConcurrency); out.writeVInt(indexVerificationConcurrency); @@ -124,6 +130,14 @@ public Task createTask(long id, String type, String action, TaskId parentTaskId, return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers); } + public String getRepository() { + return repository; + } + + public String[] getIndices() { + return indices; + } + public int getThreadpoolConcurrency() { return threadpoolConcurrency; } @@ -152,6 +166,7 @@ public Request withDefaultThreadpoolConcurrency(Settings settings) { if (threadpoolConcurrency == 0) { final var request = new Request( repository, + indices, Math.max(1, EsExecutors.allocatedProcessors(settings) / 2), snapshotVerificationConcurrency, indexVerificationConcurrency, @@ -249,7 +264,7 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) protected void masterOperation(Task task, Request request, ClusterState state, ActionListener listener) throws Exception { // TODO add mechanism to block blob deletions while this is running final var cancellableTask = (CancellableTask) task; - repositoriesService.repository(request.repository) + repositoriesService.repository(request.getRepository()) .verifyMetadataIntegrity( request.withDefaultThreadpoolConcurrency(clusterService.getSettings()), listener.map(Response::new), diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 2d3a7583910c6..6c382b67f602c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -69,6 +69,7 @@ class MetadataVerifier implements Releasable { private final ProgressLogger snapshotProgressLogger; private final ProgressLogger indexProgressLogger; private final ProgressLogger indexSnapshotProgressLogger; + private final Set requestedIndices; MetadataVerifier( BlobStoreRepository blobStoreRepository, @@ -92,6 +93,8 @@ class MetadataVerifier implements Releasable { this.snapshotProgressLogger = new ProgressLogger("snapshots", repositoryData.getSnapshotIds().size(), 100); this.indexProgressLogger = new ProgressLogger("indices", repositoryData.getIndices().size(), 20); this.indexSnapshotProgressLogger = new ProgressLogger("index snapshots", repositoryData.getIndexSnapshotCount(), 1000); + + this.requestedIndices = Set.of(verifyRequest.getIndices()); } @Override @@ -136,7 +139,11 @@ public void run() { indexSnapshotProgressLogger.getExpectedMax() ); - verifySnapshots(); + if (requestedIndices.isEmpty()) { + verifySnapshots(); + } else { + verifyIndices(); + } } private void verifySnapshots() { @@ -231,6 +238,10 @@ private class IndexVerifier { } void run() { + if (requestedIndices.isEmpty() == false && requestedIndices.contains(indexId.getName()) == false) { + return; + } + runThrottled( makeVoidListener(indexRefs, () -> {}), repositoryData.getSnapshots(indexId).iterator(), diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java index c6fda490d40e4..169e56ab03e25 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.client.internal.node.NodeClient; +import org.elasticsearch.common.Strings; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestCancellableNodeClient; @@ -35,6 +36,7 @@ public String getName() { public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { final var verifyRequest = new VerifyRepositoryIntegrityAction.Request( request.param("repository"), + request.paramAsStringArray("indices", Strings.EMPTY_ARRAY), request.paramAsInt("threadpool_concurrency", 0), request.paramAsInt("snapshot_verification_concurrency", 5), request.paramAsInt("index_verification_concurrency", 5), From 30d9928d96bf64e65cc24a2eadab5fed64b24eb7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 15 Dec 2022 10:52:04 +0000 Subject: [PATCH 09/10] Misc improvements --- .../repositories/s3/S3Service.java | 1 + .../blobstore/MetadataVerifier.java | 166 +++++++++++++----- 2 files changed, 126 insertions(+), 41 deletions(-) diff --git a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java index 2788bac6600b9..80ad0fe2631e6 100644 --- a/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java +++ b/modules/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Service.java @@ -209,6 +209,7 @@ static ClientConfiguration buildConfiguration(S3ClientSettings clientSettings) { clientConfiguration.setMaxErrorRetry(clientSettings.maxRetries); clientConfiguration.setUseThrottleRetries(clientSettings.throttleRetries); clientConfiguration.setSocketTimeout(clientSettings.readTimeoutMillis); + clientConfiguration.setMaxConnections(200); // TODO WIP reduce this return clientConfiguration; } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 6c382b67f602c..94e5a5148b6ed 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -42,6 +42,7 @@ import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; @@ -104,9 +105,9 @@ public void close() { private void addFailure(String format, Object... args) { final var failureNumber = failureCount.incrementAndGet(); + final var failure = format(format, args); + logger.debug("[{}] found metadata verification failure [{}]: {}", repositoryName, failureNumber, failure); if (failureNumber <= verifyRequest.getMaxFailures()) { - final var failure = format(format, args); - logger.debug("[{}] found metadata verification failure [{}]: {}", repositoryName, failureNumber, failure); failures.add(new RepositoryVerificationException(repositoryName, failure)); } } @@ -116,8 +117,8 @@ private void addFailure(Exception exception) { return; } final var failureNumber = failureCount.incrementAndGet(); + logger.debug(() -> format("[%s] exception [%d] during metadata verification", repositoryName, failureNumber), exception); if (failureNumber <= verifyRequest.getMaxFailures()) { - logger.debug(() -> format("[%s] exception [%d] during metadata verification", repositoryName, failureNumber), exception); failures.add( exception instanceof RepositoryVerificationException rve ? rve @@ -230,11 +231,13 @@ private class IndexVerifier { private final Set expectedSnapshots; private final Map> shardContainerContentsListener = newConcurrentMap(); private final Map> shardCountListenersByBlobId = newConcurrentMap(); + private final AtomicInteger totalSnapshotCounter = new AtomicInteger(); + private final AtomicInteger restorableSnapshotCounter = new AtomicInteger(); IndexVerifier(RefCounted indexRefs, IndexId indexId) { this.indexRefs = indexRefs; this.indexId = indexId; - this.expectedSnapshots = snapshotsByIndex.get(this.indexId.getName()); + this.expectedSnapshots = snapshotsByIndex.get(indexId.getName()); } void run() { @@ -243,7 +246,7 @@ void run() { } runThrottled( - makeVoidListener(indexRefs, () -> {}), + makeVoidListener(indexRefs, () -> logRestorability(totalSnapshotCounter.get(), restorableSnapshotCounter.get())), repositoryData.getSnapshots(indexId).iterator(), this::verifyIndexSnapshot, verifyRequest.getIndexSnapshotVerificationConcurrency(), @@ -251,7 +254,26 @@ void run() { ); } + private void logRestorability(int totalSnapshotCount, int restorableSnapshotCount) { + if (isCancelledSupplier.getAsBoolean() == false) { + if (totalSnapshotCount == restorableSnapshotCount) { + logger.debug("[{}] index {} is fully restorable from [{}] snapshots", repositoryName, indexId, totalSnapshotCount); + } else { + logger.debug( + "[{}] index {} is not fully restorable: of [{}] snapshots, [{}] are restorable and [{}] are not", + repositoryName, + indexId, + totalSnapshotCount, + restorableSnapshotCount, + totalSnapshotCount - restorableSnapshotCount + ); + } + } + } + private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapshotId) { + totalSnapshotCounter.incrementAndGet(); + if (expectedSnapshots.contains(snapshotId) == false) { addFailure("index %s has mismatched snapshot [%s]", indexId, snapshotId); } @@ -278,19 +300,32 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh }, shardCountFuture); return shardCountFuture; }).addListener(makeListener(indexSnapshotRefs, shardCount -> { - for (int i = 0; i < shardCount; i++) { - final var shardId = i; - shardContainerContentsListener.get(i) - .addListener( - makeListener( - indexSnapshotRefs, - shardContainerContents -> forkSupply( - indexSnapshotRefs, - () -> getBlobStoreIndexShardSnapshot(snapshotId, shardId), - shardSnapshot -> verifyShardSnapshot(snapshotId, shardId, shardContainerContents, shardSnapshot) + final var restorableShardCount = new AtomicInteger(); + try (var shardSnapshotsRefs = wrap(makeVoidListener(indexSnapshotRefs, () -> { + if (shardCount > 0 && shardCount == restorableShardCount.get()) { + restorableSnapshotCounter.incrementAndGet(); + } + }))) { + for (int i = 0; i < shardCount; i++) { + final var shardId = i; + shardContainerContentsListener.get(i) + .addListener( + makeListener( + shardSnapshotsRefs, + shardContainerContents -> forkSupply( + shardSnapshotsRefs, + () -> getBlobStoreIndexShardSnapshot(snapshotId, shardId), + shardSnapshot -> verifyShardSnapshot( + snapshotId, + shardId, + shardContainerContents, + shardSnapshot, + restorableShardCount::incrementAndGet + ) + ) ) - ) - ); + ); + } } })); } @@ -310,6 +345,14 @@ private BlobStoreIndexShardSnapshot getBlobStoreIndexShardSnapshot(SnapshotId sn } } + private List getSnapshotsWithIndexMetadataBlob(String indexMetaBlobId) { + final var indexMetaDataGenerations = repositoryData.indexMetaDataGenerations(); + return repositoryData.getSnapshotIds() + .stream() + .filter(s -> indexMetaBlobId.equals(indexMetaDataGenerations.indexMetaBlobId(s, indexId))) + .toList(); + } + private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { try { return blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId).getNumberOfShards(); @@ -317,7 +360,12 @@ private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { addFailure( new RepositoryVerificationException( repositoryName, - format("failed to load index %s metadata for [%s] from blob [%s]", indexId, snapshotId, indexMetaBlobId), + format( + "failed to load index %s metadata from blob [%s] for %s", + indexId, + indexMetaBlobId, + getSnapshotsWithIndexMetadataBlob(indexMetaBlobId) + ), e ) ); @@ -343,7 +391,8 @@ private void verifyShardSnapshot( SnapshotId snapshotId, int shardId, ShardContainerContents shardContainerContents, - BlobStoreIndexShardSnapshot shardSnapshot + BlobStoreIndexShardSnapshot shardSnapshot, + Runnable runIfRestorable ) { if (shardSnapshot == null) { return; @@ -359,8 +408,12 @@ private void verifyShardSnapshot( ); } + var restorable = true; for (final var fileInfo : shardSnapshot.indexFiles()) { - verifyFileInfo(snapshotId.toString(), shardId, shardContainerContents.blobsByName(), fileInfo); + restorable &= verifyFileInfo(snapshotId.toString(), shardId, shardContainerContents.blobsByName(), fileInfo); + } + if (restorable) { + runIfRestorable.run(); } final var blobStoreIndexShardSnapshots = shardContainerContents.blobStoreIndexShardSnapshots(); @@ -435,7 +488,7 @@ private static String formatExact(ByteSizeValue byteSizeValue) { } } - private void verifyFileInfo( + private boolean verifyFileInfo( String snapshot, int shardId, Map shardBlobs, @@ -455,6 +508,7 @@ private void verifyFileInfo( formatExact(actualLength), formatExact(fileLength) ); + return false; } } else { for (int part = 0; part < fileInfo.numberOfParts(); part++) { @@ -475,7 +529,7 @@ private void verifyFileInfo( formatExact(fileLength), formatExact(partLength) ); - + return false; } else if (blobInfo.length() != partLength.getBytes()) { addFailure( "snapshot [%s] for shard %s[%d] has blob [%s] for [%s] part [%d/%d] with length [%s] instead of [%s]; " @@ -491,9 +545,11 @@ private void verifyFileInfo( formatExact(partLength), formatExact(fileLength) ); + return false; } } } + return true; } } @@ -538,7 +594,8 @@ public void onRejection(Exception e) { try { runnable.onRejection(e); } finally { - onCompletion(); + threadPoolPermits.release(); + // no need to call tryProcessQueue() again here, we're still running it } } @@ -641,7 +698,7 @@ private static void runThrottled( } private static class ThrottledIterator implements Releasable { - private final RefCountedListenerWrapper refCounted; + private final RefCountedListenerWrapper throttleRefs; private final Iterator iterator; private final BiConsumer consumer; private final Semaphore permits; @@ -654,7 +711,7 @@ private static class ThrottledIterator implements Releasable { int maxConcurrency, ProgressLogger progressLogger ) { - this.refCounted = wrap(completionListener); + this.throttleRefs = wrap(completionListener); this.iterator = iterator; this.consumer = consumer; this.permits = new Semaphore(maxConcurrency); @@ -672,27 +729,54 @@ void run() { return; } } - refCounted.incRef(); - final var itemRefCount = AbstractRefCounted.of(() -> { - permits.release(); - try { - run(); - } finally { - refCounted.decRef(); - progressLogger.maybeLogProgress(); - } - }); - try { - consumer.accept(itemRefCount, item); - } finally { - itemRefCount.decRef(); + try (var itemRefsWrapper = new RefCountedWrapper()) { + consumer.accept(itemRefsWrapper.unwrap(), item); } } } @Override public void close() { - refCounted.close(); + throttleRefs.close(); + } + + // Wraps a RefCounted with protection against calling back into run() from within safeDecRef() + private class RefCountedWrapper implements Releasable { + private boolean isRecursive; + private final RefCounted itemRefs = AbstractRefCounted.of(this::onItemCompletion); + + RefCountedWrapper() { + throttleRefs.incRef(); + } + + RefCounted unwrap() { + return itemRefs; + } + + private synchronized boolean isRecursive() { + return isRecursive; + } + + private void onItemCompletion() { + progressLogger.maybeLogProgress(); + permits.release(); + try { + if (isRecursive() == false) { + run(); + } + } finally { + throttleRefs.decRef(); + } + } + + @Override + public synchronized void close() { + // if the decRef() below releases the last ref and calls onItemCompletion(), no other thread blocks on us; conversely + // if decRef() doesn't release the last ref then we release this mutex immediately so the closing thread can proceed + isRecursive = true; + itemRefs.decRef(); + isRecursive = false; + } } } @@ -714,7 +798,7 @@ long getExpectedMax() { void maybeLogProgress() { final var count = currentCount.incrementAndGet(); - if (count == expectedMax || count % logFrequency == 0) { + if (count == expectedMax || count % logFrequency == 0 && isCancelledSupplier.getAsBoolean() == false) { logger.info("[{}] processed [{}] of [{}] {}", repositoryName, count, expectedMax, type); } } From 55bd713fe2fd55cee70a509462f643a2131c62bc Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 20 Dec 2022 09:33:20 +0000 Subject: [PATCH 10/10] Rework --- .../BlobStoreMetadataIntegrityIT.java | 301 +++++-- .../VerifyRepositoryIntegrityAction.java | 248 +++--- .../util/concurrent/ThrottledIterator.java | 152 ++++ .../common/xcontent/ChunkedToXContent.java | 5 - .../java/org/elasticsearch/node/Node.java | 3 +- .../repositories/RepositoriesService.java | 13 + .../repositories/Repository.java | 10 +- .../blobstore/BlobStoreRepository.java | 39 +- .../blobstore/MetadataVerifier.java | 841 ++++++++++-------- .../action/RestChunkedToXContentListener.java | 5 +- .../RestVerifyRepositoryIntegrityAction.java | 28 +- .../concurrent/ThrottledIteratorTests.java | 110 +++ 12 files changed, 1148 insertions(+), 607 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/common/util/concurrent/ThrottledIterator.java create mode 100644 server/src/test/java/org/elasticsearch/common/util/concurrent/ThrottledIteratorTests.java diff --git a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java index 720e372a078f7..47a2659aa92c3 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/repositories/blobstore/BlobStoreMetadataIntegrityIT.java @@ -8,6 +8,8 @@ package org.elasticsearch.repositories.blobstore; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionResponse; import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.support.PlainActionFuture; @@ -15,14 +17,17 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.query.ExistsQueryBuilder; +import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshotsIntegritySuppressor; import org.elasticsearch.repositories.RepositoriesService; -import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.RepositoryException; -import org.elasticsearch.repositories.RepositoryVerificationException; -import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.snapshots.mockstore.MockRepository; +import org.elasticsearch.tasks.TaskInfo; import org.elasticsearch.test.CorruptionUtils; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.After; import org.junit.Before; @@ -31,14 +36,14 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.not; public class BlobStoreMetadataIntegrityIT extends AbstractSnapshotIntegTestCase { @@ -60,21 +65,24 @@ public void enableIntegrityChecks() { integrityCheckSuppressor = null; } + @TestLogging(reason = "testing", value = "org.elasticsearch.repositories.blobstore.MetadataVerifier:DEBUG") public void testIntegrityCheck() throws Exception { final var repoPath = randomRepoPath(); createRepository( REPOSITORY_NAME, - "fs", + "mock", Settings.builder().put(BlobStoreRepository.SUPPORT_URL_REPO.getKey(), false).put("location", repoPath) ); - final var repository = internalCluster().getCurrentMasterNodeInstance(RepositoriesService.class).repository(REPOSITORY_NAME); + final MockRepository repository = (MockRepository) internalCluster().getCurrentMasterNodeInstance(RepositoriesService.class) + .repository(REPOSITORY_NAME); final var indexCount = between(1, 3); for (int i = 0; i < indexCount; i++) { createIndexWithRandomDocs("test-index-" + i, between(1, 1000)); } - for (int snapshotIndex = 0; snapshotIndex < 4; snapshotIndex++) { + final var snapshotCount = between(2, 4); + for (int snapshotIndex = 0; snapshotIndex < snapshotCount; snapshotIndex++) { final var indexRequests = new ArrayList(); for (int i = 0; i < indexCount; i++) { if (randomBoolean()) { @@ -91,39 +99,80 @@ public void testIntegrityCheck() throws Exception { } indexRandom(true, indexRequests); assertEquals(0, client().admin().indices().prepareFlush().get().getFailedShards()); - createFullSnapshot(REPOSITORY_NAME, "test-snapshot-" + snapshotIndex); + final var snapshotInfo = clusterAdmin().prepareCreateSnapshot(REPOSITORY_NAME, "test-snapshot-" + snapshotIndex) + .setIncludeGlobalState(randomBoolean()) + .setWaitForCompletion(true) + .get() + .getSnapshotInfo(); + assertThat(snapshotInfo.successfulShards(), is(snapshotInfo.totalShards())); + assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS)); } - final var request = new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, Strings.EMPTY_ARRAY, 5, 5, 5, 5, 10000, false); + repository.setBlockOnReadIndexMeta(); - final var response = PlainActionFuture.get( - listener -> client().execute(VerifyRepositoryIntegrityAction.INSTANCE, request, listener), - 30, - TimeUnit.SECONDS - ); - assertThat(response.getRestStatus(), equalTo(RestStatus.OK)); - assertThat(response.getExceptions(), empty()); + final var tasksFuture = new PlainActionFuture>(); + repository.threadPool().generic().execute(() -> { + try { + assertBusy(() -> assertTrue(repository.blocked())); + } catch (Exception e) { + throw new AssertionError(e); + } + + ActionListener.completeWith( + tasksFuture, + () -> client().admin() + .cluster() + .prepareListTasks() + .setDetailed(true) + .get() + .getTasks() + .stream() + .filter(t -> t.action().equals(VerifyRepositoryIntegrityAction.NAME) && t.status() != null) + .toList() + ); + + repository.unblock(); + }); + + verifyAndAssertSuccessful(indexCount); + + final var tasks = tasksFuture.actionGet(30, TimeUnit.SECONDS); + assertThat(tasks, not(empty())); + for (TaskInfo task : tasks) { + if (task.status()instanceof VerifyRepositoryIntegrityAction.Status status) { + assertEquals(REPOSITORY_NAME, status.repositoryName()); + assertThat(status.repositoryGeneration(), greaterThan(0L)); + assertEquals(snapshotCount, status.snapshotCount()); + assertEquals(snapshotCount, status.snapshotsVerified()); + assertEquals(indexCount, status.indexCount()); + assertEquals(0, status.indicesVerified()); + assertThat(status.indexSnapshotCount(), greaterThanOrEqualTo((long) indexCount)); + assertEquals(0, status.indexSnapshotsVerified()); + assertEquals(0, status.anomalyCount()); + } else { + assert false : Strings.toString(task); + } + } final var tempDir = createTempDir(); + final var repositoryData = PlainActionFuture.get(repository::getRepositoryData, 10, TimeUnit.SECONDS); + final var repositoryDataBlob = repoPath.resolve("index-" + repositoryData.getGenId()); + final List blobs; try (var paths = Files.walk(repoPath)) { - blobs = paths.filter(Files::isRegularFile).sorted().toList(); + blobs = paths.filter(path -> Files.isRegularFile(path) && path.equals(repositoryDataBlob) == false).sorted().toList(); } - for (final var blob : blobs) { - logger.info("repo contents: {}", blob); - } - - final var repositoryDataFuture = new PlainActionFuture(); - repository.getRepositoryData(repositoryDataFuture); - final var repositoryData = repositoryDataFuture.get(); - final var repositoryDataBlob = repoPath.resolve("index-" + repositoryData.getGenId()); for (int i = 0; i < 2000; i++) { final var blobToDamage = randomFrom(blobs); final var isDataBlob = blobToDamage.getFileName().toString().startsWith(BlobStoreRepository.UPLOADED_DATA_BLOB_PREFIX); - final var isIndexBlob = blobToDamage.equals(repositoryDataBlob); - if (isDataBlob || isIndexBlob || randomBoolean()) { + final var truncate = randomBoolean(); + if (truncate) { + logger.info("--> truncating {}", blobToDamage); + Files.copy(blobToDamage, tempDir.resolve("tmp")); + Files.write(blobToDamage, new byte[0]); + } else if (isDataBlob || randomBoolean()) { logger.info("--> deleting {}", blobToDamage); Files.move(blobToDamage, tempDir.resolve("tmp")); } else { @@ -132,48 +181,168 @@ public void testIntegrityCheck() throws Exception { CorruptionUtils.corruptFile(random(), blobToDamage); } try { - final var isCancelled = new AtomicBoolean(); - - final var verificationResponse = PlainActionFuture.get( - (PlainActionFuture> listener) -> repository.verifyMetadataIntegrity( - request, - listener, - () -> { - if (rarely() && rarely()) { - isCancelled.set(true); - return true; - } - return isCancelled.get(); + // TODO include some cancellation tests + + final var anomalies = verifyAndGetAnomalies(indexCount, repoPath.relativize(blobToDamage)); + if (isDataBlob) { + final var expectedAnomaly = truncate + ? MetadataVerifier.Anomaly.MISMATCHED_BLOB_LENGTH + : MetadataVerifier.Anomaly.MISSING_BLOB; + var foundExpectedAnomaly = false; + for (SearchHit anomaly : anomalies) { + final var source = anomaly.getSourceAsMap(); + if (expectedAnomaly.toString().equals(source.get("anomaly")) + && blobToDamage.getFileName().toString().equals(source.get("blob_name"))) { + foundExpectedAnomaly = true; + break; } - ), - 30, - TimeUnit.SECONDS - ); - assertThat(verificationResponse, not(empty())); - final var responseString = verificationResponse.stream().map(Throwable::getMessage).collect(Collectors.joining("\n")); - if (isCancelled.get()) { - assertThat(responseString, containsString("verification task cancelled before completion")); - } - if (isDataBlob && isCancelled.get() == false) { - assertThat( - responseString, - allOf(containsString(blobToDamage.getFileName().toString()), containsString("missing blob")) - ); + } + assertTrue(foundExpectedAnomaly); } - } catch (RepositoryException e) { - // ok, this means e.g. we couldn't even read the index blob + + // + // final var isCancelled = new AtomicBoolean(); + // + // final var verificationResponse = PlainActionFuture.get( + // (PlainActionFuture listener) -> repository.verifyMetadataIntegrity( + // client(), + // () -> new RecyclerBytesStreamOutput(NON_RECYCLING_INSTANCE), + // request, + // listener, + // () -> { + // if (rarely() && rarely()) { + // isCancelled.set(true); + // return true; + // } + // return isCancelled.get(); + // } + // ), + // 30, + // TimeUnit.SECONDS + // ); + // for (SearchHit hit : client().prepareSearch("metadata_verification_results").setSize(10000).get().getHits().getHits()) { + // logger.info("--> {}", Strings.toString(hit)); + // } + // assertThat(verificationResponse, not(nullValue())); + // final var responseString = verificationResponse.stream().map(Throwable::getMessage).collect(Collectors.joining("\n")); + // if (isCancelled.get()) { + // assertThat(responseString, containsString("verification task cancelled before completion")); + // } + // if (isDataBlob && isCancelled.get() == false) { + // assertThat( + // responseString, + // allOf(containsString(blobToDamage.getFileName().toString()), containsString("missing blob")) + // ); + // } } finally { Files.deleteIfExists(blobToDamage); Files.move(tempDir.resolve("tmp"), blobToDamage); } - final var repairResponse = PlainActionFuture.get( - listener -> client().execute(VerifyRepositoryIntegrityAction.INSTANCE, request, listener), - 30, - TimeUnit.SECONDS - ); - assertThat(repairResponse.getRestStatus(), equalTo(RestStatus.OK)); - assertThat(repairResponse.getExceptions(), empty()); + verifyAndAssertSuccessful(indexCount); } } + + private void verifyAndAssertSuccessful(int indexCount) { + PlainActionFuture.get( + listener -> client().execute( + VerifyRepositoryIntegrityAction.INSTANCE, + new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, Strings.EMPTY_ARRAY, 0, 0, 0, 0), + listener + ), + 30, + TimeUnit.SECONDS + ); + assertEquals( + 0, + client().prepareSearch("metadata_verification_results") + .setSize(0) + .setQuery(new ExistsQueryBuilder("anomaly")) + .get() + .getHits() + .getTotalHits().value + ); + assertEquals( + indexCount, + client().prepareSearch("metadata_verification_results") + .setSize(0) + .setQuery(new ExistsQueryBuilder("restorability")) + .setTrackTotalHits(true) + .get() + .getHits() + .getTotalHits().value + ); + assertEquals( + indexCount, + client().prepareSearch("metadata_verification_results") + .setSize(0) + .setQuery(new TermQueryBuilder("restorability", "full")) + .setTrackTotalHits(true) + .get() + .getHits() + .getTotalHits().value + ); + assertEquals( + 0, + client().prepareSearch("metadata_verification_results") + .setSize(1) + .setQuery(new TermQueryBuilder("completed", true)) + .get() + .getHits() + .getHits()[0].getSourceAsMap().get("total_anomalies") + ); + assertAcked(client().admin().indices().prepareDelete("metadata_verification_results")); + } + + private SearchHit[] verifyAndGetAnomalies(long indexCount, Path damagedBlob) { + PlainActionFuture.get( + listener -> client().execute( + VerifyRepositoryIntegrityAction.INSTANCE, + new VerifyRepositoryIntegrityAction.Request(REPOSITORY_NAME, Strings.EMPTY_ARRAY, 0, 0, 0, 0), + listener + ), + 30, + TimeUnit.SECONDS + ); + final var anomalyHits = client().prepareSearch("metadata_verification_results") + .setSize(10000) + .setQuery(new ExistsQueryBuilder("anomaly")) + .get() + .getHits(); + assertThat(anomalyHits.getTotalHits().value, greaterThan(0L)); + assertEquals( + indexCount, + client().prepareSearch("metadata_verification_results") + .setSize(0) + .setQuery(new ExistsQueryBuilder("restorability")) + .setTrackTotalHits(true) + .get() + .getHits() + .getTotalHits().value + ); + final var damagedFileName = damagedBlob.getFileName().toString(); + assertThat( + client().prepareSearch("metadata_verification_results") + .setSize(0) + .setQuery(new TermQueryBuilder("restorability", "full")) + .setTrackTotalHits(true) + .get() + .getHits() + .getTotalHits().value, + damagedFileName.startsWith(BlobStoreRepository.SNAPSHOT_PREFIX) + || damagedFileName.startsWith(BlobStoreRepository.UPLOADED_DATA_BLOB_PREFIX) + || (damagedFileName.startsWith(BlobStoreRepository.METADATA_PREFIX) && damagedBlob.startsWith("indices")) + ? lessThan(indexCount) + : equalTo(indexCount) + ); + final int totalAnomalies = (int) client().prepareSearch("metadata_verification_results") + .setSize(1) + .setQuery(new TermQueryBuilder("completed", true)) + .get() + .getHits() + .getHits()[0].getSourceAsMap().get("total_anomalies"); + assertThat(totalAnomalies, greaterThan(0)); + assertAcked(client().admin().indices().prepareDelete("metadata_verification_results")); + return anomalyHits.getHits(); + } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java index c815c562713b0..31c8c4edba25e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/integrity/VerifyRepositoryIntegrityAction.java @@ -8,7 +8,6 @@ package org.elasticsearch.action.admin.cluster.repositories.integrity; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -16,81 +15,69 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.MasterNodeReadRequest; import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateListener; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.EsExecutors; -import org.elasticsearch.common.xcontent.ChunkedToXContent; import org.elasticsearch.repositories.RepositoriesService; -import org.elasticsearch.repositories.RepositoryVerificationException; -import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.CancellableTask; -import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; -import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; +import java.util.function.Supplier; -public class VerifyRepositoryIntegrityAction extends ActionType { +public class VerifyRepositoryIntegrityAction extends ActionType { public static final VerifyRepositoryIntegrityAction INSTANCE = new VerifyRepositoryIntegrityAction(); public static final String NAME = "cluster:admin/repository/verify_integrity"; private VerifyRepositoryIntegrityAction() { - super(NAME, VerifyRepositoryIntegrityAction.Response::new); + super(NAME, in -> ActionResponse.Empty.INSTANCE); } public static class Request extends MasterNodeReadRequest { private final String repository; private final String[] indices; - private final int threadpoolConcurrency; + private final int threadPoolConcurrency; private final int snapshotVerificationConcurrency; private final int indexVerificationConcurrency; private final int indexSnapshotVerificationConcurrency; - private final int maxFailures; - private final boolean permitMissingSnapshotDetails; public Request( String repository, String[] indices, - int threadpoolConcurrency, + int threadPoolConcurrency, int snapshotVerificationConcurrency, int indexVerificationConcurrency, - int indexSnapshotVerificationConcurrency, - int maxFailures, - boolean permitMissingSnapshotDetails + int indexSnapshotVerificationConcurrency ) { this.repository = repository; this.indices = Objects.requireNonNull(indices, "indices"); - this.threadpoolConcurrency = requireMin("threadpoolConcurrency", 0, threadpoolConcurrency); - this.snapshotVerificationConcurrency = requireMin("snapshotVerificationConcurrency", 1, snapshotVerificationConcurrency); - this.indexVerificationConcurrency = requireMin("indexVerificationConcurrency", 1, indexVerificationConcurrency); - this.indexSnapshotVerificationConcurrency = requireMin( + this.threadPoolConcurrency = requireNonNegative("threadPoolConcurrency", threadPoolConcurrency); + this.snapshotVerificationConcurrency = requireNonNegative("snapshotVerificationConcurrency", snapshotVerificationConcurrency); + this.indexVerificationConcurrency = requireNonNegative("indexVerificationConcurrency", indexVerificationConcurrency); + this.indexSnapshotVerificationConcurrency = requireNonNegative( "indexSnapshotVerificationConcurrency", - 1, indexSnapshotVerificationConcurrency ); - this.maxFailures = requireMin("maxFailure", 1, maxFailures); - this.permitMissingSnapshotDetails = permitMissingSnapshotDetails; } - private static int requireMin(String name, int min, int value) { - if (value < min) { - throw new IllegalArgumentException("argument [" + name + "] must be at least [" + min + "]"); + private static int requireNonNegative(String name, int value) { + if (value < 0) { + throw new IllegalArgumentException("argument [" + name + "] must be at least [0]"); } return value; } @@ -99,12 +86,10 @@ public Request(StreamInput in) throws IOException { super(in); this.repository = in.readString(); this.indices = in.readStringArray(); - this.threadpoolConcurrency = in.readVInt(); + this.threadPoolConcurrency = in.readVInt(); this.snapshotVerificationConcurrency = in.readVInt(); this.indexVerificationConcurrency = in.readVInt(); this.indexSnapshotVerificationConcurrency = in.readVInt(); - this.maxFailures = in.readVInt(); - this.permitMissingSnapshotDetails = in.readBoolean(); } @Override @@ -112,12 +97,10 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeString(repository); out.writeStringArray(indices); - out.writeVInt(threadpoolConcurrency); + out.writeVInt(threadPoolConcurrency); out.writeVInt(snapshotVerificationConcurrency); out.writeVInt(indexVerificationConcurrency); out.writeVInt(indexSnapshotVerificationConcurrency); - out.writeVInt(maxFailures); - out.writeBoolean(permitMissingSnapshotDetails); } @Override @@ -127,7 +110,7 @@ public ActionRequestValidationException validate() { @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { - return new CancellableTask(id, type, action, getDescription(), parentTaskId, headers); + return new VerifyRepositoryIntegrityAction.Task(id, type, action, getDescription(), parentTaskId, headers); } public String getRepository() { @@ -138,8 +121,8 @@ public String[] getIndices() { return indices; } - public int getThreadpoolConcurrency() { - return threadpoolConcurrency; + public int getThreadPoolConcurrency() { + return threadPoolConcurrency; } public int getSnapshotVerificationConcurrency() { @@ -154,82 +137,126 @@ public int getIndexSnapshotVerificationConcurrency() { return indexSnapshotVerificationConcurrency; } - public int getMaxFailures() { - return maxFailures; - } - - public boolean permitMissingSnapshotDetails() { - return permitMissingSnapshotDetails; - } - - public Request withDefaultThreadpoolConcurrency(Settings settings) { - if (threadpoolConcurrency == 0) { - final var request = new Request( - repository, - indices, - Math.max(1, EsExecutors.allocatedProcessors(settings) / 2), - snapshotVerificationConcurrency, - indexVerificationConcurrency, - indexSnapshotVerificationConcurrency, - maxFailures, - permitMissingSnapshotDetails - ); - request.masterNodeTimeout(masterNodeTimeout()); - return request; - } else { + public Request withDefaultThreadpoolConcurrency(ThreadPool.Info threadPoolInfo) { + if (threadPoolConcurrency > 0 + && snapshotVerificationConcurrency > 0 + && indexVerificationConcurrency > 0 + && indexSnapshotVerificationConcurrency > 0) { return this; } - } - } - public static class Response extends ActionResponse implements ChunkedToXContent { - - private final List exceptions; - - public Response(List exceptions) { - this.exceptions = exceptions; + final var maxThreads = Math.max(1, threadPoolInfo.getMax()); + final var halfMaxThreads = Math.max(1, maxThreads / 2); + final var request = new Request( + repository, + indices, + threadPoolConcurrency > 0 ? threadPoolConcurrency : halfMaxThreads, + snapshotVerificationConcurrency > 0 ? snapshotVerificationConcurrency : halfMaxThreads, + indexVerificationConcurrency > 0 ? indexVerificationConcurrency : maxThreads, + indexSnapshotVerificationConcurrency > 0 ? indexSnapshotVerificationConcurrency : 1 + ); + request.masterNodeTimeout(masterNodeTimeout()); + return request; } + } - public Response(StreamInput in) throws IOException { - super(in); - this.exceptions = in.readList(RepositoryVerificationException::new); + public record Status( + String repositoryName, + long repositoryGeneration, + String repositoryUUID, + long snapshotCount, + long snapshotsVerified, + long indexCount, + long indicesVerified, + long indexSnapshotCount, + long indexSnapshotsVerified, + long anomalyCount + ) implements org.elasticsearch.tasks.Task.Status { + + public static String NAME = "verify_repository_status"; + + public Status(StreamInput in) throws IOException { + this( + in.readString(), + in.readVLong(), + in.readString(), + in.readVLong(), + in.readVLong(), + in.readVLong(), + in.readVLong(), + in.readVLong(), + in.readVLong(), + in.readVLong() + ); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeList(exceptions); + out.writeString(repositoryName); + out.writeVLong(repositoryGeneration); + out.writeString(repositoryUUID); + out.writeVLong(snapshotCount); + out.writeVLong(snapshotsVerified); + out.writeVLong(indexCount); + out.writeVLong(indicesVerified); + out.writeVLong(indexSnapshotCount); + out.writeVLong(indexSnapshotsVerified); + out.writeVLong(anomalyCount); } @Override - public Iterator toXContentChunked(ToXContent.Params outerParams) { - return Iterators.concat( - Iterators.single((builder, params) -> builder.startObject().startArray("errors")), - exceptions.stream().map(e -> (builder, params) -> { - builder.startObject(); - ElasticsearchException.generateFailureXContent(builder, params, e, true); - return builder.endObject(); - }).iterator(), - Iterators.single((builder, params) -> builder.endArray().endObject()) - ); + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.startObject("repository"); + builder.field("name", repositoryName); + builder.field("uuid", repositoryUUID); + builder.field("generation", repositoryGeneration); + builder.endObject(); + builder.startObject("snapshots"); + builder.field("verified", snapshotsVerified); + builder.field("total", snapshotCount); + builder.endObject(); + builder.startObject("indices"); + builder.field("verified", indicesVerified); + builder.field("total", indexCount); + builder.endObject(); + builder.startObject("index_snapshots"); + builder.field("verified", indexSnapshotsVerified); + builder.field("total", indexSnapshotCount); + builder.endObject(); + builder.field("anomalies", anomalyCount); + builder.endObject(); + return builder; } @Override - public RestStatus getRestStatus() { - if (exceptions.isEmpty()) { - return RestStatus.OK; - } else { - return RestStatus.INTERNAL_SERVER_ERROR; - } + public String getWriteableName() { + return NAME; + } + } + + public static class Task extends CancellableTask { + + private volatile Supplier statusSupplier; + + public Task(long id, String type, String action, String description, TaskId parentTaskId, Map headers) { + super(id, type, action, description, parentTaskId, headers); + } + + public void setStatusSupplier(Supplier statusSupplier) { + this.statusSupplier = statusSupplier; } - public List getExceptions() { - return exceptions; + @Override + public Status getStatus() { + return Optional.ofNullable(statusSupplier).map(Supplier::get).orElse(null); } } - public static class TransportAction extends TransportMasterNodeReadAction { + public static class TransportAction extends TransportMasterNodeReadAction { private final RepositoriesService repositoriesService; + private final NodeClient client; @Inject public TransportAction( @@ -238,7 +265,8 @@ public TransportAction( RepositoriesService repositoriesService, ThreadPool threadPool, ActionFilters actionFilters, - IndexNameExpressionResolver indexNameExpressionResolver + IndexNameExpressionResolver indexNameExpressionResolver, + NodeClient client ) { super( NAME, @@ -249,10 +277,11 @@ public TransportAction( actionFilters, Request::new, indexNameExpressionResolver, - Response::new, + in -> ActionResponse.Empty.INSTANCE, ThreadPool.Names.SNAPSHOT_META ); this.repositoriesService = repositoriesService; + this.client = client; } @Override @@ -261,14 +290,33 @@ protected ClusterBlockException checkBlock(Request request, ClusterState state) } @Override - protected void masterOperation(Task task, Request request, ClusterState state, ActionListener listener) throws Exception { + protected void masterOperation( + org.elasticsearch.tasks.Task task, + Request request, + ClusterState state, + ActionListener listener + ) throws Exception { // TODO add mechanism to block blob deletions while this is running - final var cancellableTask = (CancellableTask) task; + final var verifyTask = (Task) task; + + final ClusterStateListener noLongerMasterListener = event -> { + if (event.localNodeMaster() == false) { + transportService.getTaskManager().cancel(verifyTask, "no longer master", () -> {}); + } + }; + clusterService.addListener(noLongerMasterListener); + repositoriesService.repository(request.getRepository()) .verifyMetadataIntegrity( - request.withDefaultThreadpoolConcurrency(clusterService.getSettings()), - listener.map(Response::new), - cancellableTask::isCancelled + client, + transportService::newNetworkBytesStream, + request.withDefaultThreadpoolConcurrency(clusterService.threadPool().info(ThreadPool.Names.SNAPSHOT_META)), + ActionListener.runAfter( + listener.map(ignored -> ActionResponse.Empty.INSTANCE), + () -> clusterService.removeListener(noLongerMasterListener) + ), + verifyTask::isCancelled, + verifyTask::setStatusSupplier ); } } diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ThrottledIterator.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThrottledIterator.java new file mode 100644 index 0000000000000..3df6629a272c1 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ThrottledIterator.java @@ -0,0 +1,152 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.common.util.concurrent; + +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.RefCounted; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Strings; +import org.elasticsearch.logging.LogManager; +import org.elasticsearch.logging.Logger; + +import java.util.Iterator; +import java.util.Objects; +import java.util.concurrent.Semaphore; +import java.util.function.BiConsumer; + +public class ThrottledIterator implements Releasable { + + private static final Logger logger = LogManager.getLogger(ThrottledIterator.class); + + /** + * Iterate through the given collection, performing an operation on each item which may fork background tasks, but with a limit on the + * number of such background tasks running concurrently to avoid overwhelming the rest of the system (e.g. starving other work of access + * to an executor). + * + * @param iterator The items to iterate. May be accessed by multiple threads, but accesses are all protected by synchronizing on itself. + * @param itemConsumer The operation to perform on each item. Each operation receives a {@link RefCounted} which can be used to track + * the execution of any background tasks spawned for this item. This operation may run on the thread which + * originally called {@link #run}, if this method has not yet returned. Otherwise it will run on a thread on which a + * background task previously called {@link RefCounted#decRef()} on its ref count. This operation should not throw + * any exceptions. + * @param maxConcurrency The maximum number of ongoing operations at any time. + * @param onItemCompletion Executed when each item is completed, which can be used for instance to report on progress. Must not throw + * exceptions. + * @param onCompletion Executed when all items are completed. + */ + public static void run( + Iterator iterator, + BiConsumer itemConsumer, + int maxConcurrency, + Runnable onItemCompletion, + Runnable onCompletion + ) { + try (var throttledIterator = new ThrottledIterator<>(iterator, itemConsumer, maxConcurrency, onItemCompletion, onCompletion)) { + throttledIterator.run(); + } + } + + private final RefCounted throttleRefs; + private final Iterator iterator; + private final BiConsumer itemConsumer; + private final Semaphore permits; + private final Runnable onItemCompletion; + + private ThrottledIterator( + Iterator iterator, + BiConsumer itemConsumer, + int maxConcurrency, + Runnable onItemCompletion, + Runnable onCompletion + ) { + this.iterator = Objects.requireNonNull(iterator); + this.itemConsumer = Objects.requireNonNull(itemConsumer); + if (maxConcurrency <= 0) { + throw new IllegalArgumentException("maxConcurrency must be positive"); + } + this.permits = new Semaphore(maxConcurrency); + this.onItemCompletion = Objects.requireNonNull(onItemCompletion); + this.throttleRefs = AbstractRefCounted.of(onCompletion); + } + + private void run() { + while (permits.tryAcquire()) { + final T item; + synchronized (iterator) { + if (iterator.hasNext()) { + item = iterator.next(); + } else { + permits.release(); + return; + } + } + try (var itemRefs = new ItemRefCounted()) { + itemConsumer.accept(itemRefs, item); + } catch (Exception e) { + logger.error(Strings.format("exception when processing [%s] with [%s]", item, itemConsumer), e); + assert false : e; + } + } + } + + @Override + public void close() { + throttleRefs.decRef(); + } + + // A RefCounted for a single item, including protection against calling back into run() if it's created and closed within a single + // invocation of run(). + private class ItemRefCounted extends AbstractRefCounted implements Releasable { + private boolean isRecursive = true; + + ItemRefCounted() { + throttleRefs.incRef(); + } + + @Override + protected void closeInternal() { + try { + onItemCompletion.run(); + } catch (Exception e) { + logger.error("exception in onItemCompletion", e); + assert false : e; + } finally { + permits.release(); + try { + // Someone must now pick up the next item. Here we might be called from the run() invocation which started processing + // the just-completed item (via close() -> decRef()) if that item's processing didn't fork or all its forked tasks + // finished first. If so, there's no need to call run() here, we can just return and the next iteration of the run() + // loop will continue the processing; moreover calling run() in this situation could lead to a stack overflow. However + // if we're not within that run() invocation then ... + if (isRecursive() == false) { + // ... we're not within any other run() invocation either, so it's safe (and necessary) to call run() here. + run(); + } + } finally { + throttleRefs.decRef(); + } + } + } + + // Note on blocking: we call both of these synchronized methods exactly once (and must enter close() before calling isRecursive()). + // If close() releases the last ref and calls closeInternal(), and hence isRecursive(), then there's no other threads involved and + // hence no blocking. In contrast if close() doesn't release the last ref then it exits immediately, so the call to isRecursive() + // will proceed without delay in this case too. + + private synchronized boolean isRecursive() { + return isRecursive; + } + + @Override + public synchronized void close() { + decRef(); + isRecursive = false; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java index a93ff4853879a..9fe5eefd45889 100644 --- a/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java +++ b/server/src/main/java/org/elasticsearch/common/xcontent/ChunkedToXContent.java @@ -8,7 +8,6 @@ package org.elasticsearch.common.xcontent; -import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xcontent.ToXContent; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; @@ -43,8 +42,4 @@ static ToXContentObject wrapAsXContentObject(ChunkedToXContent chunkedToXContent return builder; }; } - - default RestStatus getRestStatus() { - return RestStatus.OK; - } } diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 58b089940b215..b35f2c5f09536 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -532,7 +532,8 @@ protected Node( searchModule.getNamedWriteables().stream(), pluginsService.flatMap(Plugin::getNamedWriteables), ClusterModule.getNamedWriteables().stream(), - SystemIndexMigrationExecutor.getNamedWriteables().stream() + SystemIndexMigrationExecutor.getNamedWriteables().stream(), + RepositoriesService.getNamedWriteables().stream() ).flatMap(Function.identity()).toList(); final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables); NamedXContentRegistry xContentRegistry = new NamedXContentRegistry( diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index 2f28adcebc98e..1ec81bc6d7923 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -15,6 +15,7 @@ import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.cluster.repositories.delete.DeleteRepositoryRequest; +import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; @@ -35,6 +36,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; @@ -45,6 +47,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.repositories.blobstore.MeteredBlobStoreRepository; import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -130,6 +133,16 @@ public RepositoriesService( this.preRestoreChecks = preRestoreChecks; } + public static List getNamedWriteables() { + return List.of( + new NamedWriteableRegistry.Entry( + Task.Status.class, + VerifyRepositoryIntegrityAction.Status.NAME, + VerifyRepositoryIntegrityAction.Status::new + ) + ); + } + /** * Registers new repository in the cluster *

diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index 4f9c00b959a57..c50e3a61dab91 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -10,6 +10,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; +import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -17,6 +18,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.component.LifecycleComponent; +import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Nullable; import org.elasticsearch.index.shard.ShardId; @@ -35,6 +37,7 @@ import java.util.function.BooleanSupplier; import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.Supplier; /** * An interface for interacting with a repository in snapshot and restore. @@ -313,9 +316,12 @@ static boolean assertSnapshotMetaThread() { } default void verifyMetadataIntegrity( + Client client, + Supplier bytesStreamOutputSupplier, VerifyRepositoryIntegrityAction.Request request, - ActionListener> listener, - BooleanSupplier isCancelledSupplier + ActionListener listener, + BooleanSupplier isCancelledSupplier, + Consumer> statusSupplierConsumer ) { listener.onFailure(new UnsupportedOperationException("this repository type does not support metadata integrity verification")); } 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 3ed5e451d72f9..489ed34cd938c 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -33,6 +33,7 @@ import org.elasticsearch.action.support.ListenableActionFuture; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.ThreadedActionListener; +import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RepositoryCleanupInProgress; @@ -59,6 +60,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.compress.NotXContentException; import org.elasticsearch.common.io.Streams; +import org.elasticsearch.common.io.stream.RecyclerBytesStreamOutput; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.metrics.CounterMetric; @@ -133,7 +135,6 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -3545,38 +3546,14 @@ private static void failStoreIfCorrupted(Store store, Exception e) { @Override public void verifyMetadataIntegrity( + Client client, + Supplier bytesStreamOutputSupplier, VerifyRepositoryIntegrityAction.Request request, - ActionListener> listener, - BooleanSupplier isCancelledSupplier + ActionListener listener, + BooleanSupplier isCancelledSupplier, + Consumer> statusSupplierConsumer ) { - getRepositoryData(listener.delegateFailure((l, repositoryData) -> { - logger.info( - "[{}] verifying metadata integrity for index generation [{}]: repo UUID [{}], cluster UUID [{}]", - metadata.name(), - repositoryData.getGenId(), - repositoryData.getUuid(), - repositoryData.getClusterUUID() - ); - - threadPool.executor(ThreadPool.Names.SNAPSHOT_META) - .execute(ActionRunnable.supply(l.delegateFailure((l2, loadedRepositoryData) -> { - // really just checking that the repo data can be loaded, but may as well check a little consistency too - if (loadedRepositoryData.getGenId() != repositoryData.getGenId()) { - throw new IllegalStateException( - String.format( - Locale.ROOT, - "[%s] has repository data generation [%d], expected [%d]", - metadata.name(), - loadedRepositoryData.getGenId(), - repositoryData.getGenId() - ) - ); - } - try (var metadataVerifier = new MetadataVerifier(this, request, repositoryData, isCancelledSupplier, l2)) { - metadataVerifier.run(); - } - }), () -> readOnly ? repositoryData : getRepositoryData(repositoryData.getGenId()))); - })); + MetadataVerifier.run(this, client, request, isCancelledSupplier, statusSupplierConsumer, listener); } public boolean supportURLRepo() { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java index 94e5a5148b6ed..65ec5e3940234 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/MetadataVerifier.java @@ -10,92 +10,180 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRunnable; import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; +import org.elasticsearch.action.bulk.BulkRequest; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.index.IndexAction; +import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.support.ListenableActionFuture; +import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.common.CheckedSupplier; import org.elasticsearch.common.blobstore.support.BlobMetadata; -import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.time.DateFormatter; +import org.elasticsearch.common.time.FormatNames; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.ThrottledIterator; import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.CheckedConsumer; -import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Strings; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshots; import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.repositories.RepositoryVerificationException; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.ToXContent; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.XContentFactory; +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; import java.util.Iterator; -import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Queue; import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; import java.util.function.BooleanSupplier; +import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import static org.elasticsearch.common.util.concurrent.ConcurrentCollections.newConcurrentMap; -import static org.elasticsearch.core.Strings.format; class MetadataVerifier implements Releasable { private static final Logger logger = LogManager.getLogger(MetadataVerifier.class); + enum Anomaly { + FAILED_TO_LOAD_GLOBAL_METADATA, + FAILED_TO_LOAD_SHARD_SNAPSHOT, + FAILED_TO_LOAD_INDEX_METADATA, + FAILED_TO_LOAD_SHARD_GENERATION, + UNDEFINED_SHARD_GENERATION, + UNEXPECTED_EXCEPTION, + FILE_IN_SHARD_GENERATION_NOT_SNAPSHOT, + SNAPSHOT_SHARD_GENERATION_MISMATCH, + FILE_IN_SNAPSHOT_NOT_SHARD_GENERATION, + MISMATCHED_VIRTUAL_BLOB_LENGTH, + MISSING_BLOB, + MISMATCHED_BLOB_LENGTH, + UNKNOWN_SNAPSHOT_FOR_INDEX, + } + + public static void run( + BlobStoreRepository blobStoreRepository, + Client client, + VerifyRepositoryIntegrityAction.Request verifyRequest, + BooleanSupplier isCancelledSupplier, + Consumer> statusSupplierConsumer, + ActionListener listener + ) { + logger.info("[{}] verifying metadata integrity", blobStoreRepository.getMetadata().name()); + blobStoreRepository.getRepositoryData(listener.delegateFailure((l, repositoryData) -> { + try ( + var metadataVerifier = new MetadataVerifier( + blobStoreRepository, + client, + verifyRequest, + repositoryData, + isCancelledSupplier, + new ActionListener<>() { + @Override + public void onResponse(Long anomalyCount) { + logger.info( + "[{}] completed verifying metadata integrity for index generation [{}]: " + + "repo UUID [{}], cluster UUID [{}], anomalies [{}]", + blobStoreRepository.getMetadata().name(), + repositoryData.getGenId(), + repositoryData.getUuid(), + repositoryData.getClusterUUID(), + anomalyCount + ); + l.onResponse(null); + } + + @Override + public void onFailure(Exception e) { + logger.info( + () -> Strings.format( + "[%s] failed verifying metadata integrity for index generation [%d]: repo UUID [%s], cluster UUID [%s]", + blobStoreRepository.getMetadata().name(), + repositoryData.getGenId(), + repositoryData.getUuid(), + repositoryData.getClusterUUID() + ) + ); + l.onFailure(e); + } + } + ) + ) { + statusSupplierConsumer.accept(metadataVerifier::getStatus); + metadataVerifier.start(); + } + })); + } + private final BlobStoreRepository blobStoreRepository; - private final ActionListener> finalListener; + private final Client client; + private final ActionListener finalListener; private final RefCounted finalRefs = AbstractRefCounted.of(this::onCompletion); private final String repositoryName; private final VerifyRepositoryIntegrityAction.Request verifyRequest; private final RepositoryData repositoryData; private final BooleanSupplier isCancelledSupplier; - private final Queue failures = new ConcurrentLinkedQueue<>(); - private final AtomicLong failureCount = new AtomicLong(); - private final Map> snapshotsByIndex; + private final AtomicLong anomalyCount = new AtomicLong(); + private final Map snapshotDescriptionsById = ConcurrentCollections.newConcurrentMap(); private final Semaphore threadPoolPermits; - private final Queue executorQueue = new ConcurrentLinkedQueue<>(); - private final ProgressLogger snapshotProgressLogger; - private final ProgressLogger indexProgressLogger; - private final ProgressLogger indexSnapshotProgressLogger; + private final Queue executorQueue = ConcurrentCollections.newQueue(); private final Set requestedIndices; + private final long snapshotCount; + private final AtomicLong snapshotProgress = new AtomicLong(); + private final long indexCount; + private final AtomicLong indexProgress = new AtomicLong(); + private final long indexSnapshotCount; + private final AtomicLong indexSnapshotProgress = new AtomicLong(); + MetadataVerifier( BlobStoreRepository blobStoreRepository, + Client client, VerifyRepositoryIntegrityAction.Request verifyRequest, RepositoryData repositoryData, BooleanSupplier isCancelledSupplier, - ActionListener> finalListener + ActionListener finalListener ) { this.blobStoreRepository = blobStoreRepository; this.repositoryName = blobStoreRepository.metadata.name(); + this.client = client; this.verifyRequest = verifyRequest; this.repositoryData = repositoryData; this.isCancelledSupplier = isCancelledSupplier; this.finalListener = finalListener; - this.snapshotsByIndex = this.repositoryData.getIndices() - .values() - .stream() - .collect(Collectors.toMap(IndexId::getName, indexId -> Set.copyOf(this.repositoryData.getSnapshots(indexId)))); - - this.threadPoolPermits = new Semaphore(Math.max(1, verifyRequest.getThreadpoolConcurrency())); - this.snapshotProgressLogger = new ProgressLogger("snapshots", repositoryData.getSnapshotIds().size(), 100); - this.indexProgressLogger = new ProgressLogger("indices", repositoryData.getIndices().size(), 20); - this.indexSnapshotProgressLogger = new ProgressLogger("index snapshots", repositoryData.getIndexSnapshotCount(), 1000); - + this.threadPoolPermits = new Semaphore(Math.max(1, verifyRequest.getThreadPoolConcurrency())); this.requestedIndices = Set.of(verifyRequest.getIndices()); + + this.snapshotCount = repositoryData.getSnapshotIds().size(); + this.indexCount = repositoryData.getIndices().size(); + this.indexSnapshotCount = repositoryData.getIndexSnapshotCount(); } @Override @@ -103,31 +191,24 @@ public void close() { finalRefs.decRef(); } - private void addFailure(String format, Object... args) { - final var failureNumber = failureCount.incrementAndGet(); - final var failure = format(format, args); - logger.debug("[{}] found metadata verification failure [{}]: {}", repositoryName, failureNumber, failure); - if (failureNumber <= verifyRequest.getMaxFailures()) { - failures.add(new RepositoryVerificationException(repositoryName, failure)); - } - } + private static final String RESULTS_INDEX = "metadata_verification_results"; - private void addFailure(Exception exception) { - if (isCancelledSupplier.getAsBoolean() && exception instanceof TaskCancelledException) { - return; - } - final var failureNumber = failureCount.incrementAndGet(); - logger.debug(() -> format("[%s] exception [%d] during metadata verification", repositoryName, failureNumber), exception); - if (failureNumber <= verifyRequest.getMaxFailures()) { - failures.add( - exception instanceof RepositoryVerificationException rve - ? rve - : new RepositoryVerificationException(repositoryName, "exception during metadata verification", exception) - ); - } + VerifyRepositoryIntegrityAction.Status getStatus() { + return new VerifyRepositoryIntegrityAction.Status( + repositoryName, + repositoryData.getGenId(), + repositoryData.getUuid(), + snapshotCount, + snapshotProgress.get(), + indexCount, + indexProgress.get(), + indexSnapshotCount, + indexSnapshotProgress.get(), + anomalyCount.get() + ); } - public void run() { + private void start() { logger.info( "[{}] verifying metadata integrity for index generation [{}]: " + "repo UUID [{}], cluster UUID [{}], snapshots [{}], indices [{}], index snapshots [{}]", @@ -135,88 +216,60 @@ public void run() { repositoryData.getGenId(), repositoryData.getUuid(), repositoryData.getClusterUUID(), - snapshotProgressLogger.getExpectedMax(), - indexProgressLogger.getExpectedMax(), - indexSnapshotProgressLogger.getExpectedMax() + snapshotCount, + indexCount, + indexSnapshotCount ); - if (requestedIndices.isEmpty()) { - verifySnapshots(); - } else { - verifyIndices(); - } + // TODO define (strict) mappings for index + client.admin().indices().prepareCreate(RESULTS_INDEX).execute(makeListener(finalRefs, createIndexResponse -> verifySnapshots())); } private void verifySnapshots() { runThrottled( - makeVoidListener(finalRefs, this::verifyIndices), repositoryData.getSnapshotIds().iterator(), this::verifySnapshot, verifyRequest.getSnapshotVerificationConcurrency(), - snapshotProgressLogger + snapshotProgress, + wrapRunnable(finalRefs, this::verifyIndices) ); } private void verifySnapshot(RefCounted snapshotRefs, SnapshotId snapshotId) { - if (verifyRequest.permitMissingSnapshotDetails() == false && repositoryData.hasMissingDetails(snapshotId)) { - addFailure("snapshot [%s] has missing snapshot details", snapshotId); - } - if (isCancelledSupplier.getAsBoolean()) { // getSnapshotInfo does its own forking so we must check for cancellation here return; } blobStoreRepository.getSnapshotInfo(snapshotId, makeListener(snapshotRefs, snapshotInfo -> { - if (snapshotInfo.snapshotId().equals(snapshotId) == false) { - addFailure("snapshot [%s] has unexpected ID in info blob: [%s]", snapshotId, snapshotInfo.snapshotId()); - } - for (final var index : snapshotInfo.indices()) { - if (snapshotsByIndex.get(index).contains(snapshotId) == false) { - addFailure("snapshot [%s] contains unexpected index [%s]", snapshotId, index); - } - } + final var snapshotDescription = new SnapshotDescription(snapshotId, snapshotInfo.startTime(), snapshotInfo.endTime()); + snapshotDescriptionsById.put(snapshotId.getUUID(), snapshotDescription); + forkSupply(snapshotRefs, () -> getSnapshotGlobalMetadata(snapshotRefs, snapshotDescription), metadata -> { + // no checks here, loading it is enough + }); })); - - forkSupply(snapshotRefs, () -> getSnapshotGlobalMetadata(snapshotId), metadata -> { - if (metadata != null && metadata.indices().isEmpty() == false) { - addFailure("snapshot [%s] contains unexpected index metadata within global metadata", snapshotId); - } - }); } - private Metadata getSnapshotGlobalMetadata(SnapshotId snapshotId) { + private Metadata getSnapshotGlobalMetadata(RefCounted snapshotRefs, SnapshotDescription snapshotDescription) { try { - return blobStoreRepository.getSnapshotGlobalMetadata(snapshotId); + return blobStoreRepository.getSnapshotGlobalMetadata(snapshotDescription.snapshotId()); } catch (Exception e) { - addFailure( - new RepositoryVerificationException( - repositoryName, - format("failed to get snapshot global metadata for [%s]", snapshotId), - e - ) - ); + addAnomaly(Anomaly.FAILED_TO_LOAD_GLOBAL_METADATA, snapshotRefs, (builder, params) -> { + snapshotDescription.writeXContent(builder); + ElasticsearchException.generateFailureXContent(builder, params, e, true); + return builder; + }); return null; } } private void verifyIndices() { - final var indicesMap = repositoryData.getIndices(); - - for (final var indicesEntry : indicesMap.entrySet()) { - final var name = indicesEntry.getKey(); - final var indexId = indicesEntry.getValue(); - if (name.equals(indexId.getName()) == false) { - addFailure("index name [%s] has mismatched name in %s", name, indexId); - } - } - runThrottled( - makeVoidListener(finalRefs, () -> {}), - indicesMap.values().iterator(), + repositoryData.getIndices().values().iterator(), (refCounted, indexId) -> new IndexVerifier(refCounted, indexId).run(), verifyRequest.getIndexVerificationConcurrency(), - indexProgressLogger + indexProgress, + wrapRunnable(finalRefs, () -> {}) ); } @@ -228,16 +281,14 @@ private record ShardContainerContents( private class IndexVerifier { private final RefCounted indexRefs; private final IndexId indexId; - private final Set expectedSnapshots; private final Map> shardContainerContentsListener = newConcurrentMap(); - private final Map> shardCountListenersByBlobId = newConcurrentMap(); + private final Map> indexDescriptionListenersByBlobId = newConcurrentMap(); private final AtomicInteger totalSnapshotCounter = new AtomicInteger(); private final AtomicInteger restorableSnapshotCounter = new AtomicInteger(); IndexVerifier(RefCounted indexRefs, IndexId indexId) { this.indexRefs = indexRefs; this.indexId = indexId; - this.expectedSnapshots = snapshotsByIndex.get(indexId.getName()); } void run() { @@ -246,67 +297,74 @@ void run() { } runThrottled( - makeVoidListener(indexRefs, () -> logRestorability(totalSnapshotCounter.get(), restorableSnapshotCounter.get())), repositoryData.getSnapshots(indexId).iterator(), this::verifyIndexSnapshot, verifyRequest.getIndexSnapshotVerificationConcurrency(), - indexSnapshotProgressLogger + indexSnapshotProgress, + wrapRunnable(indexRefs, () -> recordRestorability(totalSnapshotCounter.get(), restorableSnapshotCounter.get())) ); } - private void logRestorability(int totalSnapshotCount, int restorableSnapshotCount) { + private void recordRestorability(int totalSnapshotCount, int restorableSnapshotCount) { if (isCancelledSupplier.getAsBoolean() == false) { - if (totalSnapshotCount == restorableSnapshotCount) { - logger.debug("[{}] index {} is fully restorable from [{}] snapshots", repositoryName, indexId, totalSnapshotCount); - } else { - logger.debug( - "[{}] index {} is not fully restorable: of [{}] snapshots, [{}] are restorable and [{}] are not", - repositoryName, - indexId, - totalSnapshotCount, - restorableSnapshotCount, - totalSnapshotCount - restorableSnapshotCount + addResult(indexRefs, (builder, params) -> { + writeIndexId(indexId, builder); + builder.field( + "restorability", + totalSnapshotCount == restorableSnapshotCount ? "full" : 0 < restorableSnapshotCount ? "partial" : "none" ); - } + builder.field("snapshots", totalSnapshotCount); + builder.field("restorable_snapshots", restorableSnapshotCount); + builder.field("unrestorable_snapshots", totalSnapshotCount - restorableSnapshotCount); + return builder; + }); } } private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapshotId) { totalSnapshotCounter.incrementAndGet(); - if (expectedSnapshots.contains(snapshotId) == false) { - addFailure("index %s has mismatched snapshot [%s]", indexId, snapshotId); + final var snapshotDescription = snapshotDescriptionsById.get(snapshotId.getUUID()); + if (snapshotDescription == null) { + addAnomaly(Anomaly.UNKNOWN_SNAPSHOT_FOR_INDEX, indexSnapshotRefs, (builder, params) -> { + writeIndexId(indexId, builder); + new SnapshotDescription(snapshotId, 0, 0).writeXContent(builder); + return builder; + }); + return; } final var indexMetaBlobId = repositoryData.indexMetaDataGenerations().indexMetaBlobId(snapshotId, indexId); - shardCountListenersByBlobId.computeIfAbsent(indexMetaBlobId, ignored -> { - final var shardCountFuture = new ListenableActionFuture(); + indexDescriptionListenersByBlobId.computeIfAbsent(indexMetaBlobId, ignored -> { + final var indexDescriptionFuture = new ListenableActionFuture(); forkSupply(() -> { final var shardCount = getNumberOfShards(indexMetaBlobId, snapshotId); + final var indexDescription = new IndexDescription(indexId, indexMetaBlobId, shardCount); for (int i = 0; i < shardCount; i++) { shardContainerContentsListener.computeIfAbsent(i, shardId -> { final var shardContainerContentsFuture = new ListenableActionFuture(); forkSupply( () -> new ShardContainerContents( blobStoreRepository.shardContainer(indexId, shardId).listBlobs(), - getBlobStoreIndexShardSnapshots(shardId) + getBlobStoreIndexShardSnapshots(indexDescription, shardId) ), shardContainerContentsFuture ); return shardContainerContentsFuture; }); } - return shardCount; - }, shardCountFuture); - return shardCountFuture; - }).addListener(makeListener(indexSnapshotRefs, shardCount -> { + return indexDescription; + }, indexDescriptionFuture); + return indexDescriptionFuture; + }).addListener(makeListener(indexSnapshotRefs, indexDescription -> { final var restorableShardCount = new AtomicInteger(); - try (var shardSnapshotsRefs = wrap(makeVoidListener(indexSnapshotRefs, () -> { - if (shardCount > 0 && shardCount == restorableShardCount.get()) { + final var shardSnapshotsRefs = AbstractRefCounted.of(wrapRunnable(indexSnapshotRefs, () -> { + if (indexDescription.shardCount() > 0 && indexDescription.shardCount() == restorableShardCount.get()) { restorableSnapshotCounter.incrementAndGet(); } - }))) { - for (int i = 0; i < shardCount; i++) { + })); + try { + for (int i = 0; i < indexDescription.shardCount(); i++) { final var shardId = i; shardContainerContentsListener.get(i) .addListener( @@ -314,9 +372,16 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh shardSnapshotsRefs, shardContainerContents -> forkSupply( shardSnapshotsRefs, - () -> getBlobStoreIndexShardSnapshot(snapshotId, shardId), + () -> getBlobStoreIndexShardSnapshot( + shardSnapshotsRefs, + snapshotDescription, + indexDescription, + shardId + ), shardSnapshot -> verifyShardSnapshot( - snapshotId, + shardSnapshotsRefs, + indexDescription, + snapshotDescription, shardId, shardContainerContents, shardSnapshot, @@ -326,69 +391,75 @@ private void verifyIndexSnapshot(RefCounted indexSnapshotRefs, SnapshotId snapsh ) ); } + } finally { + shardSnapshotsRefs.decRef(); } })); } - private BlobStoreIndexShardSnapshot getBlobStoreIndexShardSnapshot(SnapshotId snapshotId, int shardId) { + private BlobStoreIndexShardSnapshot getBlobStoreIndexShardSnapshot( + RefCounted shardSnapshotRefs, + SnapshotDescription snapshotDescription, + IndexDescription indexDescription, + int shardId + ) { try { - return blobStoreRepository.loadShardSnapshot(blobStoreRepository.shardContainer(indexId, shardId), snapshotId); - } catch (Exception e) { - addFailure( - new RepositoryVerificationException( - repositoryName, - format("failed to load shard %s[%d] snapshot for [%s]", indexId, shardId, snapshotId), - e - ) + return blobStoreRepository.loadShardSnapshot( + blobStoreRepository.shardContainer(indexId, shardId), + snapshotDescription.snapshotId() ); + } catch (Exception e) { + addAnomaly(Anomaly.FAILED_TO_LOAD_SHARD_SNAPSHOT, shardSnapshotRefs, (builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + ElasticsearchException.generateFailureXContent(builder, params, e, true); + return builder; + }); return null; } } - private List getSnapshotsWithIndexMetadataBlob(String indexMetaBlobId) { - final var indexMetaDataGenerations = repositoryData.indexMetaDataGenerations(); - return repositoryData.getSnapshotIds() - .stream() - .filter(s -> indexMetaBlobId.equals(indexMetaDataGenerations.indexMetaBlobId(s, indexId))) - .toList(); - } - private int getNumberOfShards(String indexMetaBlobId, SnapshotId snapshotId) { try { return blobStoreRepository.getSnapshotIndexMetaData(repositoryData, snapshotId, indexId).getNumberOfShards(); } catch (Exception e) { - addFailure( - new RepositoryVerificationException( - repositoryName, - format( - "failed to load index %s metadata from blob [%s] for %s", - indexId, - indexMetaBlobId, - getSnapshotsWithIndexMetadataBlob(indexMetaBlobId) - ), - e - ) - ); + addAnomaly(Anomaly.FAILED_TO_LOAD_INDEX_METADATA, indexRefs, (builder, params) -> { + writeIndexId(indexId, builder); + builder.field("metadata_blob", indexMetaBlobId); + ElasticsearchException.generateFailureXContent(builder, params, e, true); + return builder; + }); return 0; } } - private BlobStoreIndexShardSnapshots getBlobStoreIndexShardSnapshots(int shardId) { + private BlobStoreIndexShardSnapshots getBlobStoreIndexShardSnapshots(IndexDescription indexDescription, int shardId) { final var shardGen = repositoryData.shardGenerations().getShardGen(indexId, shardId); if (shardGen == null) { - addFailure("unknown shard generation for %s[%d]", indexId, shardId); + addAnomaly(Anomaly.UNDEFINED_SHARD_GENERATION, indexRefs, (builder, params) -> { + indexDescription.writeXContent(builder); + return builder.field("shard", shardId); + }); return null; } try { return blobStoreRepository.getBlobStoreIndexShardSnapshots(indexId, shardId, shardGen); } catch (Exception e) { - addFailure(e); + addAnomaly(Anomaly.FAILED_TO_LOAD_SHARD_GENERATION, indexRefs, (builder, params) -> { + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + ElasticsearchException.generateFailureXContent(builder, params, e, true); + return builder; + }); return null; } } private void verifyShardSnapshot( - SnapshotId snapshotId, + RefCounted shardSnapshotRefs, + IndexDescription indexDescription, + SnapshotDescription snapshotDescription, int shardId, ShardContainerContents shardContainerContents, BlobStoreIndexShardSnapshot shardSnapshot, @@ -398,19 +469,16 @@ private void verifyShardSnapshot( return; } - if (shardSnapshot.snapshot().equals(snapshotId.getName()) == false) { - addFailure( - "snapshot [%s] for shard %s[%d] has mismatched name [%s]", - snapshotId, - indexId, - shardId, - shardSnapshot.snapshot() - ); - } - var restorable = true; for (final var fileInfo : shardSnapshot.indexFiles()) { - restorable &= verifyFileInfo(snapshotId.toString(), shardId, shardContainerContents.blobsByName(), fileInfo); + restorable &= verifyFileInfo( + shardSnapshotRefs, + snapshotDescription, + indexDescription, + shardId, + shardContainerContents.blobsByName(), + fileInfo + ); } if (restorable) { runIfRestorable.run(); @@ -420,21 +488,36 @@ private void verifyShardSnapshot( if (blobStoreIndexShardSnapshots != null) { boolean foundSnapshot = false; for (SnapshotFiles summary : blobStoreIndexShardSnapshots.snapshots()) { - if (summary.snapshot().equals(snapshotId.getName())) { + if (summary.snapshot().equals(snapshotDescription.snapshotId().getName())) { foundSnapshot = true; - verifyConsistentShardFiles(snapshotId, shardId, shardSnapshot, summary); + verifyConsistentShardFiles( + shardSnapshotRefs, + snapshotDescription, + indexDescription, + shardId, + shardSnapshot, + summary + ); break; } } if (foundSnapshot == false) { - addFailure("snapshot [%s] for shard %s[%d] has no entry in the shard-level summary", snapshotId, indexId, shardId); + addResult(shardSnapshotRefs, ((builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("failure", "missing in shard-level summary"); + return builder; + })); } } } private void verifyConsistentShardFiles( - SnapshotId snapshotId, + RefCounted shardSnapshotRefs, + SnapshotDescription snapshotDescription, + IndexDescription indexDescription, int shardId, BlobStoreIndexShardSnapshot shardSnapshot, SnapshotFiles summary @@ -446,21 +529,21 @@ private void verifyConsistentShardFiles( for (final var summaryFile : summary.indexFiles()) { final var snapshotFile = snapshotFiles.get(summaryFile.physicalName()); if (snapshotFile == null) { - addFailure( - "snapshot [%s] for shard %s[%d] has no entry for file [%s] found in summary", - snapshotId, - indexId, - shardId, - summaryFile.physicalName() - ); + addAnomaly(Anomaly.FILE_IN_SHARD_GENERATION_NOT_SNAPSHOT, shardSnapshotRefs, (builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("file_name", summaryFile.physicalName()); + return builder; + }); } else if (summaryFile.isSame(snapshotFile) == false) { - addFailure( - "snapshot [%s] for shard %s[%d] has a mismatched entry for file [%s]", - snapshotId, - indexId, - shardId, - summaryFile.physicalName() - ); + addAnomaly(Anomaly.SNAPSHOT_SHARD_GENERATION_MISMATCH, shardSnapshotRefs, (builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("file_name", summaryFile.physicalName()); + return builder; + }); } } @@ -469,27 +552,21 @@ private void verifyConsistentShardFiles( .collect(Collectors.toMap(BlobStoreIndexShardSnapshot.FileInfo::physicalName, Function.identity())); for (final var snapshotFile : shardSnapshot.indexFiles()) { if (summaryFiles.get(snapshotFile.physicalName()) == null) { - addFailure( - "snapshot [%s] for shard %s[%d] has no entry in the shard-level summary for file [%s]", - snapshotId, - indexId, - shardId, - snapshotFile.physicalName() - ); + addAnomaly(Anomaly.FILE_IN_SNAPSHOT_NOT_SHARD_GENERATION, shardSnapshotRefs, (builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("file_name", snapshotFile.physicalName()); + return builder; + }); } } } - private static String formatExact(ByteSizeValue byteSizeValue) { - if (byteSizeValue.getBytes() >= ByteSizeUnit.KB.toBytes(1)) { - return format("%s/%dB", byteSizeValue.toString(), byteSizeValue.getBytes()); - } else { - return byteSizeValue.toString(); - } - } - private boolean verifyFileInfo( - String snapshot, + RefCounted shardSnapshotRefs, + SnapshotDescription snapshotDescription, + IndexDescription indexDescription, int shardId, Map shardBlobs, BlobStoreIndexShardSnapshot.FileInfo fileInfo @@ -498,53 +575,51 @@ private boolean verifyFileInfo( if (fileInfo.metadata().hashEqualsContents()) { final var actualLength = ByteSizeValue.ofBytes(fileInfo.metadata().hash().length); if (fileLength.getBytes() != actualLength.getBytes()) { - addFailure( - "snapshot [%s] for shard %s[%d] has virtual blob [%s] for [%s] with length [%s] instead of [%s]", - snapshot, - indexId, - shardId, - fileInfo.name(), - fileInfo.physicalName(), - formatExact(actualLength), - formatExact(fileLength) - ); + addAnomaly(Anomaly.MISMATCHED_VIRTUAL_BLOB_LENGTH, shardSnapshotRefs, ((builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("file_name", fileInfo.physicalName()); + builder.humanReadableField("actual_length_in_bytes", "actual_length", actualLength); + builder.humanReadableField("expected_length_in_bytes", "expected_length", fileLength); + return builder; + })); return false; } } else { for (int part = 0; part < fileInfo.numberOfParts(); part++) { + final var finalPart = part; final var blobName = fileInfo.partName(part); final var blobInfo = shardBlobs.get(blobName); final var partLength = ByteSizeValue.ofBytes(fileInfo.partBytes(part)); if (blobInfo == null) { - addFailure( - "snapshot [%s] for shard %s[%d] has missing blob [%s] for [%s] part [%d/%d]; " - + "file length [%s], part length [%s]", - snapshot, - indexId, - shardId, - blobName, - fileInfo.physicalName(), - part, - fileInfo.numberOfParts(), - formatExact(fileLength), - formatExact(partLength) - ); + addAnomaly(Anomaly.MISSING_BLOB, shardSnapshotRefs, ((builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("blob_name", blobName); + builder.field("file_name", fileInfo.physicalName()); + builder.field("part", finalPart); + builder.field("number_of_parts", fileInfo.numberOfParts()); + builder.humanReadableField("file_length_in_bytes", "file_length", fileLength); + builder.humanReadableField("part_length_in_bytes", "part_length", partLength); + return builder; + })); return false; } else if (blobInfo.length() != partLength.getBytes()) { - addFailure( - "snapshot [%s] for shard %s[%d] has blob [%s] for [%s] part [%d/%d] with length [%s] instead of [%s]; " - + "file length [%s]", - snapshot, - indexId, - shardId, - blobName, - fileInfo.physicalName(), - part, - fileInfo.numberOfParts(), - formatExact(ByteSizeValue.ofBytes(blobInfo.length())), - formatExact(partLength), - formatExact(fileLength) - ); + addAnomaly(Anomaly.MISMATCHED_BLOB_LENGTH, shardSnapshotRefs, ((builder, params) -> { + snapshotDescription.writeXContent(builder); + indexDescription.writeXContent(builder); + builder.field("shard", shardId); + builder.field("blob_name", blobName); + builder.field("file_name", fileInfo.physicalName()); + builder.field("part", finalPart); + builder.field("number_of_parts", fileInfo.numberOfParts()); + builder.humanReadableField("file_length_in_bytes", "file_length", fileLength); + builder.humanReadableField("part_length_in_bytes", "part_length", partLength); + builder.humanReadableField("actual_length_in_bytes", "actual_length", ByteSizeValue.ofBytes(blobInfo.length())); + return builder; + })); return false; } } @@ -555,11 +630,31 @@ private boolean verifyFileInfo( private ActionListener makeListener(RefCounted refCounted, CheckedConsumer consumer) { refCounted.incRef(); - return ActionListener.runAfter(ActionListener.wrap(consumer, this::addFailure), refCounted::decRef); + return ActionListener.runAfter( + ActionListener.wrap(consumer, exception -> addExceptionResult(refCounted, exception)), + refCounted::decRef + ); } - private ActionListener makeVoidListener(RefCounted refCounted, CheckedRunnable runnable) { - return makeListener(refCounted, ignored -> runnable.run()); + private void addExceptionResult(RefCounted refCounted, Exception exception) { + if (isCancelledSupplier.getAsBoolean() && exception instanceof TaskCancelledException) { + return; + } + addAnomaly(Anomaly.UNEXPECTED_EXCEPTION, refCounted, (builder, params) -> { + ElasticsearchException.generateFailureXContent(builder, params, exception, true); + return builder; + }); + } + + private Runnable wrapRunnable(RefCounted refCounted, Runnable runnable) { + refCounted.incRef(); + return () -> { + try { + runnable.run(); + } finally { + refCounted.decRef(); + } + }; } private void forkSupply(CheckedSupplier supplier, ActionListener listener) { @@ -632,175 +727,145 @@ private void forkSupply(RefCounted refCounted, CheckedSupplier } private void onCompletion() { - final var finalFailureCount = failureCount.get(); - if (finalFailureCount > verifyRequest.getMaxFailures()) { - failures.add( - new RepositoryVerificationException( - repositoryName, - format( - "found %d verification failures in total, %d suppressed", - finalFailureCount, - finalFailureCount - verifyRequest.getMaxFailures() + final var completionRefs = AbstractRefCounted.of( + () -> client.admin() + .indices() + .prepareFlush(RESULTS_INDEX) + .execute( + finalListener.delegateFailure( + (l1, ignored1) -> client.admin() + .indices() + .prepareRefresh(RESULTS_INDEX) + .execute(l1.delegateFailure((l2, ignored2) -> l2.onResponse(anomalyCount.get()))) ) ) - ); - } - - if (isCancelledSupplier.getAsBoolean()) { - failures.add(new RepositoryVerificationException(repositoryName, "verification task cancelled before completion")); + ); + try { + blobStoreRepository.getRepositoryData(makeListener(completionRefs, finalRepositoryData -> { + final var finalRepositoryGeneration = finalRepositoryData.getGenId(); + addResult(completionRefs, (builder, params) -> { + builder.field("completed", true); + builder.field("cancelled", isCancelledSupplier.getAsBoolean()); + builder.field("final_repository_generation", finalRepositoryGeneration); + builder.field("total_anomalies", anomalyCount.get()); + return builder; + }); + })); + } finally { + completionRefs.decRef(); } - - finalListener.onResponse(failures.stream().toList()); - } - - private interface RefCountedListenerWrapper extends Releasable, RefCounted {} - - private static RefCountedListenerWrapper wrap(ActionListener listener) { - final var refCounted = AbstractRefCounted.of(() -> listener.onResponse(null)); - return new RefCountedListenerWrapper() { - @Override - public void incRef() { - refCounted.incRef(); - } - - @Override - public boolean tryIncRef() { - return refCounted.tryIncRef(); - } - - @Override - public boolean decRef() { - return refCounted.decRef(); - } - - @Override - public boolean hasReferences() { - return refCounted.hasReferences(); - } - - @Override - public void close() { - decRef(); - } - }; } private static void runThrottled( - ActionListener completionListener, Iterator iterator, - BiConsumer consumer, + BiConsumer itemConsumer, int maxConcurrency, - ProgressLogger progressLogger + AtomicLong progressCounter, + Runnable onCompletion ) { - try (var throttledIterator = new ThrottledIterator<>(completionListener, iterator, consumer, maxConcurrency, progressLogger)) { - throttledIterator.run(); - } + ThrottledIterator.run(iterator, itemConsumer, maxConcurrency, progressCounter::incrementAndGet, onCompletion); } - private static class ThrottledIterator implements Releasable { - private final RefCountedListenerWrapper throttleRefs; - private final Iterator iterator; - private final BiConsumer consumer; - private final Semaphore permits; - private final ProgressLogger progressLogger; - - ThrottledIterator( - ActionListener completionListener, - Iterator iterator, - BiConsumer consumer, - int maxConcurrency, - ProgressLogger progressLogger - ) { - this.throttleRefs = wrap(completionListener); - this.iterator = iterator; - this.consumer = consumer; - this.permits = new Semaphore(maxConcurrency); - this.progressLogger = progressLogger; - } - - void run() { - while (permits.tryAcquire()) { - final T item; - synchronized (iterator) { - if (iterator.hasNext()) { - item = iterator.next(); - } else { - permits.release(); - return; - } - } - try (var itemRefsWrapper = new RefCountedWrapper()) { - consumer.accept(itemRefsWrapper.unwrap(), item); - } - } - } + private final Queue> pendingResults = ConcurrentCollections.newQueue(); + private final Semaphore resultsIndexingSemaphore = new Semaphore(1); - @Override - public void close() { - throttleRefs.close(); + private void indexResultDoc(IndexRequest indexRequest, Runnable onCompletion) { + if (indexRequest == null) { + onCompletion.run(); + return; } + pendingResults.add(Tuple.tuple(indexRequest, onCompletion)); + processPendingResults(); + } - // Wraps a RefCounted with protection against calling back into run() from within safeDecRef() - private class RefCountedWrapper implements Releasable { - private boolean isRecursive; - private final RefCounted itemRefs = AbstractRefCounted.of(this::onItemCompletion); + private void processPendingResults() { + while (resultsIndexingSemaphore.tryAcquire()) { + final var bulkRequest = new BulkRequest(); + final var completionActions = new ArrayList(); - RefCountedWrapper() { - throttleRefs.incRef(); + Tuple nextItem; + while ((nextItem = pendingResults.poll()) != null) { + bulkRequest.add(nextItem.v1()); + completionActions.add(nextItem.v2()); } - RefCounted unwrap() { - return itemRefs; - } - - private synchronized boolean isRecursive() { - return isRecursive; + if (completionActions.isEmpty()) { + resultsIndexingSemaphore.release(); + return; } - private void onItemCompletion() { - progressLogger.maybeLogProgress(); - permits.release(); - try { - if (isRecursive() == false) { - run(); - } - } finally { - throttleRefs.decRef(); + final var isRecursing = new AtomicBoolean(true); + client.bulk(bulkRequest, ActionListener.wrap(() -> { + resultsIndexingSemaphore.release(); + for (final var completionAction : completionActions) { + completionAction.run(); } - } - - @Override - public synchronized void close() { - // if the decRef() below releases the last ref and calls onItemCompletion(), no other thread blocks on us; conversely - // if decRef() doesn't release the last ref then we release this mutex immediately so the closing thread can proceed - isRecursive = true; - itemRefs.decRef(); - isRecursive = false; - } + if (isRecursing.get() == false) { + processPendingResults(); + } + }).delegateResponse((l, e) -> { + logger.error("error indexing results", e); + l.onFailure(e); + })); + isRecursing.set(false); } } - private class ProgressLogger { - private final String type; - private final long expectedMax; - private final long logFrequency; - private final AtomicLong currentCount = new AtomicLong(); + private static final DateFormatter dateFormatter = DateFormatter.forPattern(FormatNames.ISO8601.getName()).withLocale(Locale.ROOT); - ProgressLogger(String type, long expectedMax, long logFrequency) { - this.type = type; - this.expectedMax = expectedMax; - this.logFrequency = logFrequency; + private IndexRequest buildResultDoc(ToXContent toXContent) { + try (var builder = XContentFactory.jsonBuilder()) { + builder.startObject(); + builder.field( + "@timestamp", + dateFormatter.format(Instant.ofEpochMilli(blobStoreRepository.threadPool().absoluteTimeInMillis())) + ); + builder.field("repository", repositoryName); + builder.field("uuid", repositoryData.getUuid()); + builder.field("repository_generation", repositoryData.getGenId()); + toXContent.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + return new IndexRequestBuilder(client, IndexAction.INSTANCE, RESULTS_INDEX).setSource(builder).request(); + } catch (Exception e) { + logger.error("error generating failure output", e); + return null; } + } + + private void addAnomaly(Anomaly anomaly, RefCounted refCounted, ToXContent toXContent) { + anomalyCount.incrementAndGet(); + addResult(refCounted, (builder, params) -> toXContent.toXContent(builder.field("anomaly", anomaly.toString()), params)); + } - long getExpectedMax() { - return expectedMax; + private void addResult(RefCounted refCounted, ToXContent toXContent) { + refCounted.incRef(); + indexResultDoc(buildResultDoc(toXContent), refCounted::decRef); + } + + private record SnapshotDescription(SnapshotId snapshotId, long startTimeMillis, long endTimeMillis) { + void writeXContent(XContentBuilder builder) throws IOException { + builder.startObject("snapshot"); + builder.field("id", snapshotId.getUUID()); + builder.field("name", snapshotId.getName()); + builder.field("start_time_millis", startTimeMillis); + builder.field("end_time_millis", startTimeMillis); + builder.endObject(); } + } - void maybeLogProgress() { - final var count = currentCount.incrementAndGet(); - if (count == expectedMax || count % logFrequency == 0 && isCancelledSupplier.getAsBoolean() == false) { - logger.info("[{}] processed [{}] of [{}] {}", repositoryName, count, expectedMax, type); - } + private record IndexDescription(IndexId indexId, String indexMetadataBlob, int shardCount) { + void writeXContent(XContentBuilder builder) throws IOException { + builder.startObject("index"); + writeIndexId(indexId, builder); + builder.field("metadata_blob", indexMetadataBlob); + builder.field("shards", shardCount); + builder.endObject(); } } + + private static void writeIndexId(IndexId indexId, XContentBuilder builder) throws IOException { + builder.field("id", indexId.getId()); + builder.field("name", indexId.getName()); + } + } diff --git a/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java b/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java index 01b3febe31a2c..ece3a432f059b 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java +++ b/server/src/main/java/org/elasticsearch/rest/action/RestChunkedToXContentListener.java @@ -12,6 +12,7 @@ import org.elasticsearch.rest.ChunkedRestResponseBody; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; import java.io.IOException; @@ -27,8 +28,6 @@ public RestChunkedToXContentListener(RestChannel channel) { @Override protected void processResponse(Response response) throws IOException { - channel.sendResponse( - new RestResponse(response.getRestStatus(), ChunkedRestResponseBody.fromXContent(response, channel.request(), channel)) - ); + channel.sendResponse(new RestResponse(RestStatus.OK, ChunkedRestResponseBody.fromXContent(response, channel.request(), channel))); } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java index 169e56ab03e25..6f6696b6655f0 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestVerifyRepositoryIntegrityAction.java @@ -8,13 +8,15 @@ package org.elasticsearch.rest.action.admin.cluster; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.repositories.integrity.VerifyRepositoryIntegrityAction; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.rest.action.RestCancellableNodeClient; -import org.elasticsearch.rest.action.RestChunkedToXContentListener; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.tasks.TaskId; import java.io.IOException; import java.util.List; @@ -37,16 +39,20 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC final var verifyRequest = new VerifyRepositoryIntegrityAction.Request( request.param("repository"), request.paramAsStringArray("indices", Strings.EMPTY_ARRAY), - request.paramAsInt("threadpool_concurrency", 0), - request.paramAsInt("snapshot_verification_concurrency", 5), - request.paramAsInt("index_verification_concurrency", 5), - request.paramAsInt("index_snapshot_verification_concurrency", 5), - request.paramAsInt("max_failures", 10000), - request.paramAsBoolean("permit_missing_snapshot_details", false) + request.paramAsInt("thread_pool_concurrency", 0), + request.paramAsInt("snapshot_verification_concurrency", 0), + request.paramAsInt("index_verification_concurrency", 0), + request.paramAsInt("index_snapshot_verification_concurrency", 0) ); verifyRequest.masterNodeTimeout(request.paramAsTime("master_timeout", verifyRequest.masterNodeTimeout())); - return channel -> new RestCancellableNodeClient(client, request.getHttpChannel()).admin() - .cluster() - .execute(VerifyRepositoryIntegrityAction.INSTANCE, verifyRequest, new RestChunkedToXContentListener<>(channel)); + return channel -> { + final var task = client.executeLocally(VerifyRepositoryIntegrityAction.INSTANCE, verifyRequest, ActionListener.noop()); + try (var builder = channel.newBuilder()) { + builder.startObject(); + builder.field("task", new TaskId(client.getLocalNodeId(), task.getId()).toString()); + builder.endObject(); + channel.sendResponse(new RestResponse(RestStatus.OK, builder)); + } + }; } } diff --git a/server/src/test/java/org/elasticsearch/common/util/concurrent/ThrottledIteratorTests.java b/server/src/test/java/org/elasticsearch/common/util/concurrent/ThrottledIteratorTests.java new file mode 100644 index 0000000000000..e4d8f73af4bbb --- /dev/null +++ b/server/src/test/java/org/elasticsearch/common/util/concurrent/ThrottledIteratorTests.java @@ -0,0 +1,110 @@ +/* + * 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 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 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.common.util.concurrent; + +import org.apache.lucene.tests.util.LuceneTestCase; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.FixedExecutorBuilder; +import org.elasticsearch.threadpool.ScalingExecutorBuilder; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BooleanSupplier; +import java.util.stream.IntStream; + +public class ThrottledIteratorTests extends ESTestCase { + private static final String CONSTRAINED = "constrained"; + private static final String RELAXED = "relaxed"; + + public void testConcurrency() throws InterruptedException { + final var maxConstrainedThreads = between(1, 3); + final var maxRelaxedThreads = between(1, 100); + final var constrainedQueue = between(3, 6); + final var threadPool = new TestThreadPool( + "test", + new FixedExecutorBuilder(Settings.EMPTY, CONSTRAINED, maxConstrainedThreads, constrainedQueue, CONSTRAINED, false), + new ScalingExecutorBuilder(RELAXED, 1, maxRelaxedThreads, TimeValue.timeValueSeconds(30), true) + ); + try { + final var items = between(1, 10000); // large enough that inadvertent recursion will trigger a StackOverflowError + final var itemStartLatch = new CountDownLatch(items); + final var completedItems = new AtomicInteger(); + final var maxConcurrency = between(1, (constrainedQueue + maxConstrainedThreads) * 2); + final var itemPermits = new Semaphore(maxConcurrency); + final var completionLatch = new CountDownLatch(1); + final BooleanSupplier forkSupplier = randomFrom( + () -> false, + ESTestCase::randomBoolean, + LuceneTestCase::rarely, + LuceneTestCase::usually, + () -> true + ); + final var blockPermits = new Semaphore(between(0, Math.min(maxRelaxedThreads, maxConcurrency) - 1)); + + ThrottledIterator.run(IntStream.range(0, items).boxed().iterator(), (refs, item) -> { + assertTrue(itemPermits.tryAcquire()); + if (forkSupplier.getAsBoolean()) { + refs.incRef(); + final var executor = randomFrom(CONSTRAINED, RELAXED); + threadPool.executor(executor).execute(new AbstractRunnable() { + + @Override + public void onRejection(Exception e) { + assertEquals(CONSTRAINED, executor); + itemStartLatch.countDown(); + } + + @Override + protected void doRun() { + itemStartLatch.countDown(); + if (RELAXED.equals(executor) && randomBoolean() && blockPermits.tryAcquire()) { + // simulate at most (maxConcurrency-1) long-running operations, to demonstrate that they don't + // hold up the processing of the other operations + try { + assertTrue(itemStartLatch.await(30, TimeUnit.SECONDS)); + } catch (InterruptedException e) { + throw new AssertionError("unexpected", e); + } finally { + blockPermits.release(); + } + } + } + + @Override + public void onAfter() { + itemPermits.release(); + refs.decRef(); + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError("unexpected", e); + } + }); + } else { + itemStartLatch.countDown(); + itemPermits.release(); + } + }, maxConcurrency, completedItems::incrementAndGet, completionLatch::countDown); + + assertTrue(completionLatch.await(30, TimeUnit.SECONDS)); + assertEquals(items, completedItems.get()); + assertTrue(itemPermits.tryAcquire(maxConcurrency)); + assertTrue(itemStartLatch.await(0, TimeUnit.SECONDS)); + } finally { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + } +}