From 9a73ba75e3d2ea26bf1e6225f606002d83e65a93 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 14 Oct 2021 15:36:50 +0200 Subject: [PATCH 01/42] Add snapshots pending deletion in cluster state to delete snapshot when searchable snapshot index --- .../delete/TransportDeleteSnapshotAction.java | 2 +- .../elasticsearch/cluster/ClusterModule.java | 2 + .../cluster/SnapshotDeletionsInPending.java | 249 ++++++++++ .../cluster/SnapshotDeletionsInProgress.java | 9 + .../metadata/MetadataDeleteIndexService.java | 133 +++++- .../repositories/RepositoriesService.java | 2 + .../blobstore/BlobStoreRepository.java | 4 +- .../snapshots/RestoreService.java | 21 +- .../snapshots/SnapshotsService.java | 437 ++++++++++++++++-- .../MetadataDeleteIndexServiceTests.java | 113 ++++- .../BaseSearchableSnapshotsIntegTestCase.java | 23 +- ...leSnapshotsPendingDeletionsIntegTests.java | 429 +++++++++++++++++ ...archableSnapshotsRepositoryIntegTests.java | 26 +- 13 files changed, 1381 insertions(+), 69 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java create mode 100644 x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java index 32fdb285cb808..6de81eba3acdb 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/delete/TransportDeleteSnapshotAction.java @@ -64,6 +64,6 @@ protected void masterOperation( ClusterState state, final ActionListener listener ) { - snapshotsService.deleteSnapshots(request, listener.map(v -> AcknowledgedResponse.TRUE)); + snapshotsService.deleteSnapshotsByName(request, listener.map(v -> AcknowledgedResponse.TRUE)); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index e7bc6e4f7ce42..21514849cf60a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -125,6 +125,8 @@ public static List getNamedWriteables() { SnapshotDeletionsInProgress::readDiffFrom); registerClusterCustom(entries, RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress::new, RepositoryCleanupInProgress::readDiffFrom); + registerClusterCustom(entries, SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending::new, + SnapshotDeletionsInPending::readDiffFrom); // Metadata registerMetadataCustom(entries, RepositoriesMetadata.TYPE, RepositoriesMetadata::new, RepositoriesMetadata::readDiffFrom); registerMetadataCustom(entries, IngestMetadata.TYPE, IngestMetadata::new, IngestMetadata::readDiffFrom); diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java new file mode 100644 index 0000000000000..104292423f73c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java @@ -0,0 +1,249 @@ +/* + * 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.cluster; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterState.Custom; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.xcontent.ToXContentObject; +import org.elasticsearch.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.Objects; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.function.Consumer; + +import static java.util.Collections.unmodifiableSortedSet; + +/** + * Represents snapshots marked as to be deleted and pending deletion. + */ +public class SnapshotDeletionsInPending extends AbstractNamedDiffable implements Custom { + + public static final SnapshotDeletionsInPending EMPTY = new SnapshotDeletionsInPending(Collections.emptySortedSet()); + public static final String TYPE = "snapshot_deletions_pending"; + + public static final int MAX_PENDING_DELETIONS = 500; + + /** + * A list of snapshots to delete, sorted by creation time + */ + private final SortedSet entries; + + private SnapshotDeletionsInPending(SortedSet entries) { + this.entries = unmodifiableSortedSet(Objects.requireNonNull(entries)); + assert entries.size() <= MAX_PENDING_DELETIONS : entries.size() + " > " + MAX_PENDING_DELETIONS; + } + + public SnapshotDeletionsInPending(StreamInput in) throws IOException { + this(new TreeSet<>(in.readSet(Entry::new))); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeCollection(entries); + } + + @Override + public String getWriteableName() { + return TYPE; + } + + public boolean isEmpty() { + return entries.isEmpty(); + } + + public boolean contains(SnapshotId snapshotId) { + return entries.stream().anyMatch(entry -> Objects.equals(entry.getSnapshotId(), snapshotId)); + } + + public SortedSet entries() { + return entries; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startArray(TYPE); + for (Entry entry : entries) { + entry.toXContent(builder, params); + } + builder.endArray(); + return builder; + } + + @Override + public Version getMinimalSupportedVersion() { + return Version.CURRENT.minimumCompatibilityVersion(); + } + + public SnapshotDeletionsInPending withRemovedSnapshots(Set snapshotIds) { + if (snapshotIds == null || snapshotIds.isEmpty()) { + return this; + } + boolean changed = false; + final SortedSet updatedEntries = new TreeSet<>(entries); + if (updatedEntries.removeIf(entry -> snapshotIds.contains(entry.getSnapshotId()))) { + changed = true; + } + if (changed == false) { + return this; + } else if (updatedEntries.isEmpty()) { + return EMPTY; + } else { + return new SnapshotDeletionsInPending(updatedEntries); + } + } + + @Override + public String toString() { + final StringBuilder builder = new StringBuilder("SnapshotDeletionsInPending["); + boolean prepend = true; + + final Iterator iterator = entries.stream().iterator(); + while (iterator.hasNext()) { + if (prepend == false) { + builder.append(','); + } + final Entry entry = iterator.next(); + builder.append('[').append(entry.repositoryName).append('/').append(entry.repositoryUuid).append(']'); + builder.append('[').append(entry.snapshotId).append(',').append(entry.creationTime).append(']'); + builder.append('\n'); + prepend = false; + } + builder.append(']'); + return builder.toString(); + } + + public static class Entry implements Writeable, ToXContentObject, Comparable { + + private final String repositoryName; + private final String repositoryUuid; + private final SnapshotId snapshotId; + private final long creationTime; + + public Entry(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long creationTime) { + this.repositoryName = Objects.requireNonNull(repositoryName); + this.repositoryUuid = Objects.requireNonNull(repositoryUuid); + this.snapshotId = Objects.requireNonNull(snapshotId); + this.creationTime = creationTime; + } + + private Entry(StreamInput in) throws IOException { + this.repositoryName = in.readString(); + this.repositoryUuid = in.readString(); + this.creationTime = in.readVLong(); + this.snapshotId = new SnapshotId(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(repositoryName); + out.writeString(repositoryUuid); + out.writeVLong(creationTime); + snapshotId.writeTo(out); + } + + public String getRepositoryName() { + return repositoryName; + } + + public String getRepositoryUuid() { + return repositoryUuid; + } + + public SnapshotId getSnapshotId() { + return snapshotId; + } + + public long getCreationTime() { + return creationTime; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Entry entry = (Entry) o; + return creationTime == entry.creationTime + && Objects.equals(repositoryName, entry.repositoryName) + && Objects.equals(repositoryUuid, entry.repositoryUuid) + && Objects.equals(snapshotId, entry.snapshotId); + } + + @Override + public int hashCode() { + return Objects.hash(repositoryName, repositoryUuid, snapshotId, creationTime); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + { + builder.field("repository_name", repositoryName); + builder.field("repository_uuid", repositoryUuid); + builder.timeField("creation_time_millis", "creation_time", creationTime); + builder.field("snapshot", snapshotId); + } + builder.endObject(); + return builder; + } + + @Override + public int compareTo(final Entry other) { + return Comparator.comparingLong(Entry::getCreationTime) + .reversed() + .thenComparing(Entry::getSnapshotId) + .compare(this, other); + } + } + + public static final class Builder { + + private final SortedSet entries = new TreeSet<>(); + private final Consumer consumer; + + public Builder(SnapshotDeletionsInPending snapshotDeletionsInPending, Consumer onLimitExceeded) { + entries.addAll(snapshotDeletionsInPending.entries); + this.consumer = onLimitExceeded; + } + + private void ensureLimit() { + while (entries.size() >= MAX_PENDING_DELETIONS) { + final Entry removed = entries.last(); + entries.remove(removed); + if (consumer != null) { + consumer.accept(removed); + } + } + } + + public Builder add(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long creationTime) { + ensureLimit(); + entries.add(new Entry(repositoryName, repositoryUuid, snapshotId, creationTime)); + return this; + } + + public SnapshotDeletionsInPending build() { + ensureLimit(); + return entries.isEmpty() == false ? new SnapshotDeletionsInPending(entries) : EMPTY; + } + } + + public static NamedDiff readDiffFrom(StreamInput in) throws IOException { + return readDiffFrom(Custom.class, TYPE, in); + } +} diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java index a936669f32fd8..244bf9809a52d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java @@ -114,6 +114,15 @@ public boolean hasExecutingDeletion(String repository) { return false; } + /** + * Checks if the current {@link SnapshotDeletionsInProgress} contains the given {@link SnapshotId} + * + * @param snapshotId the snapshot id + */ + public boolean contains(SnapshotId snapshotId) { + return getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(snapshotId)); + } + /** * Returns {@code true} if there are snapshot deletions in progress in the cluster, * returns {@code false} otherwise. diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 1fd597cc7f9dc..7d59e00f3067c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -10,12 +10,14 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.delete.DeleteIndexClusterStateUpdateRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -26,16 +28,28 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.Index; +import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.snapshots.RestoreService; +import org.elasticsearch.snapshots.SearchableSnapshotsSettings; +import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInProgressException; import org.elasticsearch.snapshots.SnapshotsService; +import java.time.Instant; +import java.time.ZoneOffset; import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Objects; import java.util.Set; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY; + /** * Deletes indices. */ @@ -45,7 +59,6 @@ public class MetadataDeleteIndexService { private final Settings settings; private final ClusterService clusterService; - private final AllocationService allocationService; @Inject @@ -120,26 +133,116 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) logger.trace("{} tombstones purged from the cluster state. Previous tombstone size: {}. Current tombstone size: {}.", graveyardBuilder.getNumPurged(), previousGraveyardSize, currentGraveyard.getTombstones().size()); - Metadata newMetadata = metadataBuilder.build(); - ClusterBlocks blocks = clusterBlocksBuilder.build(); + final ClusterState.Builder builder = ClusterState.builder(currentState) + .routingTable(routingTableBuilder.build()) + .blocks(clusterBlocksBuilder.build()) + .metadata(metadataBuilder.build()); + + ImmutableOpenMap.Builder customBuilder = null; // update snapshot restore entries - ImmutableOpenMap customs = currentState.getCustoms(); final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); RestoreInProgress updatedRestoreInProgress = RestoreService.updateRestoreStateWithDeletedIndices(restoreInProgress, indices); if (updatedRestoreInProgress != restoreInProgress) { - ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(customs); - builder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); - customs = builder.build(); + customBuilder = ImmutableOpenMap.builder(currentState.getCustoms()); + customBuilder.put(RestoreInProgress.TYPE, updatedRestoreInProgress); + } + + // update snapshot(s) marked as to delete + final SnapshotDeletionsInPending deletionsInPending = currentState.custom( + SnapshotDeletionsInPending.TYPE, + SnapshotDeletionsInPending.EMPTY + ); + final SnapshotDeletionsInPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, meta); + if (updatedPendingDeletes != deletionsInPending) { + if (customBuilder == null) { + customBuilder = ImmutableOpenMap.builder(currentState.getCustoms()); + } + customBuilder.put(SnapshotDeletionsInPending.TYPE, updatedPendingDeletes); } + if (customBuilder != null) { + builder.customs(customBuilder.build()); + } + return allocationService.reroute(builder.build(), "deleted indices [" + indices + "]"); + } - return allocationService.reroute( - ClusterState.builder(currentState) - .routingTable(routingTableBuilder.build()) - .metadata(newMetadata) - .blocks(blocks) - .customs(customs) - .build(), - "deleted indices [" + indices + "]"); + private SnapshotDeletionsInPending updateSnapshotDeletionsPending( + final SnapshotDeletionsInPending pendingDeletions, + final Set indicesToDelete, + final Metadata metadata + ) { + if (indicesToDelete.isEmpty() == false) { + final long timestamp = Instant.now().toEpochMilli(); + SnapshotDeletionsInPending.Builder builder = null; + boolean changed = false; + + for (Index indexToDelete : indicesToDelete) { + final Settings indexSettings = metadata.getIndexSafe(indexToDelete).getSettings(); + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings) == false) { + continue; // not a searchable snapshot index + } + if (indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) == false) { + continue; // do not delete the snapshot when this searchable snapshot index is deleted + } + final SnapshotId snapshotId = new SnapshotId( + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY), + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY) + ); + boolean canDeleteSnapshot = true; + for (IndexMetadata other : metadata) { + if (indexToDelete.equals(other.getIndex())) { + continue; // do not check against itself + } + final Settings otherSettings = other.getSettings(); + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { + continue; // other index is not a searchable snapshot index, skip + } + final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); + if (Objects.equals(snapshotId.getUUID(), otherSnapshotUuid) == false) { + continue; // other index is backed by a different snapshot, skip + } + assert otherSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) : other; + if (indicesToDelete.contains(other.getIndex())) { + continue; // other index is going to be deleted as part of the same cluster state update + } + logger.debug( + "snapshot [{}] cannot be marked as to delete, another index [{}] is using the snapshot", + snapshotId, + other.getIndex() + ); + canDeleteSnapshot = false; // another index is using the same snapshot, do not delete the snapshot + break; + } + if (canDeleteSnapshot) { + if (builder == null) { + builder = new SnapshotDeletionsInPending.Builder( + pendingDeletions, + evicted -> logger.warn( + () -> new ParameterizedMessage( + "maximum number of snapshots [{}] awaiting deletion has been reached in " + + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", + SnapshotDeletionsInPending.MAX_PENDING_DELETIONS, + evicted.getSnapshotId(), + Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), + evicted.getRepositoryName(), + evicted.getRepositoryUuid() + ) + ) + ); + } + builder.add( + indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY), + indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, RepositoryData.MISSING_UUID), + snapshotId, + timestamp + ); + changed = true; + } + } + if (changed) { + return builder.build(); + } + } + return pendingDeletions; } } diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index f87cdf9c50a59..95a44ad39ec3a 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -350,6 +350,8 @@ public ClusterState execute(ClusterState currentState) { if (Regex.simpleMatch(request.name(), repositoryMetadata.name())) { ensureRepositoryNotInUse(currentState, repositoryMetadata.name()); ensureNoSearchableSnapshotsIndicesInUse(currentState, repositoryMetadata); + // TODO should we prevent the deletion of repositories that have snapshots pending deletions or should + // we just log a warning and rely on SnapshotDeletionsPending to catch up if the repo comes back deletedRepositories.add(repositoryMetadata.name()); changed = true; } else { 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 542994f30f8ef..5806e0cb5c188 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -35,6 +35,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RepositoryCleanupInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.metadata.IndexMetadata; @@ -2505,7 +2506,8 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state, } } updatedDeletionsInProgress = changedDeletions ? SnapshotDeletionsInProgress.of(deletionEntries) : null; - return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress); + final SnapshotDeletionsInPending pendingDeletions = state.custom(SnapshotDeletionsInPending.TYPE); + return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress, pendingDeletions); } private RepositoryMetadata getRepoMetadata(ClusterState state) { diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index f32fb46a010f2..6c41fb587208f 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.RestoreInProgress.ShardRestoreStatus; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.AliasMetadata; @@ -103,7 +104,6 @@ import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_VERSION_CREATED; import static org.elasticsearch.common.util.set.Sets.newHashSet; -import static org.elasticsearch.index.IndexModule.INDEX_STORE_TYPE_SETTING; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY; @@ -1058,7 +1058,7 @@ private static IndexMetadata updateIndexSettings( "cannot disable setting [" + IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey() + "] on restore" ); } - if ("snapshot".equals(INDEX_STORE_TYPE_SETTING.get(settings))) { + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(settings)) { final Boolean changed = changeSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, null); if (changed != null) { final Boolean previous = settings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, null); @@ -1188,7 +1188,7 @@ private final class RestoreSnapshotStateTask extends ClusterStateUpdateTask { @Override public ClusterState execute(ClusterState currentState) { // Check if the snapshot to restore is currently being deleted - ensureSnapshotNotDeleted(currentState); + ensureSnapshotNotDeletedOrPendingDeletion(currentState); // Clear out all existing indices which fall within a system index pattern being restored currentState = metadataDeleteIndexService.deleteIndices( @@ -1289,7 +1289,7 @@ && isSystemIndex(snapshotIndexMetadata) == false) { ); } - if ("snapshot".equals(INDEX_STORE_TYPE_SETTING.get(updatedIndexMetadata.getSettings()))) { + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(updatedIndexMetadata.getSettings())) { searchableSnapshotsIndices.add(updatedIndexMetadata.getIndex()); } } @@ -1366,7 +1366,7 @@ private void applyDataStreamRestores(ClusterState currentState, Metadata.Builder mdBuilder.dataStreams(updatedDataStreams, updatedDataStreamAliases); } - private void ensureSnapshotNotDeleted(ClusterState currentState) { + private void ensureSnapshotNotDeletedOrPendingDeletion(ClusterState currentState) { SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY @@ -1377,6 +1377,13 @@ private void ensureSnapshotNotDeleted(ClusterState currentState) { "cannot restore a snapshot while a snapshot deletion is in-progress [" + deletionsInProgress.getEntries().get(0) + "]" ); } + SnapshotDeletionsInPending pendingDeletions = currentState.custom(SnapshotDeletionsInPending.TYPE); + if (pendingDeletions != null && pendingDeletions.contains(snapshot.getSnapshotId())) { + throw new ConcurrentSnapshotExecutionException( + snapshot, + "cannot restore a snapshot already marked as deleted [" + snapshot.getSnapshotId() + "]" + ); + } } private void applyGlobalStateRestore(ClusterState currentState, Metadata.Builder mdBuilder) { @@ -1560,7 +1567,7 @@ private static void ensureSearchableSnapshotsRestorable( final Metadata metadata = currentState.metadata(); for (Index index : indices) { final Settings indexSettings = metadata.getIndexSafe(index).getSettings(); - assert "snapshot".equals(INDEX_STORE_TYPE_SETTING.get(indexSettings)) : "not a snapshot backed index: " + index; + assert SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings) : "not a snapshot backed index: " + index; final String repositoryUuid = indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY); final String repositoryName = indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY); @@ -1589,7 +1596,7 @@ private static void ensureSearchableSnapshotsRestorable( continue; // do not check the searchable snapshot index against itself } final Settings otherSettings = other.getSettings(); - if ("snapshot".equals(INDEX_STORE_TYPE_SETTING.get(otherSettings)) == false) { + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { continue; // other index is not a searchable snapshot index, skip } final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 0833f60e39dec..8b44cc1f1adde 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -35,6 +35,7 @@ import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus; @@ -48,6 +49,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.RepositoriesMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.IndexRoutingTable; @@ -64,7 +66,12 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; 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.CountDown; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; import org.elasticsearch.index.shard.ShardId; @@ -85,6 +92,8 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.time.Instant; +import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -113,6 +122,7 @@ import static java.util.Collections.unmodifiableList; import static org.elasticsearch.cluster.SnapshotsInProgress.completed; +import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; /** * Service responsible for creating snapshots. This service runs all the steps executed on the master node during snapshot creation and @@ -474,6 +484,14 @@ public ClusterState execute(ClusterState currentState) { "cannot clone from snapshot that is being deleted" ); } + if (currentState.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY).contains(sourceSnapshotId)) { + throw new ConcurrentSnapshotExecutionException( + repositoryName, + sourceSnapshotId.getName(), + "cannot clone a snapshot already marked as deleted [" + snapshot.getSnapshotId() + "]" + ); + } + ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); final List indicesForSnapshot = new ArrayList<>(); for (IndexId indexId : repositoryData.getIndices().values()) { @@ -559,6 +577,34 @@ private static void ensureSnapshotNameAvailableInRepo(RepositoryData repositoryD } } + private static Set listOfCloneSources(final ClusterState state) { + return state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .map(SnapshotsInProgress.Entry::source) + .collect(Collectors.toSet()); + } + + private static Set listOfRestoreSources(final ClusterState state) { + final Set snapshotIds = new HashSet<>(); + for (RestoreInProgress.Entry restore : state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) { + snapshotIds.add(restore.snapshot().getSnapshotId()); + } + return Set.copyOf(snapshotIds); + } + + private static Set listOfDeletionsSources(final ClusterState state) { + final SnapshotDeletionsInProgress deletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + if (deletionsInProgress == null) { + return Set.of(); + } + final Set snapshotIds = new HashSet<>(); + for (SnapshotDeletionsInProgress.Entry deletion : deletionsInProgress.getEntries()) { + snapshotIds.addAll(deletion.getSnapshots()); + } + return Set.copyOf(snapshotIds); + } + /** * Determine the number of shards in each index of a clone operation and update the cluster state accordingly. * @@ -663,7 +709,12 @@ public ClusterState execute(ClusterState currentState) { // shard snapshot state was based on all previous existing operations in progress // TODO: If we could eventually drop the snapshot clone init phase we don't need this any longer updatedEntries.add(updatedEntry); - return updateWithSnapshots(currentState, snapshotsInProgress.withUpdatedEntriesForRepo(repoName, updatedEntries), null); + return updateWithSnapshots( + currentState, + snapshotsInProgress.withUpdatedEntriesForRepo(repoName, updatedEntries), + null, + null + ); } return currentState; } @@ -948,6 +999,7 @@ public void applyClusterState(ClusterChangedEvent event) { newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event) ); + triggerSnapshotsPendingDeletions(event.state()); } else if (snapshotCompletionListeners.isEmpty() == false) { // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster @@ -1245,6 +1297,268 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS ); } + /** + * Set of pending snapshots deletions whose deletion is already triggered + */ + private final Set ongoingSnapshotsDeletions = ConcurrentCollections.newConcurrentSet(); + + /** + * Find snapshots to delete in the the cluster state and triggers explicit snapshot delete requests. This method attempts to detect + * conflicting situations where triggering the snapshot deletion would likely fail due to a concurrent snapshot operation. In such + * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates on the conflicting + * situation is resolved. + * + * The repository name and uuid information are extracted from the {@link SnapshotDeletionsInPending} entries in order to find the + * repository to execute the snapshot delete request against. If the repo uuid was known at the time the snapshot was added to + * {@link SnapshotDeletionsInPending} we try to find the corresponding repository, or a repository with a missing uuid but the same + * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsInPending}, we try to find a + * repository with the same name. + * + * @param state the current {@link ClusterState} + */ + private void triggerSnapshotsPendingDeletions(final ClusterState state) { + final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final SnapshotDeletionsInPending snapshotDeletionsInPending = state.custom(SnapshotDeletionsInPending.TYPE); + if (snapshotDeletionsInPending == null + || snapshotDeletionsInPending.isEmpty() + || repositories.repositories().isEmpty() + || state.nodes().isLocalNodeElectedMaster() == false + || state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { + return; + } + final Set currentDeletions = listOfDeletionsSources(state); + final Set currentRestores = listOfRestoreSources(state); + final Set currentClones = listOfCloneSources(state); + + // the list of snapshot ids to trigger deletion for, per repository + final Map> snapshotsToDelete = new HashMap<>(); + + for (SnapshotDeletionsInPending.Entry snapshot : snapshotDeletionsInPending.entries()) { + final SnapshotId snapshotId = snapshot.getSnapshotId(); + + // early add to avoid doing too much work on successive cluster state updates + if (ongoingSnapshotsDeletions.add(snapshotId)) { + boolean triggered = false; + try { + if (currentRestores.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); + continue; + } else if (currentClones.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); + continue; + } else if (currentDeletions.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is already queued", snapshotId); + continue; + } + + Optional optionalRepository; + if (RepositoryData.MISSING_UUID.equals(snapshot.getRepositoryUuid()) == false) { + // the snapshot waiting to be deleted references a repository with a known uuid, + // let's try to find this repository among the existing ones first + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), snapshot.getRepositoryUuid())) + .findFirst(); + if (optionalRepository.isEmpty()) { + // there is no existing repository matching the uuid, + // let's try to find the repository by name among the existing ones that have no uuid + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) + .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) + .findFirst(); + } + } else { + // the snapshot waiting to be deleted does not references a repository with a known uuid, + // let's try to find the repository by name among the existing ones, in the hope that + // the snapshot will be found there. + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) + .findFirst(); + } + + if (optionalRepository.isEmpty()) { + logger.debug( + "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + ); + continue; + } + + final RepositoryMetadata repository = optionalRepository.get(); + if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { + logger.debug( + "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", + repository.name(), + repository.uuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + ); + continue; + } + + // should we add some throttling to not always retry + final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); + assert ongoingSnapshotsDeletions.contains(snapshotId) : snapshotId; + assert added : snapshotId; + + logger.trace("triggering snapshot deletion for [{}]", snapshotId); + triggered = true; + } finally { + if (triggered == false) { + ongoingSnapshotsDeletions.remove(snapshotId); + } + } + } + } + snapshotsToDelete.forEach( + (repo, snapshots) -> threadPool.generic().execute(new SnapshotsToDeleteRunnable(repo.name(), repo.uuid(), snapshots)) + ); + } + + /** + * A {@link Runnable} used to process the deletion of snapshots marked as to delete for a given repository. + */ + private class SnapshotsToDeleteRunnable extends AbstractRunnable { + + private final Set snapshotIdsToDelete; + private final String repositoryName; + private final String repositoryUuid; + + SnapshotsToDeleteRunnable(String repositoryName, String repositoryUuid, Set snapshotIdsToDelete) { + this.repositoryName = Objects.requireNonNull(repositoryName); + this.repositoryUuid = Objects.requireNonNull(repositoryUuid); + this.snapshotIdsToDelete = Objects.requireNonNull(snapshotIdsToDelete); + assert snapshotIdsToDelete.isEmpty() == false; + } + + @Override + protected void doRun() throws Exception { + final Set missingSnapshots = ConcurrentCollections.newConcurrentSet(); + final CountDown countDown = new CountDown(snapshotIdsToDelete.size()); + + for (SnapshotId snapshotId : snapshotIdsToDelete) { + final ActionListener listener = new ActionListener() { + @Override + public void onResponse(Void unused) { + logger.debug("[{}] snapshot marked as to delete [{}] is now deleted", repositoryName, snapshotId); + removeSnapshot(snapshotId, true); + } + + @Override + public void onFailure(Exception e) { + boolean shouldRetry = true; + if (e instanceof SnapshotMissingException) { + logger.debug( + () -> new ParameterizedMessage( + "[{}] snapshot to delete [{}] is already deleted or is missing", + repositoryName, + snapshotId + ), + e + ); + // only retry missing snapshots if the repository uuid is unknown, otherwise the snapshot pending deletion entry + // is removed from the cluster state as we know it does not exist anymore in the repository + shouldRetry = RepositoryData.MISSING_UUID.equals(repositoryUuid) == false; + + } else if (e instanceof ConcurrentSnapshotExecutionException) { + logger.debug( + "[{}] failed to delete snapshot [{}]: a concurrent operation is running", + repositoryName, + snapshotId + ); + } else if (e instanceof RepositoryMissingException) { + logger.warn( + () -> new ParameterizedMessage( + "[{}] failed to delete snapshot [{}]: repository has been removed before snapshot marked as " + + "to delete could be deleted, the snapshot might be leaking", + repositoryName, + snapshotId + ), + e + ); + } else { + logger.warn( + () -> new ParameterizedMessage("[{}] failed to delete snapshot [{}]", repositoryName, snapshotId), + e + ); + } + removeSnapshot(snapshotId, shouldRetry); + } + + void removeSnapshot(SnapshotId snapshotId, boolean shouldRetry) { + if (shouldRetry) { + final boolean removed = ongoingSnapshotsDeletions.remove(snapshotId); + assert removed : "snapshot to delete [" + snapshotId + "] not found"; + } else { + missingSnapshots.add(snapshotId); + } + if (countDown.countDown() && missingSnapshots.isEmpty() == false) { + clusterService.submitStateUpdateTask( + "remove-missing-snapshot-deletions-in-pending", + new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + final SnapshotDeletionsInPending deletionsInPending = currentState.custom( + SnapshotDeletionsInPending.TYPE, + SnapshotDeletionsInPending.EMPTY + ); + final SnapshotDeletionsInPending updatedDeletionsInPending = deletionsInPending + .withRemovedSnapshots(missingSnapshots); + if (deletionsInPending == updatedDeletionsInPending) { + return currentState; + } + return ClusterState.builder(currentState) + .putCustom(SnapshotDeletionsInPending.TYPE, updatedDeletionsInPending) + .build(); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + ongoingSnapshotsDeletions.removeAll(missingSnapshots); + } + + @Override + public void onFailure(String source, Exception e) { + ongoingSnapshotsDeletions.removeAll(missingSnapshots); + } + } + ); + } + } + }; + + try { + deleteSnapshotsByUuid( + repositoryName, + new String[] { snapshotId.getUUID() }, + DeleteSnapshotRequest.DEFAULT_MASTER_NODE_TIMEOUT, + listener + ); + } catch (Exception e) { + logger.warn( + () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshot [{}]", repositoryName, snapshotId), + e + ); + listener.onFailure(e); + } + } + } + + @Override + public void onFailure(Exception e) { + ongoingSnapshotsDeletions.removeAll(snapshotIdsToDelete); + logger.warn( + () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshots {}", repositoryName, snapshotIdsToDelete), + e + ); + } + } + private static ImmutableOpenMap processWaitingShardsAndRemovedNodes( SnapshotsInProgress.Entry entry, RoutingTable routingTable, @@ -1946,7 +2260,8 @@ public ClusterState execute(ClusterState currentState) { updatedState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY), Collections.singletonList(snapshot.getSnapshotId()), snapshot.getRepository() - ) + ), + null ); } @@ -2016,24 +2331,74 @@ private void failSnapshotCompletionListeners(Snapshot snapshot, Exception e) { } /** - * Deletes snapshots from the repository. In-progress snapshots matched by the delete will be aborted before deleting them. + * Deletes snapshots from the repository. In-progress snapshots matched by the delete will be aborted before deleting them. Snapshots + * to delete are identified by their names. * * @param request delete snapshot request * @param listener listener */ - public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionListener listener) { - final String repositoryName = request.repository(); - final String[] snapshotNames = request.snapshots(); + public void deleteSnapshotsByName(final DeleteSnapshotRequest request, final ActionListener listener) { + deleteSnapshots(request.repository(), request.snapshots(), null, request.masterNodeTimeout(), SnapshotId::getName, listener); + } + + /** + * Deletes snapshots from the repository. In-progress snapshots matched by the delete will be aborted before deleting them. Snapshots + * to delete are identified by their UUIDs. + * + * @param repositoryName the name of the repository that contains the snapshots to delete + * @param snapshotUuids the uuids of the snapshots to delete + * @param masterNodeTimeout the timeout to use for the cluster state update task + * @param listener listener + */ + private void deleteSnapshotsByUuid( + final String repositoryName, + final String[] snapshotUuids, + final TimeValue masterNodeTimeout, + final ActionListener listener + ) { + deleteSnapshots(repositoryName, null, snapshotUuids, masterNodeTimeout, SnapshotId::getUUID, listener); + } + + /** + * Deletes snapshots from the repository. In-progress snapshots matched by the delete will be aborted before deleting them. + * Snapshots to delete are identified by converting their {@link SnapshotId} to a {@link String} using the mapping function + * {@code mapping}; the resulting string is then compared to the snapshots names/uuids/patterns to match against. + * + * @param repositoryName the name of the repository that contains the snapshots to delete + * @param snapshotNames the names of the snapshots to delete + * @param snapshotUuids the uuids of the snapshots to delete + * @param masterNodeTimeout the timeout to use for the cluster state update task + * @param mapping the mapping function used to match the {@link SnapshotId} against the given snapshotNamesOrUuids + * @param listener listener + */ + private void deleteSnapshots( + final String repositoryName, + final String[] snapshotNames, + final String[] snapshotUuids, + final TimeValue masterNodeTimeout, + final Function mapping, + final ActionListener listener + ) { + assert snapshotNames == null || snapshotNames.length > 0; + assert snapshotUuids == null || snapshotUuids.length > 0; + assert snapshotNames != null ^ snapshotUuids != null + : "either snapshots names or snapshots uuids must be not null, " + + "but got " + + Arrays.toString(snapshotNames) + + " and " + + Arrays.toString(snapshotUuids); + + final String[] snapshotsToDelete = snapshotNames != null ? snapshotNames : snapshotUuids; logger.info( () -> new ParameterizedMessage( "deleting snapshots [{}] from repository [{}]", - Strings.arrayToCommaDelimitedString(snapshotNames), + Strings.arrayToCommaDelimitedString(snapshotsToDelete), repositoryName ) ); final Repository repository = repositoriesService.repository(repositoryName); - repository.executeConsistentStateUpdate(repositoryData -> new ClusterStateUpdateTask(request.masterNodeTimeout()) { + repository.executeConsistentStateUpdate(repositoryData -> new ClusterStateUpdateTask(masterNodeTimeout) { private SnapshotDeletionsInProgress.Entry newDelete = null; @@ -2056,7 +2421,7 @@ public ClusterState execute(ClusterState currentState) { final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); for (SnapshotsInProgress.Entry entry : snapshotsInProgress.forRepo(repositoryName)) { final SnapshotId snapshotId = entry.snapshot().getSnapshotId(); - if (Regex.simpleMatch(snapshotNames, snapshotId.getName())) { + if (Regex.simpleMatch(snapshotsToDelete, mapping.apply(snapshotId))) { snapshotIds.add(snapshotId); } } @@ -2064,8 +2429,8 @@ public ClusterState execute(ClusterState currentState) { // find snapshots to delete in repository data final Map snapshotsIdsInRepository = repositoryData.getSnapshotIds() .stream() - .collect(Collectors.toMap(SnapshotId::getName, Function.identity())); - for (String snapshotOrPattern : snapshotNames) { + .collect(Collectors.toMap(mapping, Function.identity())); + for (String snapshotOrPattern : snapshotsToDelete) { if (Regex.isSimpleMatchPattern(snapshotOrPattern)) { for (Map.Entry entry : snapshotsIdsInRepository.entrySet()) { if (Regex.simpleMatch(snapshotOrPattern, entry.getKey())) { @@ -2075,7 +2440,7 @@ public ClusterState execute(ClusterState currentState) { } else { final SnapshotId foundId = snapshotsIdsInRepository.get(snapshotOrPattern); if (foundId == null) { - if (snapshotIds.stream().noneMatch(snapshotId -> snapshotId.getName().equals(snapshotOrPattern))) { + if (snapshotIds.stream().map(mapping).noneMatch(snapshot -> snapshot.equals(snapshotOrPattern))) { throw new SnapshotMissingException(repositoryName, snapshotOrPattern); } } else { @@ -2088,10 +2453,7 @@ public ClusterState execute(ClusterState currentState) { return currentState; } - final Set activeCloneSources = snapshotsInProgress.asStream() - .filter(SnapshotsInProgress.Entry::isClone) - .map(SnapshotsInProgress.Entry::source) - .collect(Collectors.toSet()); + final Set activeCloneSources = listOfCloneSources(currentState); for (SnapshotId snapshotId : snapshotIds) { if (activeCloneSources.contains(snapshotId)) { throw new ConcurrentSnapshotExecutionException( @@ -2108,11 +2470,6 @@ public ClusterState execute(ClusterState currentState) { "delete snapshot" ); - final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( - SnapshotDeletionsInProgress.TYPE, - SnapshotDeletionsInProgress.EMPTY - ); - final RestoreInProgress restoreInProgress = currentState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); // don't allow snapshot deletions while a restore is taking place, // otherwise we could end up deleting a snapshot that is being restored @@ -2154,9 +2511,14 @@ public ClusterState execute(ClusterState currentState) { ); if (snapshotIdsRequiringCleanup.isEmpty()) { // We only saw snapshots that could be removed from the cluster state right away, no need to update the deletions - return updateWithSnapshots(currentState, updatedSnapshots, null); + return updateWithSnapshots(currentState, updatedSnapshots, null, null); } + // add the snapshot deletion to the cluster state + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress.EMPTY + ); final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries() .stream() .filter(entry -> entry.repository().equals(repositoryName)) @@ -2194,7 +2556,8 @@ public ClusterState execute(ClusterState currentState) { currentState, updatedSnapshots, (replacedEntry == null ? deletionsInProgress : deletionsInProgress.withRemovedEntry(replacedEntry.uuid())) - .withAddedEntry(newDelete) + .withAddedEntry(newDelete), + null ); } @@ -2414,6 +2777,14 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres return updatedDeletions == null ? deletions : updatedDeletions; } + @Nullable + @Override + protected SnapshotDeletionsInPending filterPendingDeletions(@Nullable SnapshotDeletionsInPending pendingDeletions) { + return pendingDeletions != null + ? pendingDeletions.withRemovedSnapshots(Sets.newHashSet(deleteEntry.getSnapshots())) + : null; + } + @Override protected void handleListeners(List> deleteListeners) { assert repositoryData.getSnapshotIds().stream().noneMatch(deleteEntry.getSnapshots()::contains) @@ -2499,8 +2870,9 @@ public ClusterState execute(ClusterState currentState) { return currentState; } final SnapshotDeletionsInProgress newDeletions = filterDeletions(updatedDeletions); + SnapshotDeletionsInPending newPendingDeletions = filterPendingDeletions(currentState.custom(SnapshotDeletionsInPending.TYPE)); final Tuple> res = readyDeletions( - updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions) + updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions, newPendingDeletions) ); readyDeletions = res.v2(); return res.v1(); @@ -2517,6 +2889,10 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres return deletions; } + protected SnapshotDeletionsInPending filterPendingDeletions(SnapshotDeletionsInPending pendingDeletions) { + return pendingDeletions; + } + @Override public final void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { final List> deleteListeners; @@ -2691,19 +3067,21 @@ private void markShardReassigned(RepositoryShardId shardId, Set indices = new HashSet<>(); + + final int nbIndices = randomIntBetween(2, 10); + for (int i = 0; i < nbIndices; i++) { + Settings.Builder indexSettingsBuilder = Settings.builder() + .put("index.version.created", VersionUtils.randomVersion(random())) + .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE) + .put(SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, true) + .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY, repositoryMetadata.name()) + .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY, snapshotId.getName()) + .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY, snapshotId.getUUID()); + if (randomBoolean()) { + indexSettingsBuilder.put(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, repositoryMetadata.uuid()); + } + IndexMetadata indexMetadata = IndexMetadata.builder(randomAlphaOfLength(10) + i) + .settings(indexSettingsBuilder.build()) + .numberOfShards(randomIntBetween(1, 3)) + .numberOfReplicas(randomInt(1)) + .build(); + metadataBuilder.put(indexMetadata, false); + routingBuilder.addAsNew(indexMetadata); + indices.add(indexMetadata.getIndex()); + } + + ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .routingTable(routingBuilder.build()) + .metadata(metadataBuilder) + .build(); + + SnapshotDeletionsInPending pendingDeletions = + clusterState.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY); + while (indices.size() > 0) { + assertThat(pendingDeletions.isEmpty(), equalTo(true)); + + List indicesToDelete = randomSubsetOf(randomIntBetween(1, Math.max(1, indices.size() - 1)), indices); + clusterState = service.deleteIndices(clusterState, Set.copyOf(indicesToDelete)); + indicesToDelete.forEach(indices::remove); + + for (Index deletedIndex : indicesToDelete) { + assertThat(clusterState.metadata().index(deletedIndex), nullValue()); + assertThat(clusterState.routingTable().index(deletedIndex), nullValue()); + } + + pendingDeletions = clusterState.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY); + } + + assertThat(pendingDeletions.isEmpty(), equalTo(false)); + assertThat(pendingDeletions.contains(snapshotId), equalTo(true)); + } + private ClusterState clusterState(String index) { IndexMetadata indexMetadata = IndexMetadata.builder(index) .settings(Settings.builder().put("index.version.created", VersionUtils.randomVersion(random()))) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java index 2ca66b0871377..30a64eb693225 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java @@ -19,6 +19,8 @@ import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; +import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -26,6 +28,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.util.CollectionUtils; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; @@ -83,7 +86,7 @@ protected boolean addMockInternalEngine() { @Override protected Collection> nodePlugins() { - return List.of(LocalStateSearchableSnapshots.class); + return CollectionUtils.appendToCopy(super.nodePlugins(), LocalStateSearchableSnapshots.class); } @Override @@ -334,4 +337,22 @@ protected void assertExecutorIsIdle(String executorName) throws Exception { } }); } + + protected void awaitNoMoreSnapshotsDeletions() throws Exception { + final String master = internalCluster().getMasterName(); + awaitClusterState(logger, master, state -> { + final SnapshotDeletionsInProgress deletions = state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + if (deletions.hasDeletionsInProgress()) { + return false; + } + final SnapshotDeletionsInPending pendingDeletions = state.custom( + SnapshotDeletionsInPending.TYPE, + SnapshotDeletionsInPending.EMPTY + ); + if (pendingDeletions.isEmpty() == false) { + return false; + } + return true; + }); + } } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java new file mode 100644 index 0000000000000..081eec755c26f --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -0,0 +1,429 @@ +/* + * 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; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.searchablesnapshots; + +import org.apache.lucene.search.TotalHits; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsInPending; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.TriConsumer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException; +import org.elasticsearch.snapshots.RestoreInfo; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.snapshots.SnapshotMissingException; +import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; + +import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; +import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; +import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; + +public class SearchableSnapshotsPendingDeletionsIntegTests extends BaseFrozenSearchableSnapshotsIntegTestCase { + + public void testSnapshotPendingDeletionCannotBeMounted() throws Exception { + blockPendingDeletionThenExecute((repository, snapshot, index) -> { + ConcurrentSnapshotExecutionException exception = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> mountSnapshot(repository, snapshot.getName(), index, Settings.EMPTY) + ); + assertThat(exception.getMessage(), containsString("cannot restore a snapshot already marked as deleted")); + }); + } + + public void testSnapshotPendingDeletionCannotBeRestored() throws Exception { + blockPendingDeletionThenExecute((repository, snapshot, index) -> { + ConcurrentSnapshotExecutionException exception = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> client().admin().cluster().prepareRestoreSnapshot(repository, snapshot.getName()).setWaitForCompletion(true).get() + ); + assertThat(exception.getMessage(), containsString("cannot restore a snapshot already marked as deleted")); + }); + } + + public void testSearchableSnapshotIsDeletedWhenRepoIsRecreated() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + final Settings.Builder repositorySettings = getRepositorySettings(repository); + updateRepositoryReadOnly(repository, true); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + + assertAcked(client().admin().cluster().prepareDeleteRepository(repository)); + awaitSnapshotPendingDeletion(snapshot); + + final String repoName; + if (randomBoolean()) { + // re register the repository without verification: the snapshot + // pending deletion logic should try to delete the snapshot based + // on the repository name + repoName = repository; + createRepository(repoName, "mock", repositorySettings, false); + } else { + // re register the repository under a different name: the snapshot + // pending deletion logic should try to delete the snapshot based + // on the repository uuid + repoName = "new_" + repository; + createRepository(repoName, "mock", repositorySettings, true); + } + awaitNoMoreSnapshotsDeletions(); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repoName).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + + public void testSearchableSnapshotIsDeletedWithOnGoingRestore() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + final String masterNode = internalCluster().getMasterName(); + blockMasterOnShardLevelSnapshotFile(repository, getRepositoryData(repository).resolveIndexId(index).getId()); + + final ActionFuture restoreFuture = client().admin() + .cluster() + .prepareRestoreSnapshot(repository, snapshot.getName()) + .setIndices(index) + .setRenamePattern("(.+)") + .setRenameReplacement("old_$1") + .setWaitForCompletion(true) + .execute(); + awaitClusterState(state -> state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).isEmpty() == false); + waitForBlock(masterNode, repository); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + assertFalse(restoreFuture.isDone()); + + unblockNode(repository, masterNode); + awaitNoMoreSnapshotsDeletions(); + + final RestoreInfo restoreInfoResponse = restoreFuture.actionGet().getRestoreInfo(); + assertThat(restoreInfoResponse.successfulShards(), greaterThan(0)); + assertThat(restoreInfoResponse.failedShards(), equalTo(0)); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + + public void testSearchableSnapshotIsDeletedWithOnGoingClone() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + final String masterNode = internalCluster().getMasterName(); + blockMasterOnShardLevelSnapshotFile(repository, getRepositoryData(repository).resolveIndexId(index).getId()); + + final String cloneTarget = "target-snapshot"; + final ActionFuture cloneFuture = clusterAdmin().prepareCloneSnapshot( + repository, + snapshot.getName(), + cloneTarget + ).setIndices(index).execute(); + awaitNumberOfSnapshotsInProgress(1); + waitForBlock(masterNode, repository); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + assertFalse(cloneFuture.isDone()); + + unblockNode(repository, masterNode); + awaitNoMoreSnapshotsDeletions(); + assertAcked(cloneFuture.get()); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + + public void testSearchableSnapshotIsDeletedWithOnGoingDeletion() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + blockMasterOnWriteIndexFile(repository); + final String masterNode = internalCluster().getMasterName(); + + final ActionFuture deleteFuture = startDeleteSnapshot(repository, snapshot.getName()); + waitForBlock(masterNode, repository); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + assertFalse(deleteFuture.isDone()); + + unblockNode(repository, masterNode); + awaitNoMoreSnapshotsDeletions(); + assertAcked(deleteFuture.get()); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + + public void testSearchableSnapshotsDeletionsWithConcurrentDeletes() throws Exception { + final String repository = "repository"; + final Settings.Builder repositorySettings = randomRepositorySettings(); + createRepository(repository, FsRepository.TYPE, repositorySettings); + + final List indices = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(1, 10); i++) { + final String index = "index-" + i; + assertAcked( + prepareCreate( + index, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(INDEX_SOFT_DELETES_SETTING.getKey(), true) + ) + ); + indices.add(index); + } + + final int nbSnapshots = randomIntBetween(1, 10); + final CountDownLatch snapshotLatch = new CountDownLatch(nbSnapshots); + final List snapshots = new CopyOnWriteArrayList<>(); + for (int i = 0; i < nbSnapshots; i++) { + final String snapshot = "snapshot-" + i; + client().admin() + .cluster() + .prepareCreateSnapshot(repository, snapshot) + // must contain 1 index to be used with SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION + .setIndices(randomFrom(indices)) + .setWaitForCompletion(true) + .execute(new ActionListener<>() { + @Override + public void onResponse(CreateSnapshotResponse response) { + assertThat(response.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + snapshots.add(response.getSnapshotInfo()); + snapshotLatch.countDown(); + } + + @Override + public void onFailure(Exception e) { + final AssertionError error = new AssertionError("error during snapshot", e); + logger.error("test failed", error); + snapshotLatch.countDown(); + throw error; + } + }); + } + snapshotLatch.await(); + + final int nbMounts = randomIntBetween(1, 10); + final CountDownLatch mountLatch = new CountDownLatch(nbMounts); + final Map mounts = ConcurrentCollections.newConcurrentMap(); + + for (int i = 0; i < nbMounts; i++) { + final String mount = "mount-" + i; + + final SnapshotInfo snapshotInfo = randomFrom(snapshots); + client().execute( + MountSearchableSnapshotAction.INSTANCE, + new MountSearchableSnapshotRequest( + mount, + repository, + snapshotInfo.snapshot().getSnapshotId().getName(), + randomFrom(snapshotInfo.indices()), + Settings.builder().put(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, true).build(), + Strings.EMPTY_ARRAY, + true, + randomFrom(MountSearchableSnapshotRequest.Storage.values()) + ), + new ActionListener<>() { + @Override + public void onResponse(RestoreSnapshotResponse response) { + assertThat(response.getRestoreInfo().successfulShards(), greaterThan(0)); + assertThat(response.getRestoreInfo().failedShards(), equalTo(0)); + mounts.put(mount, snapshotInfo.snapshotId()); + mountLatch.countDown(); + } + + @Override + public void onFailure(Exception e) { + final AssertionError error = new AssertionError("error during mount", e); + logger.error("test failed", error); + mountLatch.countDown(); + throw error; + } + } + ); + } + mountLatch.await(); + + if (randomBoolean()) { + // we can't delete a repository that is used by searchable snapshot indices but we can update it + assertAcked( + clusterAdmin().preparePutRepository(repository) + .setType(FsRepository.TYPE) + .setVerify(randomBoolean()) + .setSettings(randomBoolean() ? repositorySettings : repositorySettings.put("dummy", randomInt())) + ); + } + + final CyclicBarrier startThreads = new CyclicBarrier(2); + final Thread deleteAllSnapshotsThread = new Thread(() -> { + try { + startThreads.await(); + final CountDownLatch latch = new CountDownLatch(snapshots.size()); + for (SnapshotInfo snapshot : snapshots) { + client().admin() + .cluster() + .prepareDeleteSnapshot(repository, snapshot.snapshotId().getName()) + .execute(ActionListener.wrap(latch::countDown)); + + } + latch.await(); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + deleteAllSnapshotsThread.start(); + + final Thread deleteAllMountedIndicesThread = new Thread(() -> { + try { + ensureGreen("mount-*"); + startThreads.await(); + final Set mountedIndices = new HashSet<>(mounts.keySet()); + do { + List deletions = randomSubsetOf(Math.max(1, mountedIndices.size() - 1), mountedIndices); + assertAcked(client().admin().indices().prepareDelete(deletions.toArray(String[]::new))); + deletions.forEach(mountedIndices::remove); + } while (mountedIndices.isEmpty() == false); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + deleteAllMountedIndicesThread.start(); + + deleteAllMountedIndicesThread.join(); + deleteAllSnapshotsThread.join(); + awaitNoMoreSnapshotsDeletions(); + + final GetSnapshotsResponse getSnapshotsResponse = client().admin().cluster().prepareGetSnapshots(repository).get(); + assertTrue(getSnapshotsResponse.getSnapshots().stream().noneMatch(snapshotInfo -> mounts.containsValue(snapshotInfo.snapshotId()))); + } + + private void mountIndexThenExecute(final TriConsumer test) throws Exception { + final String suffix = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); + final String repository = "repository-" + suffix; + final Settings.Builder repositorySettings = randomRepositorySettings(); + createRepository(repository, "mock", repositorySettings); + + final String index = "index-" + suffix; + assertAcked(prepareCreate(index, Settings.builder().put(INDEX_SOFT_DELETES_SETTING.getKey(), true))); + ensureGreen(index); + populateIndex(index, scaledRandomIntBetween(10, 5_000)); + refresh(index); + + final TotalHits totalHits = internalCluster().client().prepareSearch(index).setTrackTotalHits(true).get().getHits().getTotalHits(); + final SnapshotId snapshotId = createSnapshot(repository, "snapshot-" + suffix, List.of(index)).snapshotId(); + + final String restored = mountedIndex(index); + mountSnapshot( + repository, + snapshotId.getName(), + index, + restored, + Settings.builder().put(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, true).build() + ); + assertHitCount(client().prepareSearch(restored).setTrackTotalHits(true).get(), totalHits.value); + test.apply(repository, snapshotId, index); + } + + private void blockPendingDeletionThenExecute(final TriConsumer test) throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + updateRepositoryReadOnly(repository, true); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + test.apply(repository, snapshot, index); + + updateRepositoryReadOnly(repository, false); + awaitNoMoreSnapshotsDeletions(); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + + protected void awaitSnapshotPendingDeletion(final SnapshotId snapshotId) throws Exception { + logger.info("--> wait for snapshot [{}] to be show up as pending deletion in the cluster state", snapshotId); + awaitClusterState(state -> state.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY).contains(snapshotId)); + } + + private void updateRepositoryReadOnly(String repository, boolean readOnly) { + logger.info("--> updating repository [{}] with read-only [{}]", repository, readOnly); + final Settings.Builder repositorySettings = getRepositorySettings(repository); + repositorySettings.put(READONLY_SETTING_KEY, readOnly); + assertAcked( + clusterAdmin().preparePutRepository(repository) + // randomization here is important to have registered repository with _na_ uuid + .setVerify(randomBoolean()) + .setType("mock") + .setSettings(repositorySettings) + ); + } + + private Settings.Builder getRepositorySettings(String repository) { + return Settings.builder().put(client().admin().cluster().prepareGetRepositories(repository).get().repositories().get(0).settings()); + } + + private String mountedIndex(String index) { + return "restored-" + index; + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsRepositoryIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsRepositoryIntegTests.java index 1cdc6d228ac57..3785716d09e82 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsRepositoryIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsRepositoryIntegTests.java @@ -154,6 +154,7 @@ public void testMountIndexWithDeletionOfSnapshotFailsIfNotSingleIndexSnapshot() containsString("snapshot contains [" + nbIndices + "] indices instead of 1.") ) ); + awaitNoMoreSnapshotsDeletions(); } public void testMountIndexWithDifferentDeletionOfSnapshot() throws Exception { @@ -210,6 +211,7 @@ public void testMountIndexWithDifferentDeletionOfSnapshot() throws Exception { assertAcked(client().admin().indices().prepareDelete(mountedAgain)); assertAcked(client().admin().indices().prepareDelete(mounted)); + awaitNoMoreSnapshotsDeletions(); } public void testDeletionOfSnapshotSettingCannotBeUpdated() throws Exception { @@ -256,6 +258,7 @@ public void testDeletionOfSnapshotSettingCannotBeUpdated() throws Exception { ); assertAcked(client().admin().indices().prepareDelete(mounted)); + awaitNoMoreSnapshotsDeletions(); } public void testRestoreSearchableSnapshotIndexConflicts() throws Exception { @@ -270,15 +273,12 @@ public void testRestoreSearchableSnapshotIndexConflicts() throws Exception { assertAcked(client().admin().indices().prepareDelete(indexName)); final String mountedIndex = "mounted-index"; - final boolean deleteSnapshot = randomBoolean(); - final Settings indexSettings = deleteSnapshotIndexSettingsOrNull(deleteSnapshot); + final Settings indexSettings = deleteSnapshotIndexSettingsOrNull(false); logger.info("--> mounting snapshot of index [{}] as [{}] with index settings [{}]", indexName, mountedIndex, indexSettings); mountSnapshot(repository, snapshotOfIndex, indexName, mountedIndex, indexSettings, randomFrom(Storage.values())); assertThat( getDeleteSnapshotIndexSetting(mountedIndex), - indexSettings.hasValue(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION) - ? equalTo(Boolean.toString(deleteSnapshot)) - : nullValue() + indexSettings.hasValue(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION) ? equalTo("false") : nullValue() ); final String snapshotOfMountedIndex = "snapshot-of-mounted-index"; @@ -286,16 +286,10 @@ public void testRestoreSearchableSnapshotIndexConflicts() throws Exception { assertAcked(client().admin().indices().prepareDelete(mountedIndex)); final String mountedIndexAgain = "mounted-index-again"; - final boolean deleteSnapshotAgain = deleteSnapshot == false; - final Settings indexSettingsAgain = deleteSnapshotIndexSettings(deleteSnapshotAgain); + final Settings indexSettingsAgain = deleteSnapshotIndexSettings(true); logger.info("--> mounting snapshot of index [{}] again as [{}] with index settings [{}]", indexName, mountedIndex, indexSettings); mountSnapshot(repository, snapshotOfIndex, indexName, mountedIndexAgain, indexSettingsAgain, randomFrom(Storage.values())); - assertThat( - getDeleteSnapshotIndexSetting(mountedIndexAgain), - indexSettingsAgain.hasValue(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION) - ? equalTo(Boolean.toString(deleteSnapshotAgain)) - : nullValue() - ); + assertThat(getDeleteSnapshotIndexSetting(mountedIndexAgain), equalTo("true")); logger.info("--> restoring snapshot of searchable snapshot index [{}] should be conflicting", mountedIndex); final SnapshotRestoreException exception = expectThrows( @@ -312,12 +306,13 @@ public void testRestoreSearchableSnapshotIndexConflicts() throws Exception { allOf( containsString("cannot mount snapshot [" + repository + '/'), containsString(':' + snapshotOfMountedIndex + "] as index [" + mountedIndex + "] with "), - containsString("[index.store.snapshot.delete_searchable_snapshot: " + deleteSnapshot + "]; another "), + containsString("[index.store.snapshot.delete_searchable_snapshot: false]; another "), containsString("index [" + mountedIndexAgain + '/'), - containsString("is mounted with [index.store.snapshot.delete_searchable_snapshot: " + deleteSnapshotAgain + "].") + containsString("is mounted with [index.store.snapshot.delete_searchable_snapshot: true].") ) ); assertAcked(client().admin().indices().prepareDelete("mounted-*")); + awaitNoMoreSnapshotsDeletions(); } public void testRestoreSearchableSnapshotIndexWithDifferentSettingsConflicts() throws Exception { @@ -399,6 +394,7 @@ public void testRestoreSearchableSnapshotIndexWithDifferentSettingsConflicts() t assertAcked(client().admin().indices().prepareDelete("mounted-*")); assertAcked(client().admin().indices().prepareDelete("restored-with-same-setting-*")); + awaitNoMoreSnapshotsDeletions(); } private static Settings deleteSnapshotIndexSettings(boolean value) { From 579c21f91b1ae299cc56ba26604ecd1561c0ef01 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 10:08:24 +0200 Subject: [PATCH 02/42] rename to SnapshotDeletionsPending --- .../elasticsearch/cluster/ClusterModule.java | 4 +- ...ing.java => SnapshotDeletionsPending.java} | 22 ++++----- .../metadata/MetadataDeleteIndexService.java | 22 ++++----- .../blobstore/BlobStoreRepository.java | 4 +- .../snapshots/RestoreService.java | 4 +- .../snapshots/SnapshotsService.java | 49 ++++++++++--------- .../MetadataDeleteIndexServiceTests.java | 10 ++-- .../BaseSearchableSnapshotsIntegTestCase.java | 7 +-- ...leSnapshotsPendingDeletionsIntegTests.java | 4 +- 9 files changed, 62 insertions(+), 64 deletions(-) rename server/src/main/java/org/elasticsearch/cluster/{SnapshotDeletionsInPending.java => SnapshotDeletionsPending.java} (90%) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index 21514849cf60a..a0b96b0eebf2b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -125,8 +125,8 @@ public static List getNamedWriteables() { SnapshotDeletionsInProgress::readDiffFrom); registerClusterCustom(entries, RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress::new, RepositoryCleanupInProgress::readDiffFrom); - registerClusterCustom(entries, SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending::new, - SnapshotDeletionsInPending::readDiffFrom); + registerClusterCustom(entries, SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending::new, + SnapshotDeletionsPending::readDiffFrom); // Metadata registerMetadataCustom(entries, RepositoriesMetadata.TYPE, RepositoriesMetadata::new, RepositoriesMetadata::readDiffFrom); registerMetadataCustom(entries, IngestMetadata.TYPE, IngestMetadata::new, IngestMetadata::readDiffFrom); diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java similarity index 90% rename from server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java rename to server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 104292423f73c..d5b80b5a69b03 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -32,9 +32,9 @@ /** * Represents snapshots marked as to be deleted and pending deletion. */ -public class SnapshotDeletionsInPending extends AbstractNamedDiffable implements Custom { +public class SnapshotDeletionsPending extends AbstractNamedDiffable implements Custom { - public static final SnapshotDeletionsInPending EMPTY = new SnapshotDeletionsInPending(Collections.emptySortedSet()); + public static final SnapshotDeletionsPending EMPTY = new SnapshotDeletionsPending(Collections.emptySortedSet()); public static final String TYPE = "snapshot_deletions_pending"; public static final int MAX_PENDING_DELETIONS = 500; @@ -44,12 +44,12 @@ public class SnapshotDeletionsInPending extends AbstractNamedDiffable im */ private final SortedSet entries; - private SnapshotDeletionsInPending(SortedSet entries) { + private SnapshotDeletionsPending(SortedSet entries) { this.entries = unmodifiableSortedSet(Objects.requireNonNull(entries)); assert entries.size() <= MAX_PENDING_DELETIONS : entries.size() + " > " + MAX_PENDING_DELETIONS; } - public SnapshotDeletionsInPending(StreamInput in) throws IOException { + public SnapshotDeletionsPending(StreamInput in) throws IOException { this(new TreeSet<>(in.readSet(Entry::new))); } @@ -90,7 +90,7 @@ public Version getMinimalSupportedVersion() { return Version.CURRENT.minimumCompatibilityVersion(); } - public SnapshotDeletionsInPending withRemovedSnapshots(Set snapshotIds) { + public SnapshotDeletionsPending withRemovedSnapshots(Set snapshotIds) { if (snapshotIds == null || snapshotIds.isEmpty()) { return this; } @@ -104,13 +104,13 @@ public SnapshotDeletionsInPending withRemovedSnapshots(Set snapshotI } else if (updatedEntries.isEmpty()) { return EMPTY; } else { - return new SnapshotDeletionsInPending(updatedEntries); + return new SnapshotDeletionsPending(updatedEntries); } } @Override public String toString() { - final StringBuilder builder = new StringBuilder("SnapshotDeletionsInPending["); + final StringBuilder builder = new StringBuilder("SnapshotDeletionsPending["); boolean prepend = true; final Iterator iterator = entries.stream().iterator(); @@ -216,8 +216,8 @@ public static final class Builder { private final SortedSet entries = new TreeSet<>(); private final Consumer consumer; - public Builder(SnapshotDeletionsInPending snapshotDeletionsInPending, Consumer onLimitExceeded) { - entries.addAll(snapshotDeletionsInPending.entries); + public Builder(SnapshotDeletionsPending snapshotDeletionsPending, Consumer onLimitExceeded) { + entries.addAll(snapshotDeletionsPending.entries); this.consumer = onLimitExceeded; } @@ -237,9 +237,9 @@ public Builder add(String repositoryName, String repositoryUuid, SnapshotId snap return this; } - public SnapshotDeletionsInPending build() { + public SnapshotDeletionsPending build() { ensureLimit(); - return entries.isEmpty() == false ? new SnapshotDeletionsInPending(entries) : EMPTY; + return entries.isEmpty() == false ? new SnapshotDeletionsPending(entries) : EMPTY; } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 7d59e00f3067c..50acf8dc9681c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -17,7 +17,7 @@ import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.RestoreInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -149,16 +149,16 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) } // update snapshot(s) marked as to delete - final SnapshotDeletionsInPending deletionsInPending = currentState.custom( - SnapshotDeletionsInPending.TYPE, - SnapshotDeletionsInPending.EMPTY + final SnapshotDeletionsPending deletionsInPending = currentState.custom( + SnapshotDeletionsPending.TYPE, + SnapshotDeletionsPending.EMPTY ); - final SnapshotDeletionsInPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, meta); + final SnapshotDeletionsPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, meta); if (updatedPendingDeletes != deletionsInPending) { if (customBuilder == null) { customBuilder = ImmutableOpenMap.builder(currentState.getCustoms()); } - customBuilder.put(SnapshotDeletionsInPending.TYPE, updatedPendingDeletes); + customBuilder.put(SnapshotDeletionsPending.TYPE, updatedPendingDeletes); } if (customBuilder != null) { builder.customs(customBuilder.build()); @@ -166,14 +166,14 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) return allocationService.reroute(builder.build(), "deleted indices [" + indices + "]"); } - private SnapshotDeletionsInPending updateSnapshotDeletionsPending( - final SnapshotDeletionsInPending pendingDeletions, + private SnapshotDeletionsPending updateSnapshotDeletionsPending( + final SnapshotDeletionsPending pendingDeletions, final Set indicesToDelete, final Metadata metadata ) { if (indicesToDelete.isEmpty() == false) { final long timestamp = Instant.now().toEpochMilli(); - SnapshotDeletionsInPending.Builder builder = null; + SnapshotDeletionsPending.Builder builder = null; boolean changed = false; for (Index indexToDelete : indicesToDelete) { @@ -215,13 +215,13 @@ private SnapshotDeletionsInPending updateSnapshotDeletionsPending( } if (canDeleteSnapshot) { if (builder == null) { - builder = new SnapshotDeletionsInPending.Builder( + builder = new SnapshotDeletionsPending.Builder( pendingDeletions, evicted -> logger.warn( () -> new ParameterizedMessage( "maximum number of snapshots [{}] awaiting deletion has been reached in " + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", - SnapshotDeletionsInPending.MAX_PENDING_DELETIONS, + SnapshotDeletionsPending.MAX_PENDING_DELETIONS, evicted.getSnapshotId(), Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), evicted.getRepositoryName(), 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 5806e0cb5c188..8a6f7bacffa87 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -35,8 +35,8 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RepositoryCleanupInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; @@ -2506,7 +2506,7 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state, } } updatedDeletionsInProgress = changedDeletions ? SnapshotDeletionsInProgress.of(deletionEntries) : null; - final SnapshotDeletionsInPending pendingDeletions = state.custom(SnapshotDeletionsInPending.TYPE); + final SnapshotDeletionsPending pendingDeletions = state.custom(SnapshotDeletionsPending.TYPE); return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress, pendingDeletions); } diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 6c41fb587208f..de657abafbc4e 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -27,8 +27,8 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.RestoreInProgress.ShardRestoreStatus; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.DataStream; @@ -1377,7 +1377,7 @@ private void ensureSnapshotNotDeletedOrPendingDeletion(ClusterState currentState "cannot restore a snapshot while a snapshot deletion is in-progress [" + deletionsInProgress.getEntries().get(0) + "]" ); } - SnapshotDeletionsInPending pendingDeletions = currentState.custom(SnapshotDeletionsInPending.TYPE); + SnapshotDeletionsPending pendingDeletions = currentState.custom(SnapshotDeletionsPending.TYPE); if (pendingDeletions != null && pendingDeletions.contains(snapshot.getSnapshotId())) { throw new ConcurrentSnapshotExecutionException( snapshot, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 8b44cc1f1adde..e2da6fb2631e6 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -35,8 +35,8 @@ import org.elasticsearch.cluster.NotMasterException; import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.RestoreInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus; import org.elasticsearch.cluster.SnapshotsInProgress.ShardState; @@ -484,7 +484,7 @@ public ClusterState execute(ClusterState currentState) { "cannot clone from snapshot that is being deleted" ); } - if (currentState.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY).contains(sourceSnapshotId)) { + if (currentState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY).contains(sourceSnapshotId)) { throw new ConcurrentSnapshotExecutionException( repositoryName, sourceSnapshotId.getName(), @@ -1308,19 +1308,19 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates on the conflicting * situation is resolved. * - * The repository name and uuid information are extracted from the {@link SnapshotDeletionsInPending} entries in order to find the + * The repository name and uuid information are extracted from the {@link SnapshotDeletionsPending} entries in order to find the * repository to execute the snapshot delete request against. If the repo uuid was known at the time the snapshot was added to - * {@link SnapshotDeletionsInPending} we try to find the corresponding repository, or a repository with a missing uuid but the same - * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsInPending}, we try to find a + * {@link SnapshotDeletionsPending} we try to find the corresponding repository, or a repository with a missing uuid but the same + * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsPending}, we try to find a * repository with the same name. * * @param state the current {@link ClusterState} */ private void triggerSnapshotsPendingDeletions(final ClusterState state) { final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - final SnapshotDeletionsInPending snapshotDeletionsInPending = state.custom(SnapshotDeletionsInPending.TYPE); - if (snapshotDeletionsInPending == null - || snapshotDeletionsInPending.isEmpty() + final SnapshotDeletionsPending snapshotDeletionsPending = state.custom(SnapshotDeletionsPending.TYPE); + if (snapshotDeletionsPending == null + || snapshotDeletionsPending.isEmpty() || repositories.repositories().isEmpty() || state.nodes().isLocalNodeElectedMaster() == false || state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { @@ -1333,7 +1333,7 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { // the list of snapshot ids to trigger deletion for, per repository final Map> snapshotsToDelete = new HashMap<>(); - for (SnapshotDeletionsInPending.Entry snapshot : snapshotDeletionsInPending.entries()) { + for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { final SnapshotId snapshotId = snapshot.getSnapshotId(); // early add to avoid doing too much work on successive cluster state updates @@ -1503,17 +1503,18 @@ void removeSnapshot(SnapshotId snapshotId, boolean shouldRetry) { new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsInPending deletionsInPending = currentState.custom( - SnapshotDeletionsInPending.TYPE, - SnapshotDeletionsInPending.EMPTY + final SnapshotDeletionsPending deletionsInPending = currentState.custom( + SnapshotDeletionsPending.TYPE, + SnapshotDeletionsPending.EMPTY + ); + final SnapshotDeletionsPending updatedDeletionsInPending = deletionsInPending.withRemovedSnapshots( + missingSnapshots ); - final SnapshotDeletionsInPending updatedDeletionsInPending = deletionsInPending - .withRemovedSnapshots(missingSnapshots); if (deletionsInPending == updatedDeletionsInPending) { return currentState; } return ClusterState.builder(currentState) - .putCustom(SnapshotDeletionsInPending.TYPE, updatedDeletionsInPending) + .putCustom(SnapshotDeletionsPending.TYPE, updatedDeletionsInPending) .build(); } @@ -2779,7 +2780,7 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres @Nullable @Override - protected SnapshotDeletionsInPending filterPendingDeletions(@Nullable SnapshotDeletionsInPending pendingDeletions) { + protected SnapshotDeletionsPending filterPendingDeletions(@Nullable SnapshotDeletionsPending pendingDeletions) { return pendingDeletions != null ? pendingDeletions.withRemovedSnapshots(Sets.newHashSet(deleteEntry.getSnapshots())) : null; @@ -2870,7 +2871,7 @@ public ClusterState execute(ClusterState currentState) { return currentState; } final SnapshotDeletionsInProgress newDeletions = filterDeletions(updatedDeletions); - SnapshotDeletionsInPending newPendingDeletions = filterPendingDeletions(currentState.custom(SnapshotDeletionsInPending.TYPE)); + SnapshotDeletionsPending newPendingDeletions = filterPendingDeletions(currentState.custom(SnapshotDeletionsPending.TYPE)); final Tuple> res = readyDeletions( updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions, newPendingDeletions) ); @@ -2889,7 +2890,7 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres return deletions; } - protected SnapshotDeletionsInPending filterPendingDeletions(SnapshotDeletionsInPending pendingDeletions) { + protected SnapshotDeletionsPending filterPendingDeletions(SnapshotDeletionsPending pendingDeletions) { return pendingDeletions; } @@ -3067,21 +3068,21 @@ private void markShardReassigned(RepositoryShardId shardId, Set 0) { assertThat(pendingDeletions.isEmpty(), equalTo(true)); @@ -263,7 +263,7 @@ public void testDeleteMultipleIndicesWithSnapshotDeletion() { assertThat(clusterState.routingTable().index(deletedIndex), nullValue()); } - pendingDeletions = clusterState.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY); + pendingDeletions = clusterState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); } assertThat(pendingDeletions.isEmpty(), equalTo(false)); diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java index 30a64eb693225..073798cee2558 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java @@ -19,8 +19,8 @@ import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.index.IndexRequestBuilder; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; @@ -345,10 +345,7 @@ protected void awaitNoMoreSnapshotsDeletions() throws Exception { if (deletions.hasDeletionsInProgress()) { return false; } - final SnapshotDeletionsInPending pendingDeletions = state.custom( - SnapshotDeletionsInPending.TYPE, - SnapshotDeletionsInPending.EMPTY - ); + final SnapshotDeletionsPending pendingDeletions = state.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); if (pendingDeletions.isEmpty() == false) { return false; } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 081eec755c26f..ab85778e38cab 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -15,7 +15,7 @@ import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.RestoreInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsInPending; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Strings; import org.elasticsearch.common.TriConsumer; @@ -403,7 +403,7 @@ private void blockPendingDeletionThenExecute(final TriConsumer wait for snapshot [{}] to be show up as pending deletion in the cluster state", snapshotId); - awaitClusterState(state -> state.custom(SnapshotDeletionsInPending.TYPE, SnapshotDeletionsInPending.EMPTY).contains(snapshotId)); + awaitClusterState(state -> state.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY).contains(snapshotId)); } private void updateRepositoryReadOnly(String repository, boolean readOnly) { From d15ac48d5d2043171bcbcf3f1fb0c179969cb0bb Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 11:03:19 +0200 Subject: [PATCH 03/42] MAX_PENDING_DELETIONS_SETTING --- .../cluster/SnapshotDeletionsPending.java | 42 +++++++++++++++---- .../metadata/MetadataDeleteIndexService.java | 5 ++- 2 files changed, 38 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index d5b80b5a69b03..bde5a908411e8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -13,6 +13,8 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; @@ -31,13 +33,39 @@ /** * Represents snapshots marked as to be deleted and pending deletion. + * + * Snapshots pending deletion are added to the cluster state when searchable snapshots indices with a specific setting are deleted (see + * MetadataDeleteIndexService#updateSnapshotDeletionsPending()). Because deleting snapshots requires a consistent view of the repository + * they belong to it is not possible to delete searchable snapshots indices and their backing snapshots in the same cluster state update. + * + * Hence we keep in cluster state the snapshot that should be deleted from repositories. To be able to delete them we capture the snapshot + * id, the snapshot name, the repository name and the repository id (if it exists) once, along with the time at which the snapshot was added + * to the pending deletion, in a {@link SnapshotDeletionsPending} entry. + * + * + * When cluster state is updated with such entries the {@link org.elasticsearch.snapshots.SnapshotsService} executes corresponding snapshot + * delete requests to effectively delete the snapshot from the repository. It is possible that the deletion of a snapshot failed for various + * reason (ex: conflicting snapshot operation, repository removed etc). In such cases the snapshot pending deletion is kept in the cluster + * state and the deletion will be retried on the next cluster state update. To avoid too many snapshots pending deletion stored in cluster + * state the number is limited to 500 and configurable through the {@link #MAX_PENDING_DELETIONS_SETTING} setting. */ public class SnapshotDeletionsPending extends AbstractNamedDiffable implements Custom { public static final SnapshotDeletionsPending EMPTY = new SnapshotDeletionsPending(Collections.emptySortedSet()); public static final String TYPE = "snapshot_deletions_pending"; - public static final int MAX_PENDING_DELETIONS = 500; + /** + * Setting for the maximum number of snapshots pending deletion allowed in the cluster state. + *

+ * This setting is here to prevent the cluster to grow too large. In the case that the number of snapshots pending deletion exceeds + * the value of this setting the oldest entries are removed from the cluster state. Snapshots that are discarded are removed before + * they can be deleted from their repository and are therefore considered as "leaking" and should be logged as such as warnings. + */ + public static final Setting MAX_PENDING_DELETIONS_SETTING = Setting.intSetting( + "cluster.snapshot.snapshot_deletions_pending.size", + 500, + Setting.Property.NodeScope + ); /** * A list of snapshots to delete, sorted by creation time @@ -46,7 +74,6 @@ public class SnapshotDeletionsPending extends AbstractNamedDiffable impl private SnapshotDeletionsPending(SortedSet entries) { this.entries = unmodifiableSortedSet(Objects.requireNonNull(entries)); - assert entries.size() <= MAX_PENDING_DELETIONS : entries.size() + " > " + MAX_PENDING_DELETIONS; } public SnapshotDeletionsPending(StreamInput in) throws IOException { @@ -221,8 +248,8 @@ public Builder(SnapshotDeletionsPending snapshotDeletionsPending, Consumer= MAX_PENDING_DELETIONS) { + private void ensureLimit(final int maxPendingDeletions) { + while (entries.size() >= maxPendingDeletions) { final Entry removed = entries.last(); entries.remove(removed); if (consumer != null) { @@ -232,13 +259,14 @@ private void ensureLimit() { } public Builder add(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long creationTime) { - ensureLimit(); entries.add(new Entry(repositoryName, repositoryUuid, snapshotId, creationTime)); return this; } - public SnapshotDeletionsPending build() { - ensureLimit(); + public SnapshotDeletionsPending build(Settings settings) { + final int maxPendingDeletions = MAX_PENDING_DELETIONS_SETTING.get(settings); + ensureLimit(maxPendingDeletions); + assert entries.size() <= maxPendingDeletions : entries.size() + " > " + maxPendingDeletions; return entries.isEmpty() == false ? new SnapshotDeletionsPending(entries) : EMPTY; } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 50acf8dc9681c..4676e9c9a949e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -215,13 +215,14 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( } if (canDeleteSnapshot) { if (builder == null) { + final int maxPendingDeletions = SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING.get(settings); builder = new SnapshotDeletionsPending.Builder( pendingDeletions, evicted -> logger.warn( () -> new ParameterizedMessage( "maximum number of snapshots [{}] awaiting deletion has been reached in " + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", - SnapshotDeletionsPending.MAX_PENDING_DELETIONS, + maxPendingDeletions, evicted.getSnapshotId(), Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), evicted.getRepositoryName(), @@ -240,7 +241,7 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( } } if (changed) { - return builder.build(); + return builder.build(settings); } } return pendingDeletions; From 7f9c32c424654eb5070d175975a1c3a8dde7f550 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 11:37:38 +0200 Subject: [PATCH 04/42] list --- .../cluster/SnapshotDeletionsPending.java | 38 ++++++++++--------- 1 file changed, 20 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index bde5a908411e8..036d60f54dc3d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -20,16 +20,15 @@ import org.elasticsearch.xcontent.XContentBuilder; import java.io.IOException; -import java.util.Collections; +import java.util.ArrayList; import java.util.Comparator; import java.util.Iterator; +import java.util.List; import java.util.Objects; import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; import java.util.function.Consumer; -import static java.util.Collections.unmodifiableSortedSet; +import static java.util.Collections.unmodifiableList; /** * Represents snapshots marked as to be deleted and pending deletion. @@ -51,7 +50,7 @@ */ public class SnapshotDeletionsPending extends AbstractNamedDiffable implements Custom { - public static final SnapshotDeletionsPending EMPTY = new SnapshotDeletionsPending(Collections.emptySortedSet()); + public static final SnapshotDeletionsPending EMPTY = new SnapshotDeletionsPending(List.of()); public static final String TYPE = "snapshot_deletions_pending"; /** @@ -70,19 +69,19 @@ public class SnapshotDeletionsPending extends AbstractNamedDiffable impl /** * A list of snapshots to delete, sorted by creation time */ - private final SortedSet entries; + private final List entries; - private SnapshotDeletionsPending(SortedSet entries) { - this.entries = unmodifiableSortedSet(Objects.requireNonNull(entries)); + private SnapshotDeletionsPending(List entries) { + this.entries = unmodifiableList(Objects.requireNonNull(entries)); } public SnapshotDeletionsPending(StreamInput in) throws IOException { - this(new TreeSet<>(in.readSet(Entry::new))); + this(in.readList(Entry::new)); } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeCollection(entries); + out.writeList(entries); } @Override @@ -98,7 +97,7 @@ public boolean contains(SnapshotId snapshotId) { return entries.stream().anyMatch(entry -> Objects.equals(entry.getSnapshotId(), snapshotId)); } - public SortedSet entries() { + public List entries() { return entries; } @@ -122,9 +121,13 @@ public SnapshotDeletionsPending withRemovedSnapshots(Set snapshotIds return this; } boolean changed = false; - final SortedSet updatedEntries = new TreeSet<>(entries); - if (updatedEntries.removeIf(entry -> snapshotIds.contains(entry.getSnapshotId()))) { - changed = true; + final List updatedEntries = new ArrayList<>(); + for (Entry entry : entries) { + if (snapshotIds.contains(entry.snapshotId)) { + changed = true; + continue; + } + updatedEntries.add(entry); } if (changed == false) { return this; @@ -240,18 +243,17 @@ public int compareTo(final Entry other) { public static final class Builder { - private final SortedSet entries = new TreeSet<>(); + private final List entries; private final Consumer consumer; public Builder(SnapshotDeletionsPending snapshotDeletionsPending, Consumer onLimitExceeded) { - entries.addAll(snapshotDeletionsPending.entries); + this.entries = new ArrayList<>(snapshotDeletionsPending.entries); this.consumer = onLimitExceeded; } private void ensureLimit(final int maxPendingDeletions) { while (entries.size() >= maxPendingDeletions) { - final Entry removed = entries.last(); - entries.remove(removed); + final Entry removed = entries.remove(0); if (consumer != null) { consumer.accept(removed); } From 88c0a0b0852686ec7e55dbb48f93096be4569fcc Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 11:43:29 +0200 Subject: [PATCH 05/42] tostring --- .../cluster/SnapshotDeletionsPending.java | 19 +++++-------------- .../snapshots/SnapshotsService.java | 5 ++--- 2 files changed, 7 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 036d60f54dc3d..315744314b7ad 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -21,11 +21,9 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Objects; -import java.util.Set; import java.util.function.Consumer; import static java.util.Collections.unmodifiableList; @@ -116,7 +114,7 @@ public Version getMinimalSupportedVersion() { return Version.CURRENT.minimumCompatibilityVersion(); } - public SnapshotDeletionsPending withRemovedSnapshots(Set snapshotIds) { + public SnapshotDeletionsPending withRemovedSnapshots(List snapshotIds) { if (snapshotIds == null || snapshotIds.isEmpty()) { return this; } @@ -142,23 +140,19 @@ public SnapshotDeletionsPending withRemovedSnapshots(Set snapshotIds public String toString() { final StringBuilder builder = new StringBuilder("SnapshotDeletionsPending["); boolean prepend = true; - final Iterator iterator = entries.stream().iterator(); while (iterator.hasNext()) { if (prepend == false) { builder.append(','); } - final Entry entry = iterator.next(); - builder.append('[').append(entry.repositoryName).append('/').append(entry.repositoryUuid).append(']'); - builder.append('[').append(entry.snapshotId).append(',').append(entry.creationTime).append(']'); - builder.append('\n'); + builder.append(iterator.next()); prepend = false; } builder.append(']'); return builder.toString(); } - public static class Entry implements Writeable, ToXContentObject, Comparable { + public static class Entry implements Writeable, ToXContentObject { private final String repositoryName; private final String repositoryUuid; @@ -233,11 +227,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } @Override - public int compareTo(final Entry other) { - return Comparator.comparingLong(Entry::getCreationTime) - .reversed() - .thenComparing(Entry::getSnapshotId) - .compare(this, other); + public String toString() { + return '[' + repositoryName + '/' + repositoryUuid + ',' + snapshotId + ',' + creationTime + ']'; } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index e2da6fb2631e6..475c85824de18 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -69,7 +69,6 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; -import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; @@ -1438,7 +1437,7 @@ private class SnapshotsToDeleteRunnable extends AbstractRunnable { @Override protected void doRun() throws Exception { - final Set missingSnapshots = ConcurrentCollections.newConcurrentSet(); + final List missingSnapshots = new CopyOnWriteArrayList<>(); final CountDown countDown = new CountDown(snapshotIdsToDelete.size()); for (SnapshotId snapshotId : snapshotIdsToDelete) { @@ -2782,7 +2781,7 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres @Override protected SnapshotDeletionsPending filterPendingDeletions(@Nullable SnapshotDeletionsPending pendingDeletions) { return pendingDeletions != null - ? pendingDeletions.withRemovedSnapshots(Sets.newHashSet(deleteEntry.getSnapshots())) + ? pendingDeletions.withRemovedSnapshots(deleteEntry.getSnapshots()) : null; } From 9151c28dceed18dd76a488681db1227dd6069c9f Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 12:40:56 +0200 Subject: [PATCH 06/42] remove if is empty --- .../metadata/MetadataDeleteIndexService.java | 128 +++++++++--------- 1 file changed, 63 insertions(+), 65 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 4676e9c9a949e..2f093980fcfcc 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -171,78 +171,76 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( final Set indicesToDelete, final Metadata metadata ) { - if (indicesToDelete.isEmpty() == false) { - final long timestamp = Instant.now().toEpochMilli(); - SnapshotDeletionsPending.Builder builder = null; - boolean changed = false; - - for (Index indexToDelete : indicesToDelete) { - final Settings indexSettings = metadata.getIndexSafe(indexToDelete).getSettings(); - if (SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings) == false) { - continue; // not a searchable snapshot index + final long timestamp = Instant.now().toEpochMilli(); + SnapshotDeletionsPending.Builder builder = null; + boolean changed = false; + + for (Index indexToDelete : indicesToDelete) { + final Settings indexSettings = metadata.getIndexSafe(indexToDelete).getSettings(); + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings) == false) { + continue; // not a searchable snapshot index + } + if (indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) == false) { + continue; // do not delete the snapshot when this searchable snapshot index is deleted + } + final SnapshotId snapshotId = new SnapshotId( + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY), + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY) + ); + boolean canDeleteSnapshot = true; + for (IndexMetadata other : metadata) { + if (indexToDelete.equals(other.getIndex())) { + continue; // do not check against itself } - if (indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) == false) { - continue; // do not delete the snapshot when this searchable snapshot index is deleted + final Settings otherSettings = other.getSettings(); + if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { + continue; // other index is not a searchable snapshot index, skip } - final SnapshotId snapshotId = new SnapshotId( - indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY), - indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY) - ); - boolean canDeleteSnapshot = true; - for (IndexMetadata other : metadata) { - if (indexToDelete.equals(other.getIndex())) { - continue; // do not check against itself - } - final Settings otherSettings = other.getSettings(); - if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { - continue; // other index is not a searchable snapshot index, skip - } - final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); - if (Objects.equals(snapshotId.getUUID(), otherSnapshotUuid) == false) { - continue; // other index is backed by a different snapshot, skip - } - assert otherSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) : other; - if (indicesToDelete.contains(other.getIndex())) { - continue; // other index is going to be deleted as part of the same cluster state update - } - logger.debug( - "snapshot [{}] cannot be marked as to delete, another index [{}] is using the snapshot", - snapshotId, - other.getIndex() - ); - canDeleteSnapshot = false; // another index is using the same snapshot, do not delete the snapshot - break; + final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); + if (Objects.equals(snapshotId.getUUID(), otherSnapshotUuid) == false) { + continue; // other index is backed by a different snapshot, skip + } + assert otherSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) : other; + if (indicesToDelete.contains(other.getIndex())) { + continue; // other index is going to be deleted as part of the same cluster state update } - if (canDeleteSnapshot) { - if (builder == null) { - final int maxPendingDeletions = SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING.get(settings); - builder = new SnapshotDeletionsPending.Builder( - pendingDeletions, - evicted -> logger.warn( - () -> new ParameterizedMessage( - "maximum number of snapshots [{}] awaiting deletion has been reached in " - + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", - maxPendingDeletions, - evicted.getSnapshotId(), - Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), - evicted.getRepositoryName(), - evicted.getRepositoryUuid() - ) + logger.debug( + "snapshot [{}] cannot be marked as to delete, another index [{}] is using the snapshot", + snapshotId, + other.getIndex() + ); + canDeleteSnapshot = false; // another index is using the same snapshot, do not delete the snapshot + break; + } + if (canDeleteSnapshot) { + if (builder == null) { + final int maxPendingDeletions = SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING.get(settings); + builder = new SnapshotDeletionsPending.Builder( + pendingDeletions, + evicted -> logger.warn( + () -> new ParameterizedMessage( + "maximum number of snapshots [{}] awaiting deletion has been reached in " + + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", + maxPendingDeletions, + evicted.getSnapshotId(), + Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), + evicted.getRepositoryName(), + evicted.getRepositoryUuid() ) - ); - } - builder.add( - indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY), - indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, RepositoryData.MISSING_UUID), - snapshotId, - timestamp + ) ); - changed = true; } + builder.add( + indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY), + indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, RepositoryData.MISSING_UUID), + snapshotId, + timestamp + ); + changed = true; } - if (changed) { - return builder.build(settings); - } + } + if (changed) { + return builder.build(settings); } return pendingDeletions; } From 12835bb94bffda45adf988c99ae4914fba2d51ed Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 13:09:48 +0200 Subject: [PATCH 07/42] remove if --- .../snapshots/SnapshotsService.java | 148 ++++++++---------- 1 file changed, 68 insertions(+), 80 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 475c85824de18..bd01c66184908 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1334,84 +1334,74 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { final SnapshotId snapshotId = snapshot.getSnapshotId(); + boolean triggered = false; + + if (currentRestores.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); + continue; + } else if (currentClones.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); + continue; + } else if (currentDeletions.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is already queued", snapshotId); + continue; + } + + Optional optionalRepository; + if (RepositoryData.MISSING_UUID.equals(snapshot.getRepositoryUuid()) == false) { + // the snapshot waiting to be deleted references a repository with a known uuid, + // let's try to find this repository among the existing ones first + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), snapshot.getRepositoryUuid())) + .findFirst(); + if (optionalRepository.isEmpty()) { + // there is no existing repository matching the uuid, + // let's try to find the repository by name among the existing ones that have no uuid + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) + .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) + .findFirst(); + } + } else { + // the snapshot waiting to be deleted does not references a repository with a known uuid, + // let's try to find the repository by name among the existing ones, in the hope that + // the snapshot will be found there. + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) + .findFirst(); + } - // early add to avoid doing too much work on successive cluster state updates - if (ongoingSnapshotsDeletions.add(snapshotId)) { - boolean triggered = false; - try { - if (currentRestores.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); - continue; - } else if (currentClones.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); - continue; - } else if (currentDeletions.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is already queued", snapshotId); - continue; - } - - Optional optionalRepository; - if (RepositoryData.MISSING_UUID.equals(snapshot.getRepositoryUuid()) == false) { - // the snapshot waiting to be deleted references a repository with a known uuid, - // let's try to find this repository among the existing ones first - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.uuid(), snapshot.getRepositoryUuid())) - .findFirst(); - if (optionalRepository.isEmpty()) { - // there is no existing repository matching the uuid, - // let's try to find the repository by name among the existing ones that have no uuid - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) - .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) - .findFirst(); - } - } else { - // the snapshot waiting to be deleted does not references a repository with a known uuid, - // let's try to find the repository by name among the existing ones, in the hope that - // the snapshot will be found there. - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) - .findFirst(); - } - - if (optionalRepository.isEmpty()) { - logger.debug( - "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", - snapshot.getRepositoryName(), - snapshot.getRepositoryUuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) - ); - continue; - } - - final RepositoryMetadata repository = optionalRepository.get(); - if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { - logger.debug( - "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", - repository.name(), - repository.uuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) - ); - continue; - } + if (optionalRepository.isEmpty()) { + logger.debug( + "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + ); + continue; + } - // should we add some throttling to not always retry - final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); - assert ongoingSnapshotsDeletions.contains(snapshotId) : snapshotId; - assert added : snapshotId; + final RepositoryMetadata repository = optionalRepository.get(); + if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { + logger.debug( + "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", + repository.name(), + repository.uuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + ); + continue; + } - logger.trace("triggering snapshot deletion for [{}]", snapshotId); - triggered = true; - } finally { - if (triggered == false) { - ongoingSnapshotsDeletions.remove(snapshotId); - } - } + // should we add some throttling to not always retry? + if (ongoingSnapshotsDeletions.add(snapshotId)) { + logger.trace("triggering snapshot deletion for [{}]", snapshotId); + final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); + assert added : snapshotId; } } snapshotsToDelete.forEach( @@ -1519,12 +1509,12 @@ public ClusterState execute(ClusterState currentState) { @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - ongoingSnapshotsDeletions.removeAll(missingSnapshots); + missingSnapshots.forEach(ongoingSnapshotsDeletions::remove); } @Override public void onFailure(String source, Exception e) { - ongoingSnapshotsDeletions.removeAll(missingSnapshots); + missingSnapshots.forEach(ongoingSnapshotsDeletions::remove); } } ); @@ -2780,9 +2770,7 @@ protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgres @Nullable @Override protected SnapshotDeletionsPending filterPendingDeletions(@Nullable SnapshotDeletionsPending pendingDeletions) { - return pendingDeletions != null - ? pendingDeletions.withRemovedSnapshots(deleteEntry.getSnapshots()) - : null; + return pendingDeletions != null ? pendingDeletions.withRemovedSnapshots(deleteEntry.getSnapshots()) : null; } @Override From 7be719b532eb9ce16c28b13b5486d1257f423a7c Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 18 Oct 2021 13:31:17 +0200 Subject: [PATCH 08/42] remove triggered --- .../main/java/org/elasticsearch/snapshots/SnapshotsService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index bd01c66184908..84f18fd18a67c 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1334,7 +1334,6 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { final SnapshotId snapshotId = snapshot.getSnapshotId(); - boolean triggered = false; if (currentRestores.contains(snapshotId)) { logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); From 086889d94fb5bcdc26ce4ae6062d2f07b49119dd Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 9 Nov 2021 12:20:46 +0100 Subject: [PATCH 09/42] format conflicting files --- .../elasticsearch/cluster/ClusterModule.java | 192 +++++++++++++----- .../metadata/MetadataDeleteIndexService.java | 30 ++- .../MetadataDeleteIndexServiceTests.java | 130 +++++++----- 3 files changed, 246 insertions(+), 106 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java index a0b96b0eebf2b..fd731347247db 100644 --- a/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/elasticsearch/cluster/ClusterModule.java @@ -90,8 +90,12 @@ public class ClusterModule extends AbstractModule { public static final String BALANCED_ALLOCATOR = "balanced"; // default - public static final Setting SHARDS_ALLOCATOR_TYPE_SETTING = - new Setting<>("cluster.routing.allocation.type", BALANCED_ALLOCATOR, Function.identity(), Property.NodeScope); + public static final Setting SHARDS_ALLOCATOR_TYPE_SETTING = new Setting<>( + "cluster.routing.allocation.type", + BALANCED_ALLOCATOR, + Function.identity(), + Property.NodeScope + ); private final ClusterService clusterService; private final IndexNameExpressionResolver indexNameExpressionResolver; @@ -103,9 +107,15 @@ public class ClusterModule extends AbstractModule { final Collection deciderList; final ShardsAllocator shardsAllocator; - public ClusterModule(Settings settings, ClusterService clusterService, List clusterPlugins, - ClusterInfoService clusterInfoService, SnapshotsInfoService snapshotsInfoService, ThreadContext threadContext, - SystemIndices systemIndices) { + public ClusterModule( + Settings settings, + ClusterService clusterService, + List clusterPlugins, + ClusterInfoService clusterInfoService, + SnapshotsInfoService snapshotsInfoService, + ThreadContext threadContext, + SystemIndices systemIndices + ) { this.clusterPlugins = clusterPlugins; this.deciderList = createAllocationDeciders(settings, clusterService.getClusterSettings(), clusterPlugins); this.allocationDeciders = new AllocationDeciders(deciderList); @@ -121,23 +131,47 @@ public static List getNamedWriteables() { // Cluster State registerClusterCustom(entries, SnapshotsInProgress.TYPE, SnapshotsInProgress::new, SnapshotsInProgress::readDiffFrom); registerClusterCustom(entries, RestoreInProgress.TYPE, RestoreInProgress::new, RestoreInProgress::readDiffFrom); - registerClusterCustom(entries, SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress::new, - SnapshotDeletionsInProgress::readDiffFrom); - registerClusterCustom(entries, RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress::new, - RepositoryCleanupInProgress::readDiffFrom); - registerClusterCustom(entries, SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending::new, - SnapshotDeletionsPending::readDiffFrom); + registerClusterCustom( + entries, + SnapshotDeletionsInProgress.TYPE, + SnapshotDeletionsInProgress::new, + SnapshotDeletionsInProgress::readDiffFrom + ); + registerClusterCustom( + entries, + RepositoryCleanupInProgress.TYPE, + RepositoryCleanupInProgress::new, + RepositoryCleanupInProgress::readDiffFrom + ); + registerClusterCustom( + entries, + SnapshotDeletionsPending.TYPE, + SnapshotDeletionsPending::new, + SnapshotDeletionsPending::readDiffFrom + ); // Metadata registerMetadataCustom(entries, RepositoriesMetadata.TYPE, RepositoriesMetadata::new, RepositoriesMetadata::readDiffFrom); registerMetadataCustom(entries, IngestMetadata.TYPE, IngestMetadata::new, IngestMetadata::readDiffFrom); registerMetadataCustom(entries, ScriptMetadata.TYPE, ScriptMetadata::new, ScriptMetadata::readDiffFrom); registerMetadataCustom(entries, IndexGraveyard.TYPE, IndexGraveyard::new, IndexGraveyard::readDiffFrom); - registerMetadataCustom(entries, PersistentTasksCustomMetadata.TYPE, PersistentTasksCustomMetadata::new, - PersistentTasksCustomMetadata::readDiffFrom); - registerMetadataCustom(entries, ComponentTemplateMetadata.TYPE, ComponentTemplateMetadata::new, - ComponentTemplateMetadata::readDiffFrom); - registerMetadataCustom(entries, ComposableIndexTemplateMetadata.TYPE, ComposableIndexTemplateMetadata::new, - ComposableIndexTemplateMetadata::readDiffFrom); + registerMetadataCustom( + entries, + PersistentTasksCustomMetadata.TYPE, + PersistentTasksCustomMetadata::new, + PersistentTasksCustomMetadata::readDiffFrom + ); + registerMetadataCustom( + entries, + ComponentTemplateMetadata.TYPE, + ComponentTemplateMetadata::new, + ComponentTemplateMetadata::readDiffFrom + ); + registerMetadataCustom( + entries, + ComposableIndexTemplateMetadata.TYPE, + ComposableIndexTemplateMetadata::new, + ComposableIndexTemplateMetadata::readDiffFrom + ); registerMetadataCustom(entries, DataStreamMetadata.TYPE, DataStreamMetadata::new, DataStreamMetadata::readDiffFrom); registerMetadataCustom(entries, NodesShutdownMetadata.TYPE, NodesShutdownMetadata::new, NodesShutdownMetadata::readDiffFrom); @@ -149,39 +183,85 @@ public static List getNamedWriteables() { public static List getNamedXWriteables() { List entries = new ArrayList<>(); // Metadata - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(RepositoriesMetadata.TYPE), - RepositoriesMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(IngestMetadata.TYPE), - IngestMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(ScriptMetadata.TYPE), - ScriptMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(IndexGraveyard.TYPE), - IndexGraveyard::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(PersistentTasksCustomMetadata.TYPE), - PersistentTasksCustomMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(ComponentTemplateMetadata.TYPE), - ComponentTemplateMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(ComposableIndexTemplateMetadata.TYPE), - ComposableIndexTemplateMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(DataStreamMetadata.TYPE), - DataStreamMetadata::fromXContent)); - entries.add(new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(NodesShutdownMetadata.TYPE), - NodesShutdownMetadata::fromXContent)); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(RepositoriesMetadata.TYPE), + RepositoriesMetadata::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(IngestMetadata.TYPE), IngestMetadata::fromXContent) + ); + entries.add( + new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(ScriptMetadata.TYPE), ScriptMetadata::fromXContent) + ); + entries.add( + new NamedXContentRegistry.Entry(Metadata.Custom.class, new ParseField(IndexGraveyard.TYPE), IndexGraveyard::fromXContent) + ); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(PersistentTasksCustomMetadata.TYPE), + PersistentTasksCustomMetadata::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(ComponentTemplateMetadata.TYPE), + ComponentTemplateMetadata::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(ComposableIndexTemplateMetadata.TYPE), + ComposableIndexTemplateMetadata::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(DataStreamMetadata.TYPE), + DataStreamMetadata::fromXContent + ) + ); + entries.add( + new NamedXContentRegistry.Entry( + Metadata.Custom.class, + new ParseField(NodesShutdownMetadata.TYPE), + NodesShutdownMetadata::fromXContent + ) + ); return entries; } - private static void registerClusterCustom(List entries, String name, Reader reader, - Reader> diffReader) { + private static void registerClusterCustom( + List entries, + String name, + Reader reader, + Reader> diffReader + ) { registerCustom(entries, ClusterState.Custom.class, name, reader, diffReader); } - private static void registerMetadataCustom(List entries, String name, Reader reader, - Reader> diffReader) { + private static void registerMetadataCustom( + List entries, + String name, + Reader reader, + Reader> diffReader + ) { registerCustom(entries, Metadata.Custom.class, name, reader, diffReader); } - private static void registerCustom(List entries, Class category, String name, - Reader reader, Reader> diffReader) { + private static void registerCustom( + List entries, + Class category, + String name, + Reader reader, + Reader> diffReader + ) { entries.add(new Entry(category, name, reader)); entries.add(new Entry(NamedDiff.class, name, diffReader)); } @@ -192,8 +272,11 @@ public IndexNameExpressionResolver getIndexNameExpressionResolver() { // TODO: this is public so allocation benchmark can access the default deciders...can we do that in another way? /** Return a new {@link AllocationDecider} instance with builtin deciders as well as those from plugins. */ - public static Collection createAllocationDeciders(Settings settings, ClusterSettings clusterSettings, - List clusterPlugins) { + public static Collection createAllocationDeciders( + Settings settings, + ClusterSettings clusterSettings, + List clusterPlugins + ) { // collect deciders by class so that we can detect duplicates Map, AllocationDecider> deciders = new LinkedHashMap<>(); addAllocationDecider(deciders, new MaxRetryAllocationDecider()); @@ -229,8 +312,11 @@ private static void addAllocationDecider(Map, AllocationDecider> decide } } - private static ShardsAllocator createShardsAllocator(Settings settings, ClusterSettings clusterSettings, - List clusterPlugins) { + private static ShardsAllocator createShardsAllocator( + Settings settings, + ClusterSettings clusterSettings, + List clusterPlugins + ) { Map> allocators = new HashMap<>(); allocators.put(BALANCED_ALLOCATOR, () -> new BalancedShardsAllocator(settings, clusterSettings)); @@ -246,8 +332,7 @@ private static ShardsAllocator createShardsAllocator(Settings settings, ClusterS if (allocatorSupplier == null) { throw new IllegalArgumentException("Unknown ShardsAllocator [" + allocatorName + "]"); } - return Objects.requireNonNull(allocatorSupplier.get(), - "ShardsAllocator factory for [" + allocatorName + "] returned null"); + return Objects.requireNonNull(allocatorSupplier.get(), "ShardsAllocator factory for [" + allocatorName + "] returned null"); } public AllocationService getAllocationService() { @@ -284,12 +369,17 @@ public void setExistingShardsAllocators(GatewayAllocator gatewayAllocator) { existingShardsAllocators.put(GatewayAllocator.ALLOCATOR_NAME, gatewayAllocator); for (ClusterPlugin clusterPlugin : clusterPlugins) { - for (Map.Entry existingShardsAllocatorEntry - : clusterPlugin.getExistingShardsAllocators().entrySet()) { + for (Map.Entry existingShardsAllocatorEntry : clusterPlugin.getExistingShardsAllocators() + .entrySet()) { final String allocatorName = existingShardsAllocatorEntry.getKey(); if (existingShardsAllocators.put(allocatorName, existingShardsAllocatorEntry.getValue()) != null) { - throw new IllegalArgumentException("ExistingShardsAllocator [" + allocatorName + "] from [" + - clusterPlugin.getClass().getName() + "] was already defined"); + throw new IllegalArgumentException( + "ExistingShardsAllocator [" + + allocatorName + + "] from [" + + clusterPlugin.getClass().getName() + + "] was already defined" + ); } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 2bee49f7bfa5c..e9335130340d1 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -73,13 +73,15 @@ public void deleteIndices(final DeleteIndexClusterStateUpdateRequest request, fi throw new IllegalArgumentException("Index name is required"); } - clusterService.submitStateUpdateTask("delete-index " + Arrays.toString(request.indices()), + clusterService.submitStateUpdateTask( + "delete-index " + Arrays.toString(request.indices()), new AckedClusterStateUpdateTask(Priority.URGENT, request, listener) { @Override public ClusterState execute(final ClusterState currentState) { return deleteIndices(currentState, Sets.newHashSet(request.indices())); } - }); + } + ); } /** @@ -94,8 +96,13 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) IndexAbstraction.DataStream parent = meta.getIndicesLookup().get(im.getIndex().getName()).getParentDataStream(); if (parent != null) { if (parent.getWriteIndex().equals(im.getIndex())) { - throw new IllegalArgumentException("index [" + index.getName() + "] is the write index for data stream [" + - parent.getName() + "] and cannot be deleted"); + throw new IllegalArgumentException( + "index [" + + index.getName() + + "] is the write index for data stream [" + + parent.getName() + + "] and cannot be deleted" + ); } else { backingIndices.put(index, parent.getDataStream()); } @@ -106,8 +113,11 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) // Check if index deletion conflicts with any running snapshots Set snapshottingIndices = SnapshotsService.snapshottingIndices(currentState, indicesToDelete); if (snapshottingIndices.isEmpty() == false) { - throw new SnapshotInProgressException("Cannot delete indices that are being snapshotted: " + snapshottingIndices + - ". Try again after snapshot finishes or cancel the currently running snapshot."); + throw new SnapshotInProgressException( + "Cannot delete indices that are being snapshotted: " + + snapshottingIndices + + ". Try again after snapshot finishes or cancel the currently running snapshot." + ); } RoutingTable.Builder routingTableBuilder = RoutingTable.builder(currentState.routingTable()); @@ -130,8 +140,12 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) // add tombstones to the cluster state for each deleted index final IndexGraveyard currentGraveyard = graveyardBuilder.addTombstones(indices).build(settings); metadataBuilder.indexGraveyard(currentGraveyard); // the new graveyard set on the metadata - logger.trace("{} tombstones purged from the cluster state. Previous tombstone size: {}. Current tombstone size: {}.", - graveyardBuilder.getNumPurged(), previousGraveyardSize, currentGraveyard.getTombstones().size()); + logger.trace( + "{} tombstones purged from the cluster state. Previous tombstone size: {}. Current tombstone size: {}.", + graveyardBuilder.getNumPurged(), + previousGraveyardSize, + currentGraveyard.getTombstones().size() + ); final ClusterState.Builder builder = ClusterState.builder(currentState) .routingTable(routingTableBuilder.build()) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java index 379579dc76968..0368a39ced87e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java @@ -55,7 +55,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; - public class MetadataDeleteIndexServiceTests extends ESTestCase { private AllocationService allocationService; private MetadataDeleteIndexService service; @@ -65,8 +64,9 @@ public class MetadataDeleteIndexServiceTests extends ESTestCase { public void setUp() throws Exception { super.setUp(); allocationService = mock(AllocationService.class); - when(allocationService.reroute(any(ClusterState.class), any(String.class))) - .thenAnswer(mockInvocation -> mockInvocation.getArguments()[0]); + when(allocationService.reroute(any(ClusterState.class), any(String.class))).thenAnswer( + mockInvocation -> mockInvocation.getArguments()[0] + ); service = new MetadataDeleteIndexService(Settings.EMPTY, null, allocationService); } @@ -80,17 +80,35 @@ public void testDeleteMissing() { public void testDeleteSnapshotting() { String index = randomAlphaOfLength(5); Snapshot snapshot = new Snapshot("doesn't matter", new SnapshotId("snapshot name", "snapshot uuid")); - SnapshotsInProgress snaps = SnapshotsInProgress.EMPTY.withAddedEntry(new SnapshotsInProgress.Entry(snapshot, true, false, - SnapshotsInProgress.State.INIT, singletonMap(index, new IndexId(index, "doesn't matter")), - Collections.emptyList(), Collections.emptyList(), System.currentTimeMillis(), (long) randomIntBetween(0, 1000), - ImmutableOpenMap.of(), null, SnapshotInfoTestUtils.randomUserMetadata(), VersionUtils.randomVersion(random()))); - ClusterState state = ClusterState.builder(clusterState(index)) - .putCustom(SnapshotsInProgress.TYPE, snaps) - .build(); - Exception e = expectThrows(SnapshotInProgressException.class, - () -> service.deleteIndices(state, singleton(state.metadata().getIndices().get(index).getIndex()))); - assertEquals("Cannot delete indices that are being snapshotted: [[" + index + "]]. Try again after snapshot finishes " - + "or cancel the currently running snapshot.", e.getMessage()); + SnapshotsInProgress snaps = SnapshotsInProgress.EMPTY.withAddedEntry( + new SnapshotsInProgress.Entry( + snapshot, + true, + false, + SnapshotsInProgress.State.INIT, + singletonMap(index, new IndexId(index, "doesn't matter")), + Collections.emptyList(), + Collections.emptyList(), + System.currentTimeMillis(), + (long) randomIntBetween(0, 1000), + ImmutableOpenMap.of(), + null, + SnapshotInfoTestUtils.randomUserMetadata(), + VersionUtils.randomVersion(random()) + ) + ); + ClusterState state = ClusterState.builder(clusterState(index)).putCustom(SnapshotsInProgress.TYPE, snaps).build(); + Exception e = expectThrows( + SnapshotInProgressException.class, + () -> service.deleteIndices(state, singleton(state.metadata().getIndices().get(index).getIndex())) + ); + assertEquals( + "Cannot delete indices that are being snapshotted: [[" + + index + + "]]. Try again after snapshot finishes " + + "or cancel the currently running snapshot.", + e.getMessage() + ); } public void testDeleteUnassigned() { @@ -117,7 +135,9 @@ public void testDeleteBackingIndexForDataStream() { int numBackingIndices = randomIntBetween(2, 5); String dataStreamName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); ClusterState before = DataStreamTestHelper.getClusterStateWithDataStreams( - List.of(new Tuple<>(dataStreamName, numBackingIndices)), List.of()); + List.of(new Tuple<>(dataStreamName, numBackingIndices)), + List.of() + ); int numIndexToDelete = randomIntBetween(1, numBackingIndices - 1); @@ -126,8 +146,10 @@ public void testDeleteBackingIndexForDataStream() { assertThat(after.metadata().getIndices().get(indexToDelete.getName()), IsNull.nullValue()); assertThat(after.metadata().getIndices().size(), equalTo(numBackingIndices - 1)); - assertThat(after.metadata().getIndices().get( - DataStream.getDefaultBackingIndexName(dataStreamName, numIndexToDelete)), IsNull.nullValue()); + assertThat( + after.metadata().getIndices().get(DataStream.getDefaultBackingIndexName(dataStreamName, numIndexToDelete)), + IsNull.nullValue() + ); } public void testDeleteMultipleBackingIndexForDataStream() { @@ -135,10 +157,14 @@ public void testDeleteMultipleBackingIndexForDataStream() { int numBackingIndicesToDelete = randomIntBetween(2, numBackingIndices - 1); String dataStreamName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); ClusterState before = DataStreamTestHelper.getClusterStateWithDataStreams( - List.of(new Tuple<>(dataStreamName, numBackingIndices)), List.of()); + List.of(new Tuple<>(dataStreamName, numBackingIndices)), + List.of() + ); - List indexNumbersToDelete = - randomSubsetOf(numBackingIndicesToDelete, IntStream.rangeClosed(1, numBackingIndices - 1).boxed().collect(Collectors.toList())); + List indexNumbersToDelete = randomSubsetOf( + numBackingIndicesToDelete, + IntStream.rangeClosed(1, numBackingIndices - 1).boxed().collect(Collectors.toList()) + ); Set indicesToDelete = new HashSet<>(); for (int k : indexNumbersToDelete) { @@ -160,36 +186,47 @@ public void testDeleteCurrentWriteIndexForDataStream() { int numBackingIndices = randomIntBetween(1, 5); String dataStreamName = randomAlphaOfLength(6).toLowerCase(Locale.ROOT); ClusterState before = DataStreamTestHelper.getClusterStateWithDataStreams( - List.of(new Tuple<>(dataStreamName, numBackingIndices)), List.of()); + List.of(new Tuple<>(dataStreamName, numBackingIndices)), + List.of() + ); Index indexToDelete = before.metadata().index(DataStream.getDefaultBackingIndexName(dataStreamName, numBackingIndices)).getIndex(); Exception e = expectThrows(IllegalArgumentException.class, () -> service.deleteIndices(before, Set.of(indexToDelete))); - assertThat(e.getMessage(), containsString("index [" + indexToDelete.getName() + "] is the write index for data stream [" + - dataStreamName + "] and cannot be deleted")); + assertThat( + e.getMessage(), + containsString( + "index [" + indexToDelete.getName() + "] is the write index for data stream [" + dataStreamName + "] and cannot be deleted" + ) + ); } public void testDeleteIndexWithSnapshotDeletion() { final boolean deleteSnapshot = randomBoolean(); final IndexMetadata indexMetadata = IndexMetadata.builder("test") - .settings(Settings.builder() - .put("index.version.created", VersionUtils.randomVersion(random())) - .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE) - .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY, "repo_name") - .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, randomBoolean() ? null : "repo_uuid") - .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY, "snap_name") - .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY, "snap_uuid") - .put(SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, deleteSnapshot) - .build()) + .settings( + Settings.builder() + .put("index.version.created", VersionUtils.randomVersion(random())) + .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE) + .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY, "repo_name") + .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, randomBoolean() ? null : "repo_uuid") + .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY, "snap_name") + .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY, "snap_uuid") + .put(SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, deleteSnapshot) + .build() + ) .numberOfShards(1) .numberOfReplicas(1) .build(); final ClusterState initialState = ClusterState.builder(ClusterName.DEFAULT) - .metadata(Metadata.builder() - .put(indexMetadata, false) - .putCustom(RepositoriesMetadata.TYPE, - new RepositoriesMetadata( - List.of(new RepositoryMetadata("repo_name", "fs", Settings.EMPTY).withUuid("repo_uuid"))))) + .metadata( + Metadata.builder() + .put(indexMetadata, false) + .putCustom( + RepositoriesMetadata.TYPE, + new RepositoriesMetadata(List.of(new RepositoryMetadata("repo_name", "fs", Settings.EMPTY).withUuid("repo_uuid"))) + ) + ) .routingTable(RoutingTable.builder().addAsNew(indexMetadata).build()) .blocks(ClusterBlocks.builder().addBlocks(indexMetadata)) .build(); @@ -249,8 +286,7 @@ public void testDeleteMultipleIndicesWithSnapshotDeletion() { .metadata(metadataBuilder) .build(); - SnapshotDeletionsPending pendingDeletions = - clusterState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + SnapshotDeletionsPending pendingDeletions = clusterState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); while (indices.size() > 0) { assertThat(pendingDeletions.isEmpty(), equalTo(true)); @@ -272,14 +308,14 @@ public void testDeleteMultipleIndicesWithSnapshotDeletion() { private ClusterState clusterState(String index) { IndexMetadata indexMetadata = IndexMetadata.builder(index) - .settings(Settings.builder().put("index.version.created", VersionUtils.randomVersion(random()))) - .numberOfShards(1) - .numberOfReplicas(1) - .build(); + .settings(Settings.builder().put("index.version.created", VersionUtils.randomVersion(random()))) + .numberOfShards(1) + .numberOfReplicas(1) + .build(); return ClusterState.builder(ClusterName.DEFAULT) - .metadata(Metadata.builder().put(indexMetadata, false)) - .routingTable(RoutingTable.builder().addAsNew(indexMetadata).build()) - .blocks(ClusterBlocks.builder().addBlocks(indexMetadata)) - .build(); + .metadata(Metadata.builder().put(indexMetadata, false)) + .routingTable(RoutingTable.builder().addAsNew(indexMetadata).build()) + .blocks(ClusterBlocks.builder().addBlocks(indexMetadata)) + .build(); } } From 77dfe0085f8e5179f53749ac41b87e8f38097b8b Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 9 Nov 2021 14:25:33 +0100 Subject: [PATCH 10/42] nits --- .../cluster/SnapshotDeletionsPending.java | 55 +++++++++---------- .../metadata/MetadataDeleteIndexService.java | 2 +- .../snapshots/SnapshotsService.java | 43 +++++++-------- 3 files changed, 46 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 315744314b7ad..68b8deec5a703 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -21,9 +21,10 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Iterator; +import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.function.Consumer; import static java.util.Collections.unmodifiableList; @@ -39,7 +40,6 @@ * id, the snapshot name, the repository name and the repository id (if it exists) once, along with the time at which the snapshot was added * to the pending deletion, in a {@link SnapshotDeletionsPending} entry. * - * * When cluster state is updated with such entries the {@link org.elasticsearch.snapshots.SnapshotsService} executes corresponding snapshot * delete requests to effectively delete the snapshot from the repository. It is possible that the deletion of a snapshot failed for various * reason (ex: conflicting snapshot operation, repository removed etc). In such cases the snapshot pending deletion is kept in the cluster @@ -51,21 +51,28 @@ public class SnapshotDeletionsPending extends AbstractNamedDiffable impl public static final SnapshotDeletionsPending EMPTY = new SnapshotDeletionsPending(List.of()); public static final String TYPE = "snapshot_deletions_pending"; + /** + * Version from which a snapshot can be marked as to be deleted after an index is deleted. + */ + public static final Version SNAPSHOT_DELETIONS_PENDING_VERSION = Version.V_8_1_0; + /** * Setting for the maximum number of snapshots pending deletion allowed in the cluster state. *

* This setting is here to prevent the cluster to grow too large. In the case that the number of snapshots pending deletion exceeds * the value of this setting the oldest entries are removed from the cluster state. Snapshots that are discarded are removed before * they can be deleted from their repository and are therefore considered as "leaking" and should be logged as such as warnings. + *

+ * This setting is a non-dynamic, node-level only setting that is only used on the elected master node. */ public static final Setting MAX_PENDING_DELETIONS_SETTING = Setting.intSetting( "cluster.snapshot.snapshot_deletions_pending.size", - 500, + 5_000, Setting.Property.NodeScope ); /** - * A list of snapshots to delete, sorted by creation time + * A list of snapshots to delete, in the order deletions were requested. */ private final List entries; @@ -111,7 +118,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public Version getMinimalSupportedVersion() { - return Version.CURRENT.minimumCompatibilityVersion(); + return SNAPSHOT_DELETIONS_PENDING_VERSION; } public SnapshotDeletionsPending withRemovedSnapshots(List snapshotIds) { @@ -120,8 +127,9 @@ public SnapshotDeletionsPending withRemovedSnapshots(List snapshotId } boolean changed = false; final List updatedEntries = new ArrayList<>(); + final Set removedSnapshotIds = new HashSet<>(snapshotIds); for (Entry entry : entries) { - if (snapshotIds.contains(entry.snapshotId)) { + if (removedSnapshotIds.contains(entry.snapshotId)) { changed = true; continue; } @@ -138,18 +146,7 @@ public SnapshotDeletionsPending withRemovedSnapshots(List snapshotId @Override public String toString() { - final StringBuilder builder = new StringBuilder("SnapshotDeletionsPending["); - boolean prepend = true; - final Iterator iterator = entries.stream().iterator(); - while (iterator.hasNext()) { - if (prepend == false) { - builder.append(','); - } - builder.append(iterator.next()); - prepend = false; - } - builder.append(']'); - return builder.toString(); + return "SnapshotDeletionsPending[" + entries + ']'; } public static class Entry implements Writeable, ToXContentObject { @@ -157,19 +154,19 @@ public static class Entry implements Writeable, ToXContentObject { private final String repositoryName; private final String repositoryUuid; private final SnapshotId snapshotId; - private final long creationTime; + private final long indexDeletionTime; - public Entry(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long creationTime) { + public Entry(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long indexDeletionTime) { this.repositoryName = Objects.requireNonNull(repositoryName); this.repositoryUuid = Objects.requireNonNull(repositoryUuid); this.snapshotId = Objects.requireNonNull(snapshotId); - this.creationTime = creationTime; + this.indexDeletionTime = indexDeletionTime; } private Entry(StreamInput in) throws IOException { this.repositoryName = in.readString(); this.repositoryUuid = in.readString(); - this.creationTime = in.readVLong(); + this.indexDeletionTime = in.readVLong(); this.snapshotId = new SnapshotId(in); } @@ -177,7 +174,7 @@ private Entry(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeString(repositoryName); out.writeString(repositoryUuid); - out.writeVLong(creationTime); + out.writeVLong(indexDeletionTime); snapshotId.writeTo(out); } @@ -193,8 +190,8 @@ public SnapshotId getSnapshotId() { return snapshotId; } - public long getCreationTime() { - return creationTime; + public long getIndexDeletionTime() { + return indexDeletionTime; } @Override @@ -202,7 +199,7 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; Entry entry = (Entry) o; - return creationTime == entry.creationTime + return indexDeletionTime == entry.indexDeletionTime && Objects.equals(repositoryName, entry.repositoryName) && Objects.equals(repositoryUuid, entry.repositoryUuid) && Objects.equals(snapshotId, entry.snapshotId); @@ -210,7 +207,7 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(repositoryName, repositoryUuid, snapshotId, creationTime); + return Objects.hash(repositoryName, repositoryUuid, snapshotId, indexDeletionTime); } @Override @@ -219,7 +216,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws { builder.field("repository_name", repositoryName); builder.field("repository_uuid", repositoryUuid); - builder.timeField("creation_time_millis", "creation_time", creationTime); + builder.timeField("creation_time_millis", "creation_time", indexDeletionTime); builder.field("snapshot", snapshotId); } builder.endObject(); @@ -228,7 +225,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public String toString() { - return '[' + repositoryName + '/' + repositoryUuid + ',' + snapshotId + ',' + creationTime + ']'; + return '[' + repositoryName + '/' + repositoryUuid + ',' + snapshotId + ',' + indexDeletionTime + ']'; } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index e9335130340d1..c58069b7dd1a9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -237,7 +237,7 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", maxPendingDeletions, evicted.getSnapshotId(), - Instant.ofEpochMilli(evicted.getCreationTime()).atZone(ZoneOffset.UTC), + Instant.ofEpochMilli(evicted.getIndexDeletionTime()).atZone(ZoneOffset.UTC), evicted.getRepositoryName(), evicted.getRepositoryUuid() ) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index c8ddf18ef32b7..19cf97b715a63 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -118,6 +118,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static java.util.Collections.unmodifiableList; import static org.elasticsearch.cluster.SnapshotsInProgress.completed; @@ -575,32 +576,26 @@ private static void ensureSnapshotNameAvailableInRepo(RepositoryData repositoryD } } - private static Set listOfCloneSources(final ClusterState state) { + private static Set cloneSources(final ClusterState state) { return state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) .asStream() .filter(SnapshotsInProgress.Entry::isClone) .map(SnapshotsInProgress.Entry::source) - .collect(Collectors.toSet()); + .collect(Collectors.toUnmodifiableSet()); } - private static Set listOfRestoreSources(final ClusterState state) { - final Set snapshotIds = new HashSet<>(); - for (RestoreInProgress.Entry restore : state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) { - snapshotIds.add(restore.snapshot().getSnapshotId()); - } - return Set.copyOf(snapshotIds); + private static Set restoreSources(final ClusterState state) { + return StreamSupport.stream(state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).spliterator(), false) + .map(restore -> restore.snapshot().getSnapshotId()) + .collect(Collectors.toUnmodifiableSet()); } - private static Set listOfDeletionsSources(final ClusterState state) { - final SnapshotDeletionsInProgress deletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); - if (deletionsInProgress == null) { - return Set.of(); - } - final Set snapshotIds = new HashSet<>(); - for (SnapshotDeletionsInProgress.Entry deletion : deletionsInProgress.getEntries()) { - snapshotIds.addAll(deletion.getSnapshots()); - } - return Set.copyOf(snapshotIds); + private static Set deletionsSources(final ClusterState state) { + return state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY) + .getEntries() + .stream() + .flatMap(deletion -> deletion.getSnapshots().stream()) + .collect(Collectors.toUnmodifiableSet()); } /** @@ -1324,9 +1319,9 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { || state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { return; } - final Set currentDeletions = listOfDeletionsSources(state); - final Set currentRestores = listOfRestoreSources(state); - final Set currentClones = listOfCloneSources(state); + final Set currentDeletions = deletionsSources(state); + final Set currentRestores = restoreSources(state); + final Set currentClones = cloneSources(state); // the list of snapshot ids to trigger deletion for, per repository final Map> snapshotsToDelete = new HashMap<>(); @@ -1378,7 +1373,7 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { snapshot.getRepositoryName(), snapshot.getRepositoryUuid(), snapshotId, - Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) ); continue; } @@ -1390,7 +1385,7 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { repository.name(), repository.uuid(), snapshotId, - Instant.ofEpochMilli(snapshot.getCreationTime()).atZone(ZoneOffset.UTC) + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) ); continue; } @@ -2441,7 +2436,7 @@ public ClusterState execute(ClusterState currentState) { return currentState; } - final Set activeCloneSources = listOfCloneSources(currentState); + final Set activeCloneSources = cloneSources(currentState); for (SnapshotId snapshotId : snapshotIds) { if (activeCloneSources.contains(snapshotId)) { throw new ConcurrentSnapshotExecutionException( From ea065e95641f4df416395e6d3279027b7450eaf7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 10 Nov 2021 17:44:57 +0100 Subject: [PATCH 11/42] feedback --- .../metadata/MetadataDeleteIndexService.java | 162 ++++++++++-------- .../snapshots/SnapshotsService.java | 65 ++++--- 2 files changed, 129 insertions(+), 98 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index c58069b7dd1a9..b1f96d9a2036c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -27,8 +27,8 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.Index; -import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.snapshots.RestoreService; import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.SnapshotId; @@ -40,15 +40,18 @@ import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; -import java.util.Objects; +import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY; +import static org.elasticsearch.snapshots.SnapshotsService.findRepositoryForPendingDeletion; /** * Deletes indices. @@ -167,7 +170,7 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY ); - final SnapshotDeletionsPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, meta); + final SnapshotDeletionsPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, currentState); if (updatedPendingDeletes != deletionsInPending) { if (customBuilder == null) { customBuilder = ImmutableOpenMap.builder(currentState.getCustoms()); @@ -180,82 +183,99 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) return allocationService.reroute(builder.build(), "deleted indices [" + indices + "]"); } + /** + * This method updates the list of snapshots marked as to be deleted if one or more searchable snapshots are deleted. + * + * The snapshots cannot be deleted at the same time of the searchable snapshots indices because deleting one or more snapshot requires a + * consistent view of their repositories data, and getting the consistent views cannot be done in the same cluster state update. It is + * also possible than one (or more) snapshot cannot be deleted immediately because the snapshot is involved in another restore or + * cloning or the repository might not be writeable etc. To address those conflicting situations this method only captures the snapshot + * information that are required to later delete the snapshot and stores them in a {@link SnapshotDeletionsPending.Entry} in cluster + * state. Once a snapshot is pending deletion it cannot be restored, mounted or cloned. If the snapshot pending deletion is involved in + * a snapshot operation at the time it is deleted then the deletion will happen once the conflicting operation is terminated. + */ private SnapshotDeletionsPending updateSnapshotDeletionsPending( final SnapshotDeletionsPending pendingDeletions, final Set indicesToDelete, - final Metadata metadata + final ClusterState state ) { - final long timestamp = Instant.now().toEpochMilli(); - SnapshotDeletionsPending.Builder builder = null; - boolean changed = false; + final List deletedIndicesSettings = indicesToDelete.stream() + .map(index -> state.metadata().getIndexSafe(index).getSettings()) + .filter(SearchableSnapshotsSettings::isSearchableSnapshotStore) + .filter(indexSettings -> indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false)) + .collect(Collectors.toList()); + if (deletedIndicesSettings.isEmpty()) { + return pendingDeletions; + } - for (Index indexToDelete : indicesToDelete) { - final Settings indexSettings = metadata.getIndexSafe(indexToDelete).getSettings(); - if (SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings) == false) { - continue; // not a searchable snapshot index - } - if (indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) == false) { - continue; // do not delete the snapshot when this searchable snapshot index is deleted - } - final SnapshotId snapshotId = new SnapshotId( - indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY), - indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY) - ); - boolean canDeleteSnapshot = true; - for (IndexMetadata other : metadata) { - if (indexToDelete.equals(other.getIndex())) { - continue; // do not check against itself - } - final Settings otherSettings = other.getSettings(); - if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { - continue; // other index is not a searchable snapshot index, skip - } - final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); - if (Objects.equals(snapshotId.getUUID(), otherSnapshotUuid) == false) { - continue; // other index is backed by a different snapshot, skip - } - assert otherSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false) : other; - if (indicesToDelete.contains(other.getIndex())) { - continue; // other index is going to be deleted as part of the same cluster state update - } - logger.debug( - "snapshot [{}] cannot be marked as to delete, another index [{}] is using the snapshot", - snapshotId, - other.getIndex() - ); - canDeleteSnapshot = false; // another index is using the same snapshot, do not delete the snapshot - break; - } - if (canDeleteSnapshot) { - if (builder == null) { - final int maxPendingDeletions = SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING.get(settings); - builder = new SnapshotDeletionsPending.Builder( - pendingDeletions, - evicted -> logger.warn( - () -> new ParameterizedMessage( - "maximum number of snapshots [{}] awaiting deletion has been reached in " - + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", - maxPendingDeletions, - evicted.getSnapshotId(), - Instant.ofEpochMilli(evicted.getIndexDeletionTime()).atZone(ZoneOffset.UTC), - evicted.getRepositoryName(), - evicted.getRepositoryUuid() - ) - ) - ); + final Set activeSearchableSnapshots = state.metadata().indices() + .stream() + .map(Map.Entry::getValue) + .filter(index -> indicesToDelete.contains(index.getIndex()) == false) + .map(IndexMetadata::getSettings) + .filter(SearchableSnapshotsSettings::isSearchableSnapshotStore) + .map(MetadataDeleteIndexService::toSnapshotId) + .collect(Collectors.toUnmodifiableSet()); + + final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE); + // used to deduplicate snapshots that were used by multiple deleted indices + final Map snapshotsWithRepository = new HashMap<>(); + // also used to log a warning for snapshots with unknown repository + final Map> snapshotsWithoutRepository = new HashMap<>(); + + for (Settings deletedIndexSettings : deletedIndicesSettings) { + SnapshotId snapshotId = toSnapshotId(deletedIndexSettings); + if (activeSearchableSnapshots.contains(snapshotId) == false) { + String repositoryUuid = deletedIndexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY); + String repositoryName = deletedIndexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY); + Optional repository = findRepositoryForPendingDeletion(repositories, repositoryName, repositoryUuid); + if (repository.isPresent()) { + snapshotsWithRepository.putIfAbsent(snapshotId, repository.get()); + } else { + snapshotsWithoutRepository.putIfAbsent(snapshotId, Tuple.tuple(repositoryName, repositoryUuid)); } - builder.add( - indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY), - indexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, RepositoryData.MISSING_UUID), - snapshotId, - timestamp - ); - changed = true; } } - if (changed) { - return builder.build(settings); + + final int maxPendingDeletions = SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING.get(settings); + final SnapshotDeletionsPending.Builder builder = new SnapshotDeletionsPending.Builder( + pendingDeletions, + evicted -> logger.warn( + () -> new ParameterizedMessage( + "maximum number of snapshots [{}] awaiting deletion has been reached in " + + "cluster state before snapshot [{}] deleted on [{}] in repository [{}/{}] could be deleted", + maxPendingDeletions, + evicted.getSnapshotId(), + Instant.ofEpochMilli(evicted.getIndexDeletionTime()).atZone(ZoneOffset.UTC), + evicted.getRepositoryName(), + evicted.getRepositoryUuid() + ) + ) + ); + + final long timestamp = Instant.now().toEpochMilli(); + for (Map.Entry entry : snapshotsWithRepository.entrySet()) { + logger.info("snapshot [{}:{}] added to the list of snapshots pending deletion", entry.getValue().name(), entry.getKey()); + builder.add(entry.getValue().name(), entry.getValue().uuid(), entry.getKey(), timestamp); + } + for (Map.Entry> entry : snapshotsWithoutRepository.entrySet()) { + // TODO also log that it will stay as pending for a given time/attempts and then be removed? + logger.warn( + "snapshot [{}] added to the list of snapshots pending deletion but refers to an unregistered repository [{}/{}]", + entry.getKey(), + entry.getValue().v1(), + entry.getValue().v2() + ); + builder.add(entry.getValue().v1(), entry.getValue().v2(), entry.getKey(), timestamp); } - return pendingDeletions; + return builder.build(settings); + } + + private static SnapshotId toSnapshotId(final Settings indexSettings) { + assert SearchableSnapshotsSettings.isSearchableSnapshotStore(indexSettings); + return new SnapshotId( + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY), + indexSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY) + ); } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 19cf97b715a63..e497dde741af3 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1340,33 +1340,11 @@ private void triggerSnapshotsPendingDeletions(final ClusterState state) { continue; } - Optional optionalRepository; - if (RepositoryData.MISSING_UUID.equals(snapshot.getRepositoryUuid()) == false) { - // the snapshot waiting to be deleted references a repository with a known uuid, - // let's try to find this repository among the existing ones first - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.uuid(), snapshot.getRepositoryUuid())) - .findFirst(); - if (optionalRepository.isEmpty()) { - // there is no existing repository matching the uuid, - // let's try to find the repository by name among the existing ones that have no uuid - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) - .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) - .findFirst(); - } - } else { - // the snapshot waiting to be deleted does not references a repository with a known uuid, - // let's try to find the repository by name among the existing ones, in the hope that - // the snapshot will be found there. - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.name(), snapshot.getRepositoryName())) - .findFirst(); - } - + final Optional optionalRepository = findRepositoryForPendingDeletion( + repositories, + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid() + ); if (optionalRepository.isEmpty()) { logger.debug( "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", @@ -1542,6 +1520,39 @@ public void onFailure(Exception e) { } } + public static Optional findRepositoryForPendingDeletion( + final RepositoriesMetadata repositories, + final String repositoryName, + final String repositoryUuid + ) { + if (repositories != null) { + if (RepositoryData.MISSING_UUID.equals(repositoryUuid) == false) { + // the snapshot waiting to be deleted references a repository with a known uuid, + // let's try to find this repository among the existing ones first + Optional optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), repositoryUuid)) + .findFirst(); + if (optionalRepository.isEmpty()) { + // there is no existing repository matching the uuid, + // let's try to find the repository by name among the existing ones that have no uuid + optionalRepository = repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) + .filter(repo -> Objects.equals(repo.name(), repositoryName)) + .findFirst(); + } + return optionalRepository; + } else { + // the snapshot waiting to be deleted does not references a repository with a known uuid, + // let's try to find the repository by name among the existing ones, in the hope that + // the snapshot will be found there. + return repositories.repositories().stream().filter(repo -> Objects.equals(repo.name(), repositoryName)).findFirst(); + } + } + return Optional.empty(); + } + private static ImmutableOpenMap processWaitingShardsAndRemovedNodes( SnapshotsInProgress.Entry entry, RoutingTable routingTable, From 91095b7e411446019a91528ea535e9971f4f7b9c Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 10 Nov 2021 18:04:24 +0100 Subject: [PATCH 12/42] spotless --- .../cluster/metadata/MetadataDeleteIndexService.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index b1f96d9a2036c..265de358b996e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -170,7 +170,11 @@ public ClusterState deleteIndices(ClusterState currentState, Set indices) SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY ); - final SnapshotDeletionsPending updatedPendingDeletes = updateSnapshotDeletionsPending(deletionsInPending, indicesToDelete, currentState); + final SnapshotDeletionsPending updatedPendingDeletes = updateSnapshotDeletionsPending( + deletionsInPending, + indicesToDelete, + currentState + ); if (updatedPendingDeletes != deletionsInPending) { if (customBuilder == null) { customBuilder = ImmutableOpenMap.builder(currentState.getCustoms()); @@ -208,7 +212,8 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( return pendingDeletions; } - final Set activeSearchableSnapshots = state.metadata().indices() + final Set activeSearchableSnapshots = state.metadata() + .indices() .stream() .map(Map.Entry::getValue) .filter(index -> indicesToDelete.contains(index.getIndex()) == false) From 916689e76573fc1c93c8403da91a808cc838e1ab Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 24 Nov 2021 14:27:29 +0100 Subject: [PATCH 13/42] skip cluster state updates --- .../cluster/SnapshotDeletionsPending.java | 13 + .../metadata/MetadataDeleteIndexService.java | 12 +- .../snapshots/SnapshotsService.java | 226 +++++++++++++----- ...leSnapshotsPendingDeletionsIntegTests.java | 2 +- 4 files changed, 182 insertions(+), 71 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 68b8deec5a703..6645e06353515 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -144,6 +144,19 @@ public SnapshotDeletionsPending withRemovedSnapshots(List snapshotId } } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SnapshotDeletionsPending that = (SnapshotDeletionsPending) o; + return Objects.equals(entries, that.entries); + } + + @Override + public int hashCode() { + return Objects.hash(entries); + } + @Override public String toString() { return "SnapshotDeletionsPending[" + entries + ']'; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 265de358b996e..8f5e65245f2b3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -223,8 +223,8 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( .collect(Collectors.toUnmodifiableSet()); final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE); - // used to deduplicate snapshots that were used by multiple deleted indices - final Map snapshotsWithRepository = new HashMap<>(); + // also used to deduplicate snapshots that were used by multiple deleted indices + final Map> snapshotsWithRepository = new HashMap<>(); // also used to log a warning for snapshots with unknown repository final Map> snapshotsWithoutRepository = new HashMap<>(); @@ -235,7 +235,7 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( String repositoryName = deletedIndexSettings.get(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY); Optional repository = findRepositoryForPendingDeletion(repositories, repositoryName, repositoryUuid); if (repository.isPresent()) { - snapshotsWithRepository.putIfAbsent(snapshotId, repository.get()); + snapshotsWithRepository.putIfAbsent(snapshotId, Tuple.tuple(repositoryName, repositoryUuid)); } else { snapshotsWithoutRepository.putIfAbsent(snapshotId, Tuple.tuple(repositoryName, repositoryUuid)); } @@ -259,9 +259,9 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( ); final long timestamp = Instant.now().toEpochMilli(); - for (Map.Entry entry : snapshotsWithRepository.entrySet()) { - logger.info("snapshot [{}:{}] added to the list of snapshots pending deletion", entry.getValue().name(), entry.getKey()); - builder.add(entry.getValue().name(), entry.getValue().uuid(), entry.getKey(), timestamp); + for (Map.Entry> entry : snapshotsWithRepository.entrySet()) { + logger.debug("snapshot [{}:{}] added to the list of snapshots pending deletion", entry.getValue().v1(), entry.getKey()); + builder.add(entry.getValue().v1(), entry.getValue().v2(), entry.getKey(), timestamp); } for (Map.Entry> entry : snapshotsWithoutRepository.entrySet()) { // TODO also log that it will stay as pending for a given time/attempts and then be removed? diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index e497dde741af3..d84937423ee54 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -992,17 +992,19 @@ public void applyClusterState(ClusterChangedEvent event) { newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event) ); - triggerSnapshotsPendingDeletions(event.state()); - } else if (snapshotCompletionListeners.isEmpty() == false) { - // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already - // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster - // state). - for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { - if (endingSnapshots.add(snapshot)) { - failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); + } else { + if (snapshotCompletionListeners.isEmpty() == false) { + // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already + // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster + // state). + for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { + if (endingSnapshots.add(snapshot)) { + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); + } } } } + triggerSnapshotsPendingDeletions(event); } catch (Exception e) { assert false : new AssertionError(e); logger.warn("Failed to update snapshot state ", e); @@ -1295,6 +1297,13 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS */ private final Set ongoingSnapshotsDeletions = ConcurrentCollections.newConcurrentSet(); + /** + * Set of pending snapshots deletions whose deletion is conflicting with on-going restores, clones or repository statuses + */ + private final Set pendingDeletionsWithConflictingRestores = ConcurrentCollections.newConcurrentSet(); + private final Set pendingDeletionsWithConflictingClones = ConcurrentCollections.newConcurrentSet(); + private final Set pendingDeletionsWithConflictingRepos = ConcurrentCollections.newConcurrentSet(); + /** * Find snapshots to delete in the the cluster state and triggers explicit snapshot delete requests. This method attempts to detect * conflicting situations where triggering the snapshot deletion would likely fail due to a concurrent snapshot operation. In such @@ -1307,77 +1316,165 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsPending}, we try to find a * repository with the same name. * - * @param state the current {@link ClusterState} + * @param event the current {@link ClusterChangedEvent} */ - private void triggerSnapshotsPendingDeletions(final ClusterState state) { - final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - final SnapshotDeletionsPending snapshotDeletionsPending = state.custom(SnapshotDeletionsPending.TYPE); - if (snapshotDeletionsPending == null - || snapshotDeletionsPending.isEmpty() - || repositories.repositories().isEmpty() - || state.nodes().isLocalNodeElectedMaster() == false - || state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { + private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { + if (event.localNodeMaster() == false) { + if (event.previousState().nodes().isLocalNodeElectedMaster()) { + clearPendingDeletionsWithConflicts(); + } return; } - final Set currentDeletions = deletionsSources(state); - final Set currentRestores = restoreSources(state); - final Set currentClones = cloneSources(state); - // the list of snapshot ids to trigger deletion for, per repository - final Map> snapshotsToDelete = new HashMap<>(); + if (pendingDeletionsChanged(event) || pendingDeletionsWithConflictsChanged(event)) { + final ClusterState state = event.state(); + final SnapshotDeletionsPending snapshotDeletionsPending = state.custom(SnapshotDeletionsPending.TYPE); + if (snapshotDeletionsPending == null || snapshotDeletionsPending.isEmpty()) { + clearPendingDeletionsWithConflicts(); + return; + } - for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { - final SnapshotId snapshotId = snapshot.getSnapshotId(); + final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - if (currentRestores.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); - continue; - } else if (currentClones.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); - continue; - } else if (currentDeletions.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is already queued", snapshotId); - continue; - } + final Set currentDeletions = deletionsSources(state); + final Set currentRestores = restoreSources(state); + final Set currentClones = cloneSources(state); - final Optional optionalRepository = findRepositoryForPendingDeletion( - repositories, - snapshot.getRepositoryName(), - snapshot.getRepositoryUuid() - ); - if (optionalRepository.isEmpty()) { - logger.debug( - "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", + // the list of snapshot ids to trigger deletion for, per repository + final Map> snapshotsToDelete = new HashMap<>(); + + for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { + final SnapshotId snapshotId = snapshot.getSnapshotId(); + + if (currentRestores.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); + pendingDeletionsWithConflictingRestores.add(snapshotId); + continue; + } + pendingDeletionsWithConflictingRestores.remove(snapshotId); + + if (currentClones.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); + pendingDeletionsWithConflictingClones.add(snapshotId); + continue; + } + pendingDeletionsWithConflictingClones.remove(snapshotId); + + if (currentDeletions.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is already queued", snapshotId); + pendingDeletionsWithConflictingRepos.remove(snapshotId); + continue; + } + + if (state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { + if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { + logger.debug( + "a repository clean-up is in progress, cannot delete pending snapshot [{}] created at {}", + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + + final Optional optionalRepository = findRepositoryForPendingDeletion( + repositories, snapshot.getRepositoryName(), - snapshot.getRepositoryUuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + snapshot.getRepositoryUuid() ); - continue; + if (optionalRepository.isEmpty()) { + if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { + logger.debug( + "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + final RepositoryMetadata repository = optionalRepository.get(); + if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { + if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { + logger.debug( + "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", + repository.name(), + repository.uuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + pendingDeletionsWithConflictingRepos.remove(snapshotId); + + // should we add some throttling to not always retry? + if (ongoingSnapshotsDeletions.add(snapshotId)) { + logger.info("triggering snapshot deletion for [{}]", snapshotId); + final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); + assert added : snapshotId; + } } + snapshotsToDelete.forEach( + (repo, snapshots) -> threadPool.generic().execute(new SnapshotsToDeleteRunnable(repo.name(), repo.uuid(), snapshots)) + ); + } + } - final RepositoryMetadata repository = optionalRepository.get(); - if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { - logger.debug( - "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", - repository.name(), - repository.uuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) - ); - continue; + private void clearPendingDeletionsWithConflicts() { + pendingDeletionsWithConflictingRestores.clear(); + pendingDeletionsWithConflictingClones.clear(); + pendingDeletionsWithConflictingRepos.clear(); + } + + private static boolean pendingDeletionsChanged(ClusterChangedEvent event) { + SnapshotDeletionsPending previous = event.previousState().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + SnapshotDeletionsPending currents = event.state().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + return Objects.equals(previous, currents) == false; + } + + private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) { + if (pendingDeletionsWithConflictingRestores.isEmpty() == false) { + RestoreInProgress previous = event.previousState().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); + RestoreInProgress currents = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); + if (Objects.equals(previous, currents) == false) { + return true; + } + } + if (pendingDeletionsWithConflictingClones.isEmpty() == false) { + Set previous = event.previousState() + .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .collect(Collectors.toSet()); + Set currents = event.state() + .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .collect(Collectors.toSet()); + if (Objects.equals(previous, currents) == false) { + return true; + } + } + if (pendingDeletionsWithConflictingRepos.isEmpty() == false) { + boolean previousCleanUp = event.previousState() + .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) + .hasCleanupInProgress(); + boolean currentCleanUp = event.state() + .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) + .hasCleanupInProgress(); + if (previousCleanUp != currentCleanUp) { + return true; } - // should we add some throttling to not always retry? - if (ongoingSnapshotsDeletions.add(snapshotId)) { - logger.trace("triggering snapshot deletion for [{}]", snapshotId); - final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); - assert added : snapshotId; + RepositoriesMetadata previous = event.previousState().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + RepositoriesMetadata current = event.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + if (previous.equals(current) == false) { + return true; } } - snapshotsToDelete.forEach( - (repo, snapshots) -> threadPool.generic().execute(new SnapshotsToDeleteRunnable(repo.name(), repo.uuid(), snapshots)) - ); + return false; } /** @@ -1426,6 +1523,7 @@ public void onFailure(Exception e) { shouldRetry = RepositoryData.MISSING_UUID.equals(repositoryUuid) == false; } else if (e instanceof ConcurrentSnapshotExecutionException) { + assert false : e; logger.debug( "[{}] failed to delete snapshot [{}]: a concurrent operation is running", repositoryName, diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index ab85778e38cab..a61961e0334b4 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -94,7 +94,7 @@ public void testSearchableSnapshotIsDeletedWhenRepoIsRecreated() throws Exceptio } else { // re register the repository under a different name: the snapshot // pending deletion logic should try to delete the snapshot based - // on the repository uuid + // on the repository uuid, that is why we force a verification here repoName = "new_" + repository; createRepository(repoName, "mock", repositorySettings, true); } From 858d4c102b1ac6488b5b91c19a9b1ad24e7dc867 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 25 Nov 2021 16:35:58 +0100 Subject: [PATCH 14/42] add repo clean up test --- ...leSnapshotsPendingDeletionsIntegTests.java | 66 +++++++++++++++++++ 1 file changed, 66 insertions(+) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index a61961e0334b4..dffd35023865c 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -10,17 +10,24 @@ import org.apache.lucene.search.TotalHits; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionRunnable; +import org.elasticsearch.action.admin.cluster.repositories.cleanup.CleanupRepositoryResponse; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.elasticsearch.action.support.GroupedActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Strings; import org.elasticsearch.common.TriConsumer; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException; import org.elasticsearch.snapshots.RestoreInfo; @@ -32,6 +39,7 @@ import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -352,6 +360,64 @@ public void onFailure(Exception e) { assertTrue(getSnapshotsResponse.getSnapshots().stream().noneMatch(snapshotInfo -> mounts.containsValue(snapshotInfo.snapshotId()))); } + public void testSearchableSnapshotIsDeletedWithOnRepoCleanUp() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + final int garbageFiles = between(1, 10); + final int garbageSize = between(1, 50); + final PlainActionFuture> garbageFuture = PlainActionFuture.newFuture(); + final GroupedActionListener garbageGroupedListener = new GroupedActionListener<>(garbageFuture, garbageFiles); + + final BlobStoreRepository blobRepository = getRepositoryOnMaster(repository); + for (int i = 0; i < garbageFiles; i++) { + int garbageId = i; + blobRepository.threadPool() + .generic() + .execute( + ActionRunnable.run( + garbageGroupedListener, + () -> blobRepository.blobStore() + .blobContainer(blobRepository.basePath()) + .writeBlob("snap-" + garbageId + ".dat", new BytesArray(randomByteArrayOfLength(garbageSize)), true) + ) + ); + } + garbageFuture.get(); + + blockMasterOnWriteIndexFile(repository); + + final ActionFuture cleanUpFuture = client().admin() + .cluster() + .prepareCleanupRepository(repository) + .execute(); + + final String masterNode = internalCluster().getMasterName(); + awaitClusterState( + state -> state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress() + ); + waitForBlock(masterNode, repository); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + assertFalse(cleanUpFuture.isDone()); + + unblockNode(repository, masterNode); + awaitNoMoreSnapshotsDeletions(); + + final CleanupRepositoryResponse cleanUpResponse = cleanUpFuture.get(); + assertThat(cleanUpResponse.result().blobs(), equalTo((long) garbageFiles)); + assertThat(cleanUpResponse.result().bytes(), equalTo((long) garbageSize * garbageFiles)); + + expectThrows( + SnapshotMissingException.class, + () -> client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get() + ); + } catch (Exception e) { + throw new AssertionError(e); + } + }); + } + private void mountIndexThenExecute(final TriConsumer test) throws Exception { final String suffix = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); final String repository = "repository-" + suffix; From 34de0d092c90ae2ddf72161075efcd1868c9eee5 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 30 Nov 2021 10:37:59 +0100 Subject: [PATCH 15/42] also clean ups --- .../snapshots/SnapshotsService.java | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index d84937423ee54..3423d0e2d581e 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1298,9 +1298,11 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS private final Set ongoingSnapshotsDeletions = ConcurrentCollections.newConcurrentSet(); /** - * Set of pending snapshots deletions whose deletion is conflicting with on-going restores, clones or repository statuses + * Set of pending snapshots deletions whose deletion is conflicting with on-going restores/clones/repository clean up or repository + * missing or read-only. Those sets are used to identify the cluster state updates to process in case they resolve some conflict. */ private final Set pendingDeletionsWithConflictingRestores = ConcurrentCollections.newConcurrentSet(); + private final Set pendingDeletionsWithConflictingCleanUps = ConcurrentCollections.newConcurrentSet(); private final Set pendingDeletionsWithConflictingClones = ConcurrentCollections.newConcurrentSet(); private final Set pendingDeletionsWithConflictingRepos = ConcurrentCollections.newConcurrentSet(); @@ -1335,6 +1337,7 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { } final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final RepositoryCleanupInProgress cleanUps = state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); final Set currentDeletions = deletionsSources(state); final Set currentRestores = restoreSources(state); @@ -1360,14 +1363,8 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { } pendingDeletionsWithConflictingClones.remove(snapshotId); - if (currentDeletions.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is already queued", snapshotId); - pendingDeletionsWithConflictingRepos.remove(snapshotId); - continue; - } - - if (state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY).hasCleanupInProgress()) { - if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { + if (cleanUps.hasCleanupInProgress()) { + if (pendingDeletionsWithConflictingCleanUps.add(snapshotId)) { logger.debug( "a repository clean-up is in progress, cannot delete pending snapshot [{}] created at {}", snapshotId, @@ -1376,6 +1373,13 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { } continue; } + pendingDeletionsWithConflictingCleanUps.remove(snapshotId); + + if (currentDeletions.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is already queued", snapshotId); + pendingDeletionsWithConflictingRepos.remove(snapshotId); + continue; + } final Optional optionalRepository = findRepositoryForPendingDeletion( repositories, @@ -1424,6 +1428,7 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { private void clearPendingDeletionsWithConflicts() { pendingDeletionsWithConflictingRestores.clear(); + pendingDeletionsWithConflictingCleanUps.clear(); pendingDeletionsWithConflictingClones.clear(); pendingDeletionsWithConflictingRepos.clear(); } @@ -1458,21 +1463,20 @@ private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) } } if (pendingDeletionsWithConflictingRepos.isEmpty() == false) { + RepositoriesMetadata previous = event.previousState().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + RepositoriesMetadata current = event.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + if (previous.equals(current) == false) { + return true; + } + } + if (pendingDeletionsWithConflictingCleanUps.isEmpty() == false) { boolean previousCleanUp = event.previousState() .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) .hasCleanupInProgress(); boolean currentCleanUp = event.state() .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) .hasCleanupInProgress(); - if (previousCleanUp != currentCleanUp) { - return true; - } - - RepositoriesMetadata previous = event.previousState().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - RepositoriesMetadata current = event.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - if (previous.equals(current) == false) { - return true; - } + return previousCleanUp != currentCleanUp; } return false; } From cf66c462179b0a1400ecb6b6637ecc6de456ab5a Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 30 Nov 2021 17:02:07 +0100 Subject: [PATCH 16/42] retries + expiration --- .../common/settings/ClusterSettings.java | 2 + .../snapshots/SnapshotsService.java | 171 ++++++++++++------ ...leSnapshotsPendingDeletionsIntegTests.java | 80 ++++++++ 3 files changed, 194 insertions(+), 59 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index f2baa71c171a6..291ff5a03960d 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -486,6 +486,8 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, + SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, + SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, RestoreService.REFRESH_REPO_UUID_ON_RESTORE_SETTING, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 3423d0e2d581e..1860157d44173 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -226,6 +226,15 @@ public SnapshotsService( maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); + pendingDeletionsRetryInterval = PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.get(settings); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, t -> pendingDeletionsRetryInterval = t); + pendingDeletionsExpirationInterval = PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.get(settings); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer( + PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, + t -> pendingDeletionsExpirationInterval = t + ); } this.systemIndexDescriptorMap = systemIndexDescriptorMap; } @@ -1297,6 +1306,11 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS */ private final Set ongoingSnapshotsDeletions = ConcurrentCollections.newConcurrentSet(); + // only used in tests + public boolean hasOngoingSnapshotsDeletions(SnapshotId snapshotId) { + return ongoingSnapshotsDeletions.contains(snapshotId); + } + /** * Set of pending snapshots deletions whose deletion is conflicting with on-going restores/clones/repository clean up or repository * missing or read-only. Those sets are used to identify the cluster state updates to process in case they resolve some conflict. @@ -1309,7 +1323,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS /** * Find snapshots to delete in the the cluster state and triggers explicit snapshot delete requests. This method attempts to detect * conflicting situations where triggering the snapshot deletion would likely fail due to a concurrent snapshot operation. In such - * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates on the conflicting + * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates once the conflicting * situation is resolved. * * The repository name and uuid information are extracted from the {@link SnapshotDeletionsPending} entries in order to find the @@ -1343,8 +1357,8 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { final Set currentRestores = restoreSources(state); final Set currentClones = cloneSources(state); - // the list of snapshot ids to trigger deletion for, per repository - final Map> snapshotsToDelete = new HashMap<>(); + // the snapshots to trigger deletion for, per repository + final Map> snapshotsToDelete = new HashMap<>(); for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { final SnapshotId snapshotId = snapshot.getSnapshotId(); @@ -1413,11 +1427,11 @@ private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { } pendingDeletionsWithConflictingRepos.remove(snapshotId); - // should we add some throttling to not always retry? if (ongoingSnapshotsDeletions.add(snapshotId)) { logger.info("triggering snapshot deletion for [{}]", snapshotId); - final boolean added = snapshotsToDelete.computeIfAbsent(repository, r -> new HashSet<>()).add(snapshotId); - assert added : snapshotId; + final Long previous = snapshotsToDelete.computeIfAbsent(repository, r -> new HashMap<>()) + .put(snapshotId, snapshot.getIndexDeletionTime()); + assert previous == null : snapshotId; } } snapshotsToDelete.forEach( @@ -1481,116 +1495,155 @@ private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) return false; } + public static final Setting PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING = Setting.timeSetting( + "snapshot.snapshot_deletions_pending.retry_interval", + TimeValue.timeValueSeconds(30L), + TimeValue.ZERO, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING = Setting.timeSetting( + "snapshot.snapshot_deletions_pending.expiration_interval", + TimeValue.timeValueHours(12L), + TimeValue.ZERO, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + private volatile TimeValue pendingDeletionsRetryInterval; + private volatile TimeValue pendingDeletionsExpirationInterval; + + private boolean isExpiredPendingDeletion(long deletionTimeEpochMillis) { + return Instant.ofEpochMilli(deletionTimeEpochMillis) + .plusMillis(pendingDeletionsExpirationInterval.getMillis()) + .isBefore(Instant.now()); + } + /** * A {@link Runnable} used to process the deletion of snapshots marked as to delete for a given repository. */ private class SnapshotsToDeleteRunnable extends AbstractRunnable { - private final Set snapshotIdsToDelete; + private final Map snapshots; private final String repositoryName; private final String repositoryUuid; + private final boolean missingUuid; - SnapshotsToDeleteRunnable(String repositoryName, String repositoryUuid, Set snapshotIdsToDelete) { + SnapshotsToDeleteRunnable(String repositoryName, String repositoryUuid, Map snapshots) { this.repositoryName = Objects.requireNonNull(repositoryName); this.repositoryUuid = Objects.requireNonNull(repositoryUuid); - this.snapshotIdsToDelete = Objects.requireNonNull(snapshotIdsToDelete); - assert snapshotIdsToDelete.isEmpty() == false; + this.snapshots = Objects.requireNonNull(snapshots); + this.missingUuid = RepositoryData.MISSING_UUID.equals(repositoryUuid); } @Override protected void doRun() throws Exception { - final List missingSnapshots = new CopyOnWriteArrayList<>(); - final CountDown countDown = new CountDown(snapshotIdsToDelete.size()); + final Set pendingDeletionsToRemove = ConcurrentCollections.newConcurrentSet(); + final CountDown countDown = new CountDown(snapshots.size()); - for (SnapshotId snapshotId : snapshotIdsToDelete) { + for (Map.Entry snapshot : snapshots.entrySet()) { + final SnapshotId snapshotId = snapshot.getKey(); final ActionListener listener = new ActionListener() { @Override public void onResponse(Void unused) { - logger.debug("[{}] snapshot marked as to delete [{}] is now deleted", repositoryName, snapshotId); - removeSnapshot(snapshotId, true); + logger.debug( + "snapshot marked as to delete [{}] successfully deleted from repository [{}/{}]", + snapshotId, + repositoryName, + repositoryUuid + ); + pendingDeletionsToRemove.add(snapshotId); + finish(); } @Override public void onFailure(Exception e) { - boolean shouldRetry = true; - if (e instanceof SnapshotMissingException) { + if (e instanceof SnapshotMissingException && missingUuid == false) { + pendingDeletionsToRemove.add(snapshotId); logger.debug( () -> new ParameterizedMessage( - "[{}] snapshot to delete [{}] is already deleted or is missing", + "snapshot marked as to delete [{}] is missing in repository [{}/{}], removing from pending deletions", + snapshotId, repositoryName, - snapshotId + repositoryUuid ), e ); - // only retry missing snapshots if the repository uuid is unknown, otherwise the snapshot pending deletion entry - // is removed from the cluster state as we know it does not exist anymore in the repository - shouldRetry = RepositoryData.MISSING_UUID.equals(repositoryUuid) == false; - - } else if (e instanceof ConcurrentSnapshotExecutionException) { - assert false : e; - logger.debug( - "[{}] failed to delete snapshot [{}]: a concurrent operation is running", - repositoryName, - snapshotId - ); - } else if (e instanceof RepositoryMissingException) { + } else if (isExpiredPendingDeletion(snapshot.getValue())) { + pendingDeletionsToRemove.add(snapshotId); logger.warn( () -> new ParameterizedMessage( - "[{}] failed to delete snapshot [{}]: repository has been removed before snapshot marked as " - + "to delete could be deleted, the snapshot might be leaking", + "snapshot marked as to delete [{}] failed to be deleted within [{}]. The pending snapshot " + + "expired before the snapshot could be deleted from the repository and as such might still " + + "exist in the original repository [{}/{}]. This snapshot will now be removed from the list of " + + "pending deletions.", + snapshotId, + pendingDeletionsExpirationInterval, repositoryName, - snapshotId + repositoryUuid ), e ); } else { - logger.warn( - () -> new ParameterizedMessage("[{}] failed to delete snapshot [{}]", repositoryName, snapshotId), + logger.debug( + () -> new ParameterizedMessage( + "[{}/{}] attempt to delete snapshot marked as to delete [{}] failed; deletion will be retried in [{}]", + repositoryName, + repositoryUuid, + snapshotId, + pendingDeletionsRetryInterval + ), e ); } - removeSnapshot(snapshotId, shouldRetry); + finish(); } - void removeSnapshot(SnapshotId snapshotId, boolean shouldRetry) { - if (shouldRetry) { - final boolean removed = ongoingSnapshotsDeletions.remove(snapshotId); - assert removed : "snapshot to delete [" + snapshotId + "] not found"; - } else { - missingSnapshots.add(snapshotId); - } - if (countDown.countDown() && missingSnapshots.isEmpty() == false) { - clusterService.submitStateUpdateTask( - "remove-missing-snapshot-deletions-in-pending", - new ClusterStateUpdateTask() { + void finish() { + if (countDown.countDown()) { + final Map retryables = snapshots.entrySet() + .stream() + .filter(snap -> pendingDeletionsToRemove.contains(snap.getKey()) == false) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (retryables.isEmpty() == false) { + // TODO maybe re-resolve repository here if the uuid is missing? + threadPool.scheduleUnlessShuttingDown( + pendingDeletionsRetryInterval, + ThreadPool.Names.GENERIC, + new SnapshotsToDeleteRunnable(repositoryName, repositoryUuid, retryables) + ); + } + if (pendingDeletionsToRemove.isEmpty() == false) { + clusterService.submitStateUpdateTask("remove-snapshot-deletions-in-pending", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsPending deletionsInPending = currentState.custom( + final SnapshotDeletionsPending currentPendings = currentState.custom( SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY ); - final SnapshotDeletionsPending updatedDeletionsInPending = deletionsInPending.withRemovedSnapshots( - missingSnapshots + final SnapshotDeletionsPending updatedPendings = currentPendings.withRemovedSnapshots( + List.copyOf(pendingDeletionsToRemove) ); - if (deletionsInPending == updatedDeletionsInPending) { + if (currentPendings == updatedPendings) { return currentState; } return ClusterState.builder(currentState) - .putCustom(SnapshotDeletionsPending.TYPE, updatedDeletionsInPending) + .putCustom(SnapshotDeletionsPending.TYPE, updatedPendings) .build(); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - missingSnapshots.forEach(ongoingSnapshotsDeletions::remove); + ongoingSnapshotsDeletions.removeAll(pendingDeletionsToRemove); } @Override public void onFailure(String source, Exception e) { - missingSnapshots.forEach(ongoingSnapshotsDeletions::remove); + ongoingSnapshotsDeletions.removeAll(pendingDeletionsToRemove); } - } - ); + }); + } } } }; @@ -1614,9 +1667,9 @@ public void onFailure(String source, Exception e) { @Override public void onFailure(Exception e) { - ongoingSnapshotsDeletions.removeAll(snapshotIdsToDelete); + ongoingSnapshotsDeletions.removeAll(snapshots.keySet()); logger.warn( - () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshots {}", repositoryName, snapshotIdsToDelete), + () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshots {}", repositoryName, snapshots.keySet()), e ); } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index dffd35023865c..593f406b5f597 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException; @@ -35,6 +36,7 @@ import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; import org.elasticsearch.snapshots.SnapshotState; +import org.elasticsearch.snapshots.SnapshotsService; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; @@ -57,6 +59,7 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; public class SearchableSnapshotsPendingDeletionsIntegTests extends BaseFrozenSearchableSnapshotsIntegTestCase { @@ -418,6 +421,83 @@ public void testSearchableSnapshotIsDeletedWithOnRepoCleanUp() throws Exception }); } + public void testSearchableSnapshotIsDeletedAfterExpiration() throws Exception { + mountIndexThenExecute((repository, snapshot, index) -> { + try { + assertAcked( + clusterAdmin().prepareUpdateSettings() + .setTransientSettings( + Settings.builder() + .put( + SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey(), + TimeValue.timeValueMillis(randomLongBetween(100L, 1000L)) + ) + .build() + ) + ); + + assertAcked( + clusterAdmin().preparePutRepository(repository) + .setVerify(false) + .setType("mock") + .setSettings( + Settings.builder() + .put(getRepositorySettings(repository).build()) + .put("random_control_io_exception_rate", 1.0) + .build() + ) + ); + + assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); + awaitSnapshotPendingDeletion(snapshot); + + final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); + assertBusy(() -> assertTrue(snapshotsService.hasOngoingSnapshotsDeletions(snapshot))); + + assertAcked( + clusterAdmin().prepareUpdateSettings() + .setTransientSettings( + Settings.builder() + .put(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey(), TimeValue.ZERO) + .build() + ) + ); + + assertBusy(() -> assertFalse(snapshotsService.hasOngoingSnapshotsDeletions(snapshot))); + awaitNoMoreSnapshotsDeletions(); + + } catch (Exception e) { + throw new AssertionError(e); + } finally { + assertAcked( + clusterAdmin().prepareUpdateSettings() + .setTransientSettings( + Settings.builder() + .putNull(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey()) + .putNull(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey()) + .build() + ) + ); + assertAcked( + clusterAdmin().preparePutRepository(repository) + .setVerify(false) + .setType("mock") + .setSettings( + Settings.builder() + .put(getRepositorySettings(repository).build()) + .put("random_control_io_exception_rate", 0.0) + .build() + ) + ); + } + + assertThat( + client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot.getName()).get().getSnapshots(), + hasSize(1) + ); + }); + } + private void mountIndexThenExecute(final TriConsumer test) throws Exception { final String suffix = randomAlphaOfLength(5).toLowerCase(Locale.ROOT); final String repository = "repository-" + suffix; From e181d4a85c17a61556e629418e7cbbb396c0cbd7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 30 Nov 2021 17:38:59 +0100 Subject: [PATCH 17/42] missing uuid --- .../org/elasticsearch/cluster/SnapshotDeletionsPending.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 6645e06353515..0619e4d7e178d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -15,6 +15,7 @@ import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.xcontent.ToXContentObject; import org.elasticsearch.xcontent.XContentBuilder; @@ -171,7 +172,7 @@ public static class Entry implements Writeable, ToXContentObject { public Entry(String repositoryName, String repositoryUuid, SnapshotId snapshotId, long indexDeletionTime) { this.repositoryName = Objects.requireNonNull(repositoryName); - this.repositoryUuid = Objects.requireNonNull(repositoryUuid); + this.repositoryUuid = Objects.requireNonNullElse(repositoryUuid, RepositoryData.MISSING_UUID); this.snapshotId = Objects.requireNonNull(snapshotId); this.indexDeletionTime = indexDeletionTime; } From 3bd59a3c3f67e928e8240ed78736b50efc7a9a41 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:15:43 +0100 Subject: [PATCH 18/42] 5000 --- .../org/elasticsearch/cluster/SnapshotDeletionsPending.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index 0619e4d7e178d..b1669cffefe0f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -45,7 +45,7 @@ * delete requests to effectively delete the snapshot from the repository. It is possible that the deletion of a snapshot failed for various * reason (ex: conflicting snapshot operation, repository removed etc). In such cases the snapshot pending deletion is kept in the cluster * state and the deletion will be retried on the next cluster state update. To avoid too many snapshots pending deletion stored in cluster - * state the number is limited to 500 and configurable through the {@link #MAX_PENDING_DELETIONS_SETTING} setting. + * state the number is limited to 5000 and configurable through the {@link #MAX_PENDING_DELETIONS_SETTING} setting. */ public class SnapshotDeletionsPending extends AbstractNamedDiffable implements Custom { From 4b9080a2f4943f572e64ac9f5e073cc6398ff8e7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:17:57 +0100 Subject: [PATCH 19/42] order of fields --- .../org/elasticsearch/cluster/SnapshotDeletionsPending.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index b1669cffefe0f..f6a2170253698 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -180,16 +180,16 @@ public Entry(String repositoryName, String repositoryUuid, SnapshotId snapshotId private Entry(StreamInput in) throws IOException { this.repositoryName = in.readString(); this.repositoryUuid = in.readString(); - this.indexDeletionTime = in.readVLong(); this.snapshotId = new SnapshotId(in); + this.indexDeletionTime = in.readVLong(); } @Override public void writeTo(StreamOutput out) throws IOException { out.writeString(repositoryName); out.writeString(repositoryUuid); - out.writeVLong(indexDeletionTime); snapshotId.writeTo(out); + out.writeVLong(indexDeletionTime); } public String getRepositoryName() { From 44552c58aa95590cbcbf5e256fb6ebbefe2109ca Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:26:54 +0100 Subject: [PATCH 20/42] Optional.or() --- .../snapshots/SnapshotsService.java | 23 +++++++++---------- 1 file changed, 11 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 7d7b8c4125e32..4a07e87e35c18 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1701,20 +1701,19 @@ public static Optional findRepositoryForPendingDeletion( if (RepositoryData.MISSING_UUID.equals(repositoryUuid) == false) { // the snapshot waiting to be deleted references a repository with a known uuid, // let's try to find this repository among the existing ones first - Optional optionalRepository = repositories.repositories() + return repositories.repositories() .stream() .filter(repo -> Objects.equals(repo.uuid(), repositoryUuid)) - .findFirst(); - if (optionalRepository.isEmpty()) { - // there is no existing repository matching the uuid, - // let's try to find the repository by name among the existing ones that have no uuid - optionalRepository = repositories.repositories() - .stream() - .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) - .filter(repo -> Objects.equals(repo.name(), repositoryName)) - .findFirst(); - } - return optionalRepository; + .findFirst() + .or( + // there is no existing repository matching the uuid, + // let's try to find the repository by name among the existing ones that have no uuid + () -> repositories.repositories() + .stream() + .filter(repo -> Objects.equals(repo.uuid(), RepositoryData.MISSING_UUID)) + .filter(repo -> Objects.equals(repo.name(), repositoryName)) + .findFirst() + ); } else { // the snapshot waiting to be deleted does not references a repository with a known uuid, // let's try to find the repository by name among the existing ones, in the hope that From bd863e9f86ff06e442d2cc6a451a25a7df07b8e9 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:28:32 +0100 Subject: [PATCH 21/42] remove //TODO --- .../cluster/metadata/MetadataDeleteIndexService.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 8f5e65245f2b3..dc41c8a71cac3 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -264,7 +264,6 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( builder.add(entry.getValue().v1(), entry.getValue().v2(), entry.getKey(), timestamp); } for (Map.Entry> entry : snapshotsWithoutRepository.entrySet()) { - // TODO also log that it will stay as pending for a given time/attempts and then be removed? logger.warn( "snapshot [{}] added to the list of snapshots pending deletion but refers to an unregistered repository [{}/{}]", entry.getKey(), From 8bb56d3a40fb97bdf7a3a36f915db561c75fb3a7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:30:15 +0100 Subject: [PATCH 22/42] remove curly --- .../snapshots/SnapshotsService.java | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 4a07e87e35c18..58bfc2f0bcced 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1018,15 +1018,13 @@ public void applyClusterState(ClusterChangedEvent event) { newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event) ); - } else { - if (snapshotCompletionListeners.isEmpty() == false) { - // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already - // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster - // state). - for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { - if (endingSnapshots.add(snapshot)) { - failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); - } + } else if (snapshotCompletionListeners.isEmpty() == false) { + // We have snapshot listeners but are not the master any more. Fail all waiting listeners except for those that already + // have their snapshots finalizing (those that are already finalizing will fail on their own from to update the cluster + // state). + for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { + if (endingSnapshots.add(snapshot)) { + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); } } } From e541a4cc8fdbbc7c25aae0386ab38ed573e09ff7 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:31:49 +0100 Subject: [PATCH 23/42] currents --- .../elasticsearch/snapshots/SnapshotsService.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 58bfc2f0bcced..525e6629a61fc 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1464,15 +1464,15 @@ private void clearPendingDeletionsWithConflicts() { private static boolean pendingDeletionsChanged(ClusterChangedEvent event) { SnapshotDeletionsPending previous = event.previousState().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); - SnapshotDeletionsPending currents = event.state().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); - return Objects.equals(previous, currents) == false; + SnapshotDeletionsPending current = event.state().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + return Objects.equals(previous, current) == false; } private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) { if (pendingDeletionsWithConflictingRestores.isEmpty() == false) { RestoreInProgress previous = event.previousState().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); - RestoreInProgress currents = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); - if (Objects.equals(previous, currents) == false) { + RestoreInProgress current = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); + if (Objects.equals(previous, current) == false) { return true; } } @@ -1482,12 +1482,12 @@ private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) .asStream() .filter(SnapshotsInProgress.Entry::isClone) .collect(Collectors.toSet()); - Set currents = event.state() + Set current = event.state() .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) .asStream() .filter(SnapshotsInProgress.Entry::isClone) .collect(Collectors.toSet()); - if (Objects.equals(previous, currents) == false) { + if (Objects.equals(previous, current) == false) { return true; } } From 2188a703442dc0769e5b9141bd5c41b2058e7e5d Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 16:51:26 +0100 Subject: [PATCH 24/42] no timeout --- .../snapshots/SnapshotsService.java | 25 ++++++------------- 1 file changed, 7 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 525e6629a61fc..96767de29efb6 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -1664,12 +1664,7 @@ public void onFailure(String source, Exception e) { }; try { - deleteSnapshotsByUuid( - repositoryName, - new String[] { snapshotId.getUUID() }, - DeleteSnapshotRequest.DEFAULT_MASTER_NODE_TIMEOUT, - listener - ); + deleteSnapshotsByUuid(repositoryName, new String[] { snapshotId.getUUID() }, listener); } catch (Exception e) { logger.warn( () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshot [{}]", repositoryName, snapshotId), @@ -2510,16 +2505,10 @@ public void deleteSnapshotsByName(final DeleteSnapshotRequest request, final Act * * @param repositoryName the name of the repository that contains the snapshots to delete * @param snapshotUuids the uuids of the snapshots to delete - * @param masterNodeTimeout the timeout to use for the cluster state update task * @param listener listener */ - private void deleteSnapshotsByUuid( - final String repositoryName, - final String[] snapshotUuids, - final TimeValue masterNodeTimeout, - final ActionListener listener - ) { - deleteSnapshots(repositoryName, null, snapshotUuids, masterNodeTimeout, SnapshotId::getUUID, listener); + private void deleteSnapshotsByUuid(final String repositoryName, final String[] snapshotUuids, final ActionListener listener) { + deleteSnapshots(repositoryName, null, snapshotUuids, null, SnapshotId::getUUID, listener); } /** @@ -2530,15 +2519,15 @@ private void deleteSnapshotsByUuid( * @param repositoryName the name of the repository that contains the snapshots to delete * @param snapshotNames the names of the snapshots to delete * @param snapshotUuids the uuids of the snapshots to delete - * @param masterNodeTimeout the timeout to use for the cluster state update task + * @param masterNodeTimeout the timeout to use for the cluster state update task, or null if no time out is needed * @param mapping the mapping function used to match the {@link SnapshotId} against the given snapshotNamesOrUuids * @param listener listener */ private void deleteSnapshots( final String repositoryName, - final String[] snapshotNames, - final String[] snapshotUuids, - final TimeValue masterNodeTimeout, + @Nullable final String[] snapshotNames, + @Nullable final String[] snapshotUuids, + @Nullable final TimeValue masterNodeTimeout, final Function mapping, final ActionListener listener ) { From 69608bf8e1610ec246a33df7deca48ad5920d41e Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 17:10:07 +0100 Subject: [PATCH 25/42] pass null --- .../repositories/blobstore/BlobStoreRepository.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index e5e0c9f2e4e7e..10812d12b7d86 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -36,7 +36,6 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.Metadata; @@ -2524,8 +2523,7 @@ private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state, } } updatedDeletionsInProgress = changedDeletions ? SnapshotDeletionsInProgress.of(deletionEntries) : null; - final SnapshotDeletionsPending pendingDeletions = state.custom(SnapshotDeletionsPending.TYPE); - return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress, pendingDeletions); + return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress, null); } private RepositoryMetadata getRepoMetadata(ClusterState state) { From fce436543320a8b2e610280cb8328223e49c3fbd Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 12 Jan 2022 17:25:02 +0100 Subject: [PATCH 26/42] test assertions --- .../metadata/MetadataDeleteIndexServiceTests.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java index 92abd49e005a7..312fa99bd5f6c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexServiceTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.snapshots.SearchableSnapshotsSettings; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotId; @@ -46,6 +47,8 @@ import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.mockito.ArgumentMatchers.any; @@ -223,13 +226,14 @@ public void testDeleteCurrentWriteIndexForDataStream() { public void testDeleteIndexWithSnapshotDeletion() { final boolean deleteSnapshot = randomBoolean(); + final boolean knownRepositoryUuid = randomBoolean(); final IndexMetadata indexMetadata = IndexMetadata.builder("test") .settings( Settings.builder() .put("index.version.created", VersionUtils.randomVersion(random())) .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOT_STORE_TYPE) .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY, "repo_name") - .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, randomBoolean() ? null : "repo_uuid") + .put(SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY, knownRepositoryUuid ? "repo_uuid" : null) .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_NAME_SETTING_KEY, "snap_name") .put(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY, "snap_uuid") .put(SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, deleteSnapshot) @@ -260,6 +264,10 @@ public void testDeleteIndexWithSnapshotDeletion() { if (deleteSnapshot) { assertThat(updatedPendingDeletions, notNullValue()); assertThat(updatedPendingDeletions.isEmpty(), equalTo(false)); + assertThat(updatedPendingDeletions.entries(), hasSize(1)); + SnapshotDeletionsPending.Entry entry = updatedPendingDeletions.entries().get(0); + assertThat(entry.getRepositoryName(), equalTo("repo_name")); + assertThat(entry.getRepositoryUuid(), knownRepositoryUuid ? equalTo("repo_uuid") : equalTo(RepositoryData.MISSING_UUID)); assertThat(updatedPendingDeletions.contains(new SnapshotId("snap_name", "snap_uuid")), equalTo(true)); } else { assertThat(updatedPendingDeletions, nullValue()); @@ -323,7 +331,12 @@ public void testDeleteMultipleIndicesWithSnapshotDeletion() { } assertThat(pendingDeletions.isEmpty(), equalTo(false)); + assertThat(pendingDeletions.entries(), hasSize(1)); assertThat(pendingDeletions.contains(snapshotId), equalTo(true)); + SnapshotDeletionsPending.Entry entry = pendingDeletions.entries().get(0); + assertThat(entry.getRepositoryName(), equalTo(repositoryMetadata.name())); + assertThat(entry.getRepositoryUuid(), anyOf(equalTo(repositoryMetadata.uuid()), equalTo(RepositoryData.MISSING_UUID))); + assertThat(entry.getSnapshotId(), equalTo(snapshotId)); } private ClusterState clusterState(String index) { From df1ec28a06bec4a77d06554cf274348ca9a724fd Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Jan 2022 11:30:04 +0100 Subject: [PATCH 27/42] clone test --- .../snapshots/RestoreService.java | 23 ++------------ .../snapshots/SnapshotUtils.java | 31 +++++++++++++++++++ .../snapshots/SnapshotsService.java | 17 ++-------- ...leSnapshotsPendingDeletionsIntegTests.java | 20 +++++++++--- 4 files changed, 50 insertions(+), 41 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 542f735bca77c..8ccfc550a7955 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -104,6 +104,7 @@ import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_NAME_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_REPOSITORY_UUID_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY; +import static org.elasticsearch.snapshots.SnapshotUtils.ensureSnapshotNotDeletedOrPendingDeletion; import static org.elasticsearch.snapshots.SnapshotUtils.filterIndices; import static org.elasticsearch.snapshots.SnapshotsService.NO_FEATURE_STATES_VALUE; @@ -1259,7 +1260,7 @@ private final class RestoreSnapshotStateTask extends ClusterStateUpdateTask { @Override public ClusterState execute(ClusterState currentState) { // Check if the snapshot to restore is currently being deleted - ensureSnapshotNotDeletedOrPendingDeletion(currentState); + ensureSnapshotNotDeletedOrPendingDeletion(currentState, snapshot.getRepository(), snapshot.getSnapshotId(), "restore"); // Clear out all existing indices which fall within a system index pattern being restored currentState = metadataDeleteIndexService.deleteIndices( @@ -1427,26 +1428,6 @@ private void applyDataStreamRestores(ClusterState currentState, Metadata.Builder mdBuilder.dataStreams(updatedDataStreams, updatedDataStreamAliases); } - private void ensureSnapshotNotDeletedOrPendingDeletion(ClusterState currentState) { - SnapshotDeletionsInProgress deletionsInProgress = currentState.custom( - SnapshotDeletionsInProgress.TYPE, - SnapshotDeletionsInProgress.EMPTY - ); - if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(snapshot.getSnapshotId()))) { - throw new ConcurrentSnapshotExecutionException( - snapshot, - "cannot restore a snapshot while a snapshot deletion is in-progress [" + deletionsInProgress.getEntries().get(0) + "]" - ); - } - SnapshotDeletionsPending pendingDeletions = currentState.custom(SnapshotDeletionsPending.TYPE); - if (pendingDeletions != null && pendingDeletions.contains(snapshot.getSnapshotId())) { - throw new ConcurrentSnapshotExecutionException( - snapshot, - "cannot restore a snapshot already marked as deleted [" + snapshot.getSnapshotId() + "]" - ); - } - } - private void applyGlobalStateRestore(ClusterState currentState, Metadata.Builder mdBuilder) { if (metadata.persistentSettings() != null) { Settings settings = metadata.persistentSettings(); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java index ab5f1f4ea9f26..47498c3292aa1 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java @@ -8,6 +8,9 @@ package org.elasticsearch.snapshots; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexNotFoundException; @@ -108,4 +111,32 @@ public static List filterIndices(List availableIndices, String[] return List.copyOf(result); } + static void ensureSnapshotNotDeletedOrPendingDeletion( + final ClusterState currentState, + final String repositoryName, + final SnapshotId snapshotId, + final String reason + ) { + final SnapshotDeletionsPending pendingDeletions = currentState.custom(SnapshotDeletionsPending.TYPE); + if (pendingDeletions != null && pendingDeletions.contains(snapshotId)) { + throw new ConcurrentSnapshotExecutionException( + repositoryName, + snapshotId.getName(), + "cannot " + reason + " a snapshot already marked as deleted [" + repositoryName + ":" + snapshotId + "]" + ); + } + final SnapshotDeletionsInProgress deletionsInProgress = currentState.custom(SnapshotDeletionsInProgress.TYPE); + if (deletionsInProgress != null + && deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(snapshotId))) { + throw new ConcurrentSnapshotExecutionException( + repositoryName, + snapshotId.getName(), + "cannot " + + reason + + " a snapshot while a snapshot deletion is in-progress [" + + deletionsInProgress.getEntries().get(0) + + "]" + ); + } + } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 96767de29efb6..89f5348ef3c89 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -121,6 +121,7 @@ import static java.util.Collections.unmodifiableList; import static org.elasticsearch.cluster.SnapshotsInProgress.completed; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; +import static org.elasticsearch.snapshots.SnapshotUtils.ensureSnapshotNotDeletedOrPendingDeletion; /** * Service responsible for creating snapshots. This service runs all the steps executed on the master node during snapshot creation and @@ -502,21 +503,7 @@ public ClusterState execute(ClusterState currentState) { SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY ); - if (deletionsInProgress.getEntries().stream().anyMatch(entry -> entry.getSnapshots().contains(sourceSnapshotId))) { - throw new ConcurrentSnapshotExecutionException( - repositoryName, - sourceSnapshotId.getName(), - "cannot clone from snapshot that is being deleted" - ); - } - if (currentState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY).contains(sourceSnapshotId)) { - throw new ConcurrentSnapshotExecutionException( - repositoryName, - sourceSnapshotId.getName(), - "cannot clone a snapshot already marked as deleted [" + snapshot.getSnapshotId() + "]" - ); - } - + ensureSnapshotNotDeletedOrPendingDeletion(currentState, repositoryName, sourceSnapshotId, "clone"); ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); final List indicesForSnapshot = new ArrayList<>(); for (IndexId indexId : repositoryData.getIndices().values()) { diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 593f406b5f597..70eef07f778bc 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -64,7 +64,7 @@ public class SearchableSnapshotsPendingDeletionsIntegTests extends BaseFrozenSearchableSnapshotsIntegTestCase { public void testSnapshotPendingDeletionCannotBeMounted() throws Exception { - blockPendingDeletionThenExecute((repository, snapshot, index) -> { + blockSnapshotDeletionThenExecute((repository, snapshot, index) -> { ConcurrentSnapshotExecutionException exception = expectThrows( ConcurrentSnapshotExecutionException.class, () -> mountSnapshot(repository, snapshot.getName(), index, Settings.EMPTY) @@ -74,7 +74,7 @@ public void testSnapshotPendingDeletionCannotBeMounted() throws Exception { } public void testSnapshotPendingDeletionCannotBeRestored() throws Exception { - blockPendingDeletionThenExecute((repository, snapshot, index) -> { + blockSnapshotDeletionThenExecute((repository, snapshot, index) -> { ConcurrentSnapshotExecutionException exception = expectThrows( ConcurrentSnapshotExecutionException.class, () -> client().admin().cluster().prepareRestoreSnapshot(repository, snapshot.getName()).setWaitForCompletion(true).get() @@ -83,6 +83,16 @@ public void testSnapshotPendingDeletionCannotBeRestored() throws Exception { }); } + public void testSnapshotPendingDeletionCannotBeCloned() throws Exception { + blockSnapshotDeletionThenExecute((repository, snapshot, index) -> { + ConcurrentSnapshotExecutionException exception = expectThrows( + ConcurrentSnapshotExecutionException.class, + () -> client().admin().cluster().prepareCloneSnapshot(repository, snapshot.getName(), "target").setIndices("*").get() + ); + assertThat(exception.getMessage(), containsString("cannot clone a snapshot already marked as deleted")); + }); + } + public void testSearchableSnapshotIsDeletedWhenRepoIsRecreated() throws Exception { mountIndexThenExecute((repository, snapshot, index) -> { try { @@ -525,16 +535,16 @@ private void mountIndexThenExecute(final TriConsumer test.apply(repository, snapshotId, index); } - private void blockPendingDeletionThenExecute(final TriConsumer test) throws Exception { + private void blockSnapshotDeletionThenExecute(final TriConsumer test) throws Exception { mountIndexThenExecute((repository, snapshot, index) -> { try { - updateRepositoryReadOnly(repository, true); + blockMasterOnWriteIndexFile(repository); assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); test.apply(repository, snapshot, index); - updateRepositoryReadOnly(repository, false); + unblockNode(repository, internalCluster().getMasterName()); awaitNoMoreSnapshotsDeletions(); expectThrows( From 2d8b32a649e777a273253716bbaccc8303f462fe Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Jan 2022 13:00:43 +0100 Subject: [PATCH 28/42] comment --- .../SearchableSnapshotsPendingDeletionsIntegTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 70eef07f778bc..7b86e6c941b9f 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -321,7 +321,9 @@ public void onFailure(Exception e) { mountLatch.await(); if (randomBoolean()) { - // we can't delete a repository that is used by searchable snapshot indices but we can update it + // Force the re-registration of the repository with randomized value for the "verify" flag; + // it helps to test the case where the repository UUID is unknown at the time the snapshot + // is marked to be deleted. assertAcked( clusterAdmin().preparePutRepository(repository) .setType(FsRepository.TYPE) From 025952e7c763b2b3225b5b09d37134219e58c9f6 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Jan 2022 13:05:31 +0100 Subject: [PATCH 29/42] randomInt() --- .../SearchableSnapshotsPendingDeletionsIntegTests.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 7b86e6c941b9f..c29153de1d445 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -40,7 +40,6 @@ import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; -import java.util.ArrayList; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -234,8 +233,8 @@ public void testSearchableSnapshotsDeletionsWithConcurrentDeletes() throws Excep final Settings.Builder repositorySettings = randomRepositorySettings(); createRepository(repository, FsRepository.TYPE, repositorySettings); - final List indices = new ArrayList<>(); - for (int i = 0; i < randomIntBetween(1, 10); i++) { + final String[] indices = new String[randomIntBetween(1, 10)]; + for (int i = 0; i < indices.length; i++) { final String index = "index-" + i; assertAcked( prepareCreate( @@ -246,7 +245,7 @@ public void testSearchableSnapshotsDeletionsWithConcurrentDeletes() throws Excep .put(INDEX_SOFT_DELETES_SETTING.getKey(), true) ) ); - indices.add(index); + indices[i] = index; } final int nbSnapshots = randomIntBetween(1, 10); From ad62175111bfbf16472c604c66e44520ce90fb9a Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Jan 2022 13:16:35 +0100 Subject: [PATCH 30/42] randomSubsetOf() --- .../SearchableSnapshotsPendingDeletionsIntegTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index c29153de1d445..f1fa158b0eab1 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -356,7 +356,7 @@ public void onFailure(Exception e) { startThreads.await(); final Set mountedIndices = new HashSet<>(mounts.keySet()); do { - List deletions = randomSubsetOf(Math.max(1, mountedIndices.size() - 1), mountedIndices); + List deletions = randomSubsetOf(randomIntBetween(1, mountedIndices.size()), mountedIndices); assertAcked(client().admin().indices().prepareDelete(deletions.toArray(String[]::new))); deletions.forEach(mountedIndices::remove); } while (mountedIndices.isEmpty() == false); From 350ab86278ecf309f48e86ade7488632dd55e76a Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Jan 2022 16:22:04 +0100 Subject: [PATCH 31/42] add missing setting --- .../xpack/searchablesnapshots/SearchableSnapshots.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java index 554eb09cd4b40..06b245c1c0319 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshots.java @@ -15,6 +15,7 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterStateListener; +import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.RepositoriesMetadata; @@ -313,7 +314,8 @@ public List> getSettings() { BlobStoreCacheMaintenanceService.SNAPSHOT_SNAPSHOT_CLEANUP_INTERVAL_SETTING, BlobStoreCacheMaintenanceService.SNAPSHOT_SNAPSHOT_CLEANUP_KEEP_ALIVE_SETTING, BlobStoreCacheMaintenanceService.SNAPSHOT_SNAPSHOT_CLEANUP_BATCH_SIZE_SETTING, - BlobStoreCacheMaintenanceService.SNAPSHOT_SNAPSHOT_CLEANUP_RETENTION_PERIOD + BlobStoreCacheMaintenanceService.SNAPSHOT_SNAPSHOT_CLEANUP_RETENTION_PERIOD, + SnapshotDeletionsPending.MAX_PENDING_DELETIONS_SETTING ); } From d9d711ebf0a583e9d45e14641f8514ed55841928 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 14 Jan 2022 18:11:36 +0100 Subject: [PATCH 32/42] add SnapshotDeletionsPendingExecutor --- .../common/settings/ClusterSettings.java | 5 +- .../snapshots/RestoreService.java | 2 - .../SnapshotDeletionsPendingExecutor.java | 454 ++++++++++++++++++ .../snapshots/SnapshotUtils.java | 26 + .../snapshots/SnapshotsService.java | 415 +--------------- ...SnapshotDeletionsPendingExecutorTests.java | 415 ++++++++++++++++ ...leSnapshotsPendingDeletionsIntegTests.java | 49 +- 7 files changed, 949 insertions(+), 417 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java create mode 100644 server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 4eeecdd62cf2b..4a8f0ee332e91 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -99,6 +99,7 @@ import org.elasticsearch.search.fetch.subphase.highlight.FastVectorHighlighter; import org.elasticsearch.snapshots.InternalSnapshotsInfoService; import org.elasticsearch.snapshots.RestoreService; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor; import org.elasticsearch.snapshots.SnapshotsService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ProxyConnectionStrategy; @@ -489,8 +490,8 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, - SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, - SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, + SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, + SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, RestoreService.REFRESH_REPO_UUID_ON_RESTORE_SETTING, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 8ccfc550a7955..5c54a9cf68143 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -23,8 +23,6 @@ import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.RestoreInProgress.ShardRestoreStatus; -import org.elasticsearch.cluster.SnapshotDeletionsInProgress; -import org.elasticsearch.cluster.SnapshotDeletionsPending; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.AliasMetadata; import org.elasticsearch.cluster.metadata.DataStream; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java new file mode 100644 index 0000000000000..a6890855985b6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java @@ -0,0 +1,454 @@ +/* + * 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.snapshots; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.RepositoryCleanupInProgress; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.metadata.RepositoriesMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.service.ClusterApplierService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.threadpool.ThreadPool; + +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; +import static org.elasticsearch.snapshots.SnapshotUtils.cloneSources; +import static org.elasticsearch.snapshots.SnapshotUtils.deletionsSources; +import static org.elasticsearch.snapshots.SnapshotUtils.restoreSources; +import static org.elasticsearch.snapshots.SnapshotsService.findRepositoryForPendingDeletion; + +public class SnapshotDeletionsPendingExecutor { + + private static final Logger logger = LogManager.getLogger(SnapshotDeletionsPendingExecutor.class); + + public static final Setting PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING = Setting.timeSetting( + "snapshot.snapshot_deletions_pending.retry_interval", + TimeValue.timeValueSeconds(30L), + TimeValue.ZERO, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + public static final Setting PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING = Setting.timeSetting( + "snapshot.snapshot_deletions_pending.expiration_interval", + TimeValue.timeValueHours(12L), + TimeValue.ZERO, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + /** + * Set of pending snapshots deletions whose deletion is already triggered + */ + private final Set triggered = ConcurrentCollections.newConcurrentSet(); + + /** + * Map of pending snapshots deletions whose deletion is conflicting with on-going restores/clones/repository clean up or repository + * missing or read-only. Those sets are used to identify the cluster state updates to process in case they resolve some conflict. + */ + private final Map conflicting = new HashMap<>(); + + /** + * Counters for the type of conflicts for the current set of conflicting pending snapshots deletions. This is used to look for + * updates in cluster state updates only when it is really needed. + */ + enum ConflictType { + RESTORING, + CLONING, + REPO_MISSING, + REPO_READONLY, + REPO_CLEANUP + } + + private final SnapshotsService snapshotsService; + private final ClusterService clusterService; + private final ThreadPool threadPool; + + private volatile TimeValue pendingDeletionsRetryInterval; + private volatile TimeValue pendingDeletionsExpirationInterval; + + SnapshotDeletionsPendingExecutor( + SnapshotsService snapshotsService, + ClusterService clusterService, + ThreadPool threadPool, + Settings settings + ) { + this.snapshotsService = Objects.requireNonNull(snapshotsService); + this.clusterService = Objects.requireNonNull(clusterService); + this.threadPool = Objects.requireNonNull(threadPool); + pendingDeletionsRetryInterval = PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.get(settings); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, t -> pendingDeletionsRetryInterval = t); + pendingDeletionsExpirationInterval = PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.get(settings); + clusterService.getClusterSettings() + .addSettingsUpdateConsumer(PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, t -> pendingDeletionsExpirationInterval = t); + } + + /** + * Find snapshots to delete in the the cluster state and triggers explicit snapshot delete requests. This method attempts to detect + * conflicting situations where triggering the snapshot deletion would likely fail due to a concurrent snapshot operation. In such + * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates once the conflicting + * situation is resolved. + * + * The repository name and uuid information are extracted from the {@link SnapshotDeletionsPending} entries in order to find the + * repository to execute the snapshot delete request against. If the repo uuid was known at the time the snapshot was added to + * {@link SnapshotDeletionsPending} we try to find the corresponding repository, or a repository with a missing uuid but the same + * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsPending}, we try to find a + * repository with the same name. + * + * @param state the current {@link ClusterState} + * @param previousState the previous {@link ClusterState} + */ + public synchronized void processPendingDeletions(ClusterState state, ClusterState previousState) { + assert Thread.currentThread().getName().contains('[' + ClusterApplierService.CLUSTER_UPDATE_THREAD_NAME + ']') + || Thread.currentThread().getName().startsWith("TEST-") : Thread.currentThread().getName(); + + final SnapshotDeletionsPending snapshotDeletionsPending = state.custom(SnapshotDeletionsPending.TYPE); + if (state.nodes().isLocalNodeElectedMaster() == false || snapshotDeletionsPending == null || snapshotDeletionsPending.isEmpty()) { + clearConflicts(); + return; + } + + if (pendingDeletionsChanged(state, previousState) || pendingDeletionsWithConflictsChanged(state, previousState)) { + final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + final RepositoryCleanupInProgress cleanUps = state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + + final Set currentDeletions = deletionsSources(state); + final Set currentRestores = restoreSources(state); + final Set currentClones = cloneSources(state); + + // the snapshots to trigger deletion for, per repository + final Map> snapshotsToDelete = new HashMap<>(); + + for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { + final SnapshotId snapshotId = snapshot.getSnapshotId(); + + if (currentRestores.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); + conflicting.put(snapshotId, ConflictType.RESTORING); + continue; + } + if (currentClones.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); + conflicting.put(snapshotId, ConflictType.CLONING); + continue; + } + if (cleanUps.hasCleanupInProgress()) { + if (conflicting.put(snapshotId, ConflictType.REPO_CLEANUP) != ConflictType.REPO_CLEANUP) { + logger.debug( + "a repository clean-up is in progress, cannot delete pending snapshot [{}] created at {}", + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + if (currentDeletions.contains(snapshotId)) { + logger.trace("snapshot to delete [{}] is already queued", snapshotId); + conflicting.remove(snapshotId); + continue; + } + + final Optional optionalRepository = findRepositoryForPendingDeletion( + repositories, + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid() + ); + if (optionalRepository.isEmpty()) { + if (conflicting.put(snapshotId, ConflictType.REPO_MISSING) != ConflictType.REPO_MISSING) { + logger.debug( + "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", + snapshot.getRepositoryName(), + snapshot.getRepositoryUuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + final RepositoryMetadata repository = optionalRepository.get(); + if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { + if (conflicting.put(snapshotId, ConflictType.REPO_READONLY) != ConflictType.REPO_READONLY) { + logger.debug( + "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", + repository.name(), + repository.uuid(), + snapshotId, + Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) + ); + } + continue; + } + conflicting.remove(snapshotId); + + if (triggered.add(snapshotId)) { + logger.info("triggering snapshot deletion for [{}]", snapshotId); + final Long previous = snapshotsToDelete.computeIfAbsent(repository, r -> new HashMap<>()) + .put(snapshotId, snapshot.getIndexDeletionTime()); + assert previous == null : snapshotId; + } + } + + assert snapshotDeletionsPending.entries() + .stream() + .map(SnapshotDeletionsPending.Entry::getSnapshotId) + .allMatch(snapId -> triggered.contains(snapId) || conflicting.containsKey(snapId) || currentDeletions.contains(snapId)); + + snapshotsToDelete.forEach( + (repo, snapshots) -> threadPool.generic().execute(new SnapshotsToDeleteRunnable(repo.name(), repo.uuid(), snapshots)) + ); + } + assert Sets.intersection(triggered, conflicting.keySet()).isEmpty() + : "pending snapshot deletion cannot be both triggered and in conflict: " + triggered + " vs " + conflicting.keySet(); + assert conflicting.keySet().stream().allMatch(snapshotDeletionsPending::contains); + } + + // only used in tests + boolean isTriggered(SnapshotId snapshotId) { + return triggered.contains(snapshotId); + } + + // only used in tests + synchronized ConflictType getConflict(SnapshotId snapshotId) { + return conflicting.get(snapshotId); + } + + synchronized void clearConflicts() { + if (conflicting.isEmpty() == false) { + conflicting.clear(); + } + } + + // package-private for testing + synchronized boolean pendingDeletionsChanged(ClusterState state, ClusterState previousState) { + SnapshotDeletionsPending previous = previousState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + SnapshotDeletionsPending current = state.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + return Objects.equals(previous, current) == false; + } + + // package-private for testing + synchronized boolean pendingDeletionsWithConflictsChanged(ClusterState state, ClusterState previousState) { + if (conflicting.values().stream().anyMatch(c -> c == ConflictType.RESTORING)) { + RestoreInProgress previous = previousState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); + RestoreInProgress current = state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); + if (Objects.equals(previous, current) == false) { + return true; + } + } + if (conflicting.values().stream().anyMatch(c -> c == ConflictType.CLONING)) { + Set previous = previousState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .collect(Collectors.toSet()); + Set current = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .collect(Collectors.toSet()); + if (Objects.equals(previous, current) == false) { + return true; + } + } + if (conflicting.values().stream().anyMatch(c -> c == ConflictType.REPO_MISSING || c == ConflictType.REPO_READONLY)) { + RepositoriesMetadata previous = previousState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + RepositoriesMetadata current = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + if (previous.equals(current) == false) { + return true; + } + } + if (conflicting.values().stream().anyMatch(c -> c == ConflictType.REPO_CLEANUP)) { + boolean previousCleanUp = previousState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) + .hasCleanupInProgress(); + boolean currentCleanUp = state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) + .hasCleanupInProgress(); + return previousCleanUp != currentCleanUp; + } + return false; + } + + private boolean isExpiredPendingDeletion(long deletionTimeEpochMillis) { + return Instant.ofEpochMilli(deletionTimeEpochMillis) + .plusMillis(pendingDeletionsExpirationInterval.getMillis()) + .isBefore(Instant.now()); + } + + /** + * A {@link Runnable} used to process the deletion of snapshots marked as to delete for a given repository. + */ + private class SnapshotsToDeleteRunnable extends AbstractRunnable { + + private final Map snapshots; + private final String repositoryName; + private final String repositoryUuid; + private final boolean missingUuid; + + SnapshotsToDeleteRunnable(String repositoryName, String repositoryUuid, Map snapshots) { + this.repositoryName = Objects.requireNonNull(repositoryName); + this.repositoryUuid = Objects.requireNonNull(repositoryUuid); + this.snapshots = Objects.requireNonNull(snapshots); + this.missingUuid = RepositoryData.MISSING_UUID.equals(repositoryUuid); + } + + @Override + protected void doRun() throws Exception { + final Set pendingDeletionsToRemove = ConcurrentCollections.newConcurrentSet(); + final CountDown countDown = new CountDown(snapshots.size()); + + for (Map.Entry snapshot : snapshots.entrySet()) { + final SnapshotId snapshotId = snapshot.getKey(); + final ActionListener listener = new ActionListener() { + @Override + public void onResponse(Void unused) { + logger.debug( + "snapshot marked as to delete [{}] successfully deleted from repository [{}/{}]", + snapshotId, + repositoryName, + repositoryUuid + ); + pendingDeletionsToRemove.add(snapshotId); + finish(); + } + + @Override + public void onFailure(Exception e) { + if (e instanceof SnapshotMissingException && missingUuid == false) { + pendingDeletionsToRemove.add(snapshotId); + logger.debug( + () -> new ParameterizedMessage( + "snapshot marked as to delete [{}] is missing in repository [{}/{}], removing from pending deletions", + snapshotId, + repositoryName, + repositoryUuid + ), + e + ); + } else if (isExpiredPendingDeletion(snapshot.getValue())) { + pendingDeletionsToRemove.add(snapshotId); + logger.warn( + () -> new ParameterizedMessage( + "snapshot marked as to delete [{}] failed to be deleted within [{}]. The pending snapshot " + + "expired before the snapshot could be deleted from the repository and as such might still " + + "exist in the original repository [{}/{}]. This snapshot will now be removed from the list of " + + "pending deletions.", + snapshotId, + pendingDeletionsExpirationInterval, + repositoryName, + repositoryUuid + ), + e + ); + } else { + logger.debug( + () -> new ParameterizedMessage( + "[{}/{}] attempt to delete snapshot marked as to delete [{}] failed; deletion will be retried in [{}]", + repositoryName, + repositoryUuid, + snapshotId, + pendingDeletionsRetryInterval + ), + e + ); + } + finish(); + } + + void finish() { + if (countDown.countDown()) { + final Map retryables = snapshots.entrySet() + .stream() + .filter(snap -> pendingDeletionsToRemove.contains(snap.getKey()) == false) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + if (retryables.isEmpty() == false) { + // TODO maybe re-resolve repository here if the uuid is missing? + threadPool.scheduleUnlessShuttingDown( + pendingDeletionsRetryInterval, + ThreadPool.Names.GENERIC, + new SnapshotsToDeleteRunnable(repositoryName, repositoryUuid, retryables) + ); + } + if (pendingDeletionsToRemove.isEmpty() == false) { + clusterService.submitStateUpdateTask("remove-snapshot-deletions-in-pending", new ClusterStateUpdateTask() { + @Override + public ClusterState execute(ClusterState currentState) { + final SnapshotDeletionsPending currentPendings = currentState.custom( + SnapshotDeletionsPending.TYPE, + SnapshotDeletionsPending.EMPTY + ); + final SnapshotDeletionsPending updatedPendings = currentPendings.withRemovedSnapshots( + List.copyOf(pendingDeletionsToRemove) + ); + if (currentPendings == updatedPendings) { + return currentState; + } + return ClusterState.builder(currentState) + .putCustom(SnapshotDeletionsPending.TYPE, updatedPendings) + .build(); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + triggered.removeAll(pendingDeletionsToRemove); + } + + @Override + public void onFailure(String source, Exception e) { + triggered.removeAll(pendingDeletionsToRemove); + } + }); + } + } + } + }; + + try { + snapshotsService.deleteSnapshotsByUuid(repositoryName, new String[] { snapshotId.getUUID() }, listener); + } catch (Exception e) { + logger.warn( + () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshot [{}]", repositoryName, snapshotId), + e + ); + listener.onFailure(e); + } + } + } + + @Override + public void onFailure(Exception e) { + triggered.removeAll(snapshots.keySet()); + logger.warn( + () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshots {}", repositoryName, snapshots.keySet()), + e + ); + } + } +} diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java index 47498c3292aa1..8e6141770e997 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotUtils.java @@ -9,8 +9,10 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.SnapshotDeletionsInProgress; import org.elasticsearch.cluster.SnapshotDeletionsPending; +import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.index.IndexNotFoundException; @@ -19,6 +21,8 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * Snapshot utilities @@ -111,6 +115,28 @@ public static List filterIndices(List availableIndices, String[] return List.copyOf(result); } + static Set cloneSources(final ClusterState state) { + return state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .asStream() + .filter(SnapshotsInProgress.Entry::isClone) + .map(SnapshotsInProgress.Entry::source) + .collect(Collectors.toUnmodifiableSet()); + } + + static Set restoreSources(final ClusterState state) { + return StreamSupport.stream(state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).spliterator(), false) + .map(restore -> restore.snapshot().getSnapshotId()) + .collect(Collectors.toUnmodifiableSet()); + } + + static Set deletionsSources(final ClusterState state) { + return state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY) + .getEntries() + .stream() + .flatMap(deletion -> deletion.getSnapshots().stream()) + .collect(Collectors.toUnmodifiableSet()); + } + static void ensureSnapshotNotDeletedOrPendingDeletion( final ClusterState currentState, final String repositoryName, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 89f5348ef3c89..2d8055ea70f1e 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -64,9 +64,6 @@ import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; 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.CountDown; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.core.Tuple; @@ -89,8 +86,6 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; -import java.time.Instant; -import java.time.ZoneOffset; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -116,11 +111,10 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; import static java.util.Collections.unmodifiableList; import static org.elasticsearch.cluster.SnapshotsInProgress.completed; -import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; +import static org.elasticsearch.snapshots.SnapshotUtils.cloneSources; import static org.elasticsearch.snapshots.SnapshotUtils.ensureSnapshotNotDeletedOrPendingDeletion; /** @@ -181,6 +175,8 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final SystemIndices systemIndices; + private final SnapshotDeletionsPendingExecutor snapshotPendingDeletions; + /** * Setting that specifies the maximum number of allowed concurrent snapshot create and delete operations in the * cluster state. The number of concurrent operations in a cluster state is defined as the sum of @@ -225,17 +221,9 @@ public SnapshotsService( maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); - pendingDeletionsRetryInterval = PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.get(settings); - clusterService.getClusterSettings() - .addSettingsUpdateConsumer(PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, t -> pendingDeletionsRetryInterval = t); - pendingDeletionsExpirationInterval = PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.get(settings); - clusterService.getClusterSettings() - .addSettingsUpdateConsumer( - PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, - t -> pendingDeletionsExpirationInterval = t - ); } this.systemIndices = systemIndices; + this.snapshotPendingDeletions = new SnapshotDeletionsPendingExecutor(this, clusterService, threadPool, settings); } /** @@ -589,28 +577,6 @@ private static void ensureSnapshotNameAvailableInRepo(RepositoryData repositoryD } } - private static Set cloneSources(final ClusterState state) { - return state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) - .asStream() - .filter(SnapshotsInProgress.Entry::isClone) - .map(SnapshotsInProgress.Entry::source) - .collect(Collectors.toUnmodifiableSet()); - } - - private static Set restoreSources(final ClusterState state) { - return StreamSupport.stream(state.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY).spliterator(), false) - .map(restore -> restore.snapshot().getSnapshotId()) - .collect(Collectors.toUnmodifiableSet()); - } - - private static Set deletionsSources(final ClusterState state) { - return state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY) - .getEntries() - .stream() - .flatMap(deletion -> deletion.getSnapshots().stream()) - .collect(Collectors.toUnmodifiableSet()); - } - /** * Determine the number of shards in each index of a clone operation and update the cluster state accordingly. * @@ -1015,7 +981,7 @@ public void applyClusterState(ClusterChangedEvent event) { } } } - triggerSnapshotsPendingDeletions(event); + snapshotPendingDeletions.processPendingDeletions(event.state(), event.previousState()); } catch (Exception e) { assert false : new AssertionError(e); logger.warn("Failed to update snapshot state ", e); @@ -1303,373 +1269,14 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS ); } - /** - * Set of pending snapshots deletions whose deletion is already triggered - */ - private final Set ongoingSnapshotsDeletions = ConcurrentCollections.newConcurrentSet(); - // only used in tests - public boolean hasOngoingSnapshotsDeletions(SnapshotId snapshotId) { - return ongoingSnapshotsDeletions.contains(snapshotId); + public boolean isSnapshotPendingDeletionTriggered(SnapshotId snapshotId) { + return snapshotPendingDeletions.isTriggered(snapshotId); } - /** - * Set of pending snapshots deletions whose deletion is conflicting with on-going restores/clones/repository clean up or repository - * missing or read-only. Those sets are used to identify the cluster state updates to process in case they resolve some conflict. - */ - private final Set pendingDeletionsWithConflictingRestores = ConcurrentCollections.newConcurrentSet(); - private final Set pendingDeletionsWithConflictingCleanUps = ConcurrentCollections.newConcurrentSet(); - private final Set pendingDeletionsWithConflictingClones = ConcurrentCollections.newConcurrentSet(); - private final Set pendingDeletionsWithConflictingRepos = ConcurrentCollections.newConcurrentSet(); - - /** - * Find snapshots to delete in the the cluster state and triggers explicit snapshot delete requests. This method attempts to detect - * conflicting situations where triggering the snapshot deletion would likely fail due to a concurrent snapshot operation. In such - * cases the snapshot deletion is not triggered as it should be triggered by subsequent cluster state updates once the conflicting - * situation is resolved. - * - * The repository name and uuid information are extracted from the {@link SnapshotDeletionsPending} entries in order to find the - * repository to execute the snapshot delete request against. If the repo uuid was known at the time the snapshot was added to - * {@link SnapshotDeletionsPending} we try to find the corresponding repository, or a repository with a missing uuid but the same - * name. If the repo uuid was not known at the time the snapshot was added to {@link SnapshotDeletionsPending}, we try to find a - * repository with the same name. - * - * @param event the current {@link ClusterChangedEvent} - */ - private void triggerSnapshotsPendingDeletions(final ClusterChangedEvent event) { - if (event.localNodeMaster() == false) { - if (event.previousState().nodes().isLocalNodeElectedMaster()) { - clearPendingDeletionsWithConflicts(); - } - return; - } - - if (pendingDeletionsChanged(event) || pendingDeletionsWithConflictsChanged(event)) { - final ClusterState state = event.state(); - final SnapshotDeletionsPending snapshotDeletionsPending = state.custom(SnapshotDeletionsPending.TYPE); - if (snapshotDeletionsPending == null || snapshotDeletionsPending.isEmpty()) { - clearPendingDeletionsWithConflicts(); - return; - } - - final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - final RepositoryCleanupInProgress cleanUps = state.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); - - final Set currentDeletions = deletionsSources(state); - final Set currentRestores = restoreSources(state); - final Set currentClones = cloneSources(state); - - // the snapshots to trigger deletion for, per repository - final Map> snapshotsToDelete = new HashMap<>(); - - for (SnapshotDeletionsPending.Entry snapshot : snapshotDeletionsPending.entries()) { - final SnapshotId snapshotId = snapshot.getSnapshotId(); - - if (currentRestores.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being restored, waiting for restore to complete", snapshotId); - pendingDeletionsWithConflictingRestores.add(snapshotId); - continue; - } - pendingDeletionsWithConflictingRestores.remove(snapshotId); - - if (currentClones.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is being cloned, waiting for cloning to complete", snapshotId); - pendingDeletionsWithConflictingClones.add(snapshotId); - continue; - } - pendingDeletionsWithConflictingClones.remove(snapshotId); - - if (cleanUps.hasCleanupInProgress()) { - if (pendingDeletionsWithConflictingCleanUps.add(snapshotId)) { - logger.debug( - "a repository clean-up is in progress, cannot delete pending snapshot [{}] created at {}", - snapshotId, - Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) - ); - } - continue; - } - pendingDeletionsWithConflictingCleanUps.remove(snapshotId); - - if (currentDeletions.contains(snapshotId)) { - logger.trace("snapshot to delete [{}] is already queued", snapshotId); - pendingDeletionsWithConflictingRepos.remove(snapshotId); - continue; - } - - final Optional optionalRepository = findRepositoryForPendingDeletion( - repositories, - snapshot.getRepositoryName(), - snapshot.getRepositoryUuid() - ); - if (optionalRepository.isEmpty()) { - if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { - logger.debug( - "repository [{}/{}] not found, cannot delete pending snapshot [{}] created at {}", - snapshot.getRepositoryName(), - snapshot.getRepositoryUuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) - ); - } - continue; - } - final RepositoryMetadata repository = optionalRepository.get(); - if (repository.settings().getAsBoolean(READONLY_SETTING_KEY, false)) { - if (pendingDeletionsWithConflictingRepos.add(snapshotId)) { - logger.debug( - "repository [{}/{}] is read-only, cannot delete pending snapshot [{}] created at {}", - repository.name(), - repository.uuid(), - snapshotId, - Instant.ofEpochMilli(snapshot.getIndexDeletionTime()).atZone(ZoneOffset.UTC) - ); - } - continue; - } - pendingDeletionsWithConflictingRepos.remove(snapshotId); - - if (ongoingSnapshotsDeletions.add(snapshotId)) { - logger.info("triggering snapshot deletion for [{}]", snapshotId); - final Long previous = snapshotsToDelete.computeIfAbsent(repository, r -> new HashMap<>()) - .put(snapshotId, snapshot.getIndexDeletionTime()); - assert previous == null : snapshotId; - } - } - snapshotsToDelete.forEach( - (repo, snapshots) -> threadPool.generic().execute(new SnapshotsToDeleteRunnable(repo.name(), repo.uuid(), snapshots)) - ); - } - } - - private void clearPendingDeletionsWithConflicts() { - pendingDeletionsWithConflictingRestores.clear(); - pendingDeletionsWithConflictingCleanUps.clear(); - pendingDeletionsWithConflictingClones.clear(); - pendingDeletionsWithConflictingRepos.clear(); - } - - private static boolean pendingDeletionsChanged(ClusterChangedEvent event) { - SnapshotDeletionsPending previous = event.previousState().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); - SnapshotDeletionsPending current = event.state().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); - return Objects.equals(previous, current) == false; - } - - private boolean pendingDeletionsWithConflictsChanged(ClusterChangedEvent event) { - if (pendingDeletionsWithConflictingRestores.isEmpty() == false) { - RestoreInProgress previous = event.previousState().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); - RestoreInProgress current = event.state().custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY); - if (Objects.equals(previous, current) == false) { - return true; - } - } - if (pendingDeletionsWithConflictingClones.isEmpty() == false) { - Set previous = event.previousState() - .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) - .asStream() - .filter(SnapshotsInProgress.Entry::isClone) - .collect(Collectors.toSet()); - Set current = event.state() - .custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) - .asStream() - .filter(SnapshotsInProgress.Entry::isClone) - .collect(Collectors.toSet()); - if (Objects.equals(previous, current) == false) { - return true; - } - } - if (pendingDeletionsWithConflictingRepos.isEmpty() == false) { - RepositoriesMetadata previous = event.previousState().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - RepositoriesMetadata current = event.state().metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); - if (previous.equals(current) == false) { - return true; - } - } - if (pendingDeletionsWithConflictingCleanUps.isEmpty() == false) { - boolean previousCleanUp = event.previousState() - .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) - .hasCleanupInProgress(); - boolean currentCleanUp = event.state() - .custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) - .hasCleanupInProgress(); - return previousCleanUp != currentCleanUp; - } - return false; - } - - public static final Setting PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING = Setting.timeSetting( - "snapshot.snapshot_deletions_pending.retry_interval", - TimeValue.timeValueSeconds(30L), - TimeValue.ZERO, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); - - public static final Setting PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING = Setting.timeSetting( - "snapshot.snapshot_deletions_pending.expiration_interval", - TimeValue.timeValueHours(12L), - TimeValue.ZERO, - Setting.Property.NodeScope, - Setting.Property.Dynamic - ); - - private volatile TimeValue pendingDeletionsRetryInterval; - private volatile TimeValue pendingDeletionsExpirationInterval; - - private boolean isExpiredPendingDeletion(long deletionTimeEpochMillis) { - return Instant.ofEpochMilli(deletionTimeEpochMillis) - .plusMillis(pendingDeletionsExpirationInterval.getMillis()) - .isBefore(Instant.now()); - } - - /** - * A {@link Runnable} used to process the deletion of snapshots marked as to delete for a given repository. - */ - private class SnapshotsToDeleteRunnable extends AbstractRunnable { - - private final Map snapshots; - private final String repositoryName; - private final String repositoryUuid; - private final boolean missingUuid; - - SnapshotsToDeleteRunnable(String repositoryName, String repositoryUuid, Map snapshots) { - this.repositoryName = Objects.requireNonNull(repositoryName); - this.repositoryUuid = Objects.requireNonNull(repositoryUuid); - this.snapshots = Objects.requireNonNull(snapshots); - this.missingUuid = RepositoryData.MISSING_UUID.equals(repositoryUuid); - } - - @Override - protected void doRun() throws Exception { - final Set pendingDeletionsToRemove = ConcurrentCollections.newConcurrentSet(); - final CountDown countDown = new CountDown(snapshots.size()); - - for (Map.Entry snapshot : snapshots.entrySet()) { - final SnapshotId snapshotId = snapshot.getKey(); - final ActionListener listener = new ActionListener() { - @Override - public void onResponse(Void unused) { - logger.debug( - "snapshot marked as to delete [{}] successfully deleted from repository [{}/{}]", - snapshotId, - repositoryName, - repositoryUuid - ); - pendingDeletionsToRemove.add(snapshotId); - finish(); - } - - @Override - public void onFailure(Exception e) { - if (e instanceof SnapshotMissingException && missingUuid == false) { - pendingDeletionsToRemove.add(snapshotId); - logger.debug( - () -> new ParameterizedMessage( - "snapshot marked as to delete [{}] is missing in repository [{}/{}], removing from pending deletions", - snapshotId, - repositoryName, - repositoryUuid - ), - e - ); - } else if (isExpiredPendingDeletion(snapshot.getValue())) { - pendingDeletionsToRemove.add(snapshotId); - logger.warn( - () -> new ParameterizedMessage( - "snapshot marked as to delete [{}] failed to be deleted within [{}]. The pending snapshot " - + "expired before the snapshot could be deleted from the repository and as such might still " - + "exist in the original repository [{}/{}]. This snapshot will now be removed from the list of " - + "pending deletions.", - snapshotId, - pendingDeletionsExpirationInterval, - repositoryName, - repositoryUuid - ), - e - ); - } else { - logger.debug( - () -> new ParameterizedMessage( - "[{}/{}] attempt to delete snapshot marked as to delete [{}] failed; deletion will be retried in [{}]", - repositoryName, - repositoryUuid, - snapshotId, - pendingDeletionsRetryInterval - ), - e - ); - } - finish(); - } - - void finish() { - if (countDown.countDown()) { - final Map retryables = snapshots.entrySet() - .stream() - .filter(snap -> pendingDeletionsToRemove.contains(snap.getKey()) == false) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - if (retryables.isEmpty() == false) { - // TODO maybe re-resolve repository here if the uuid is missing? - threadPool.scheduleUnlessShuttingDown( - pendingDeletionsRetryInterval, - ThreadPool.Names.GENERIC, - new SnapshotsToDeleteRunnable(repositoryName, repositoryUuid, retryables) - ); - } - if (pendingDeletionsToRemove.isEmpty() == false) { - clusterService.submitStateUpdateTask("remove-snapshot-deletions-in-pending", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsPending currentPendings = currentState.custom( - SnapshotDeletionsPending.TYPE, - SnapshotDeletionsPending.EMPTY - ); - final SnapshotDeletionsPending updatedPendings = currentPendings.withRemovedSnapshots( - List.copyOf(pendingDeletionsToRemove) - ); - if (currentPendings == updatedPendings) { - return currentState; - } - return ClusterState.builder(currentState) - .putCustom(SnapshotDeletionsPending.TYPE, updatedPendings) - .build(); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - ongoingSnapshotsDeletions.removeAll(pendingDeletionsToRemove); - } - - @Override - public void onFailure(String source, Exception e) { - ongoingSnapshotsDeletions.removeAll(pendingDeletionsToRemove); - } - }); - } - } - } - }; - - try { - deleteSnapshotsByUuid(repositoryName, new String[] { snapshotId.getUUID() }, listener); - } catch (Exception e) { - logger.warn( - () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshot [{}]", repositoryName, snapshotId), - e - ); - listener.onFailure(e); - } - } - } - - @Override - public void onFailure(Exception e) { - ongoingSnapshotsDeletions.removeAll(snapshots.keySet()); - logger.warn( - () -> new ParameterizedMessage("[{}] failed to trigger deletion of snapshots {}", repositoryName, snapshots.keySet()), - e - ); - } + // only used in tests + public boolean isSnapshotPendingDeletionConflicting(SnapshotId snapshotId) { + return snapshotPendingDeletions.getConflict(snapshotId) != null; } public static Optional findRepositoryForPendingDeletion( @@ -2494,7 +2101,7 @@ public void deleteSnapshotsByName(final DeleteSnapshotRequest request, final Act * @param snapshotUuids the uuids of the snapshots to delete * @param listener listener */ - private void deleteSnapshotsByUuid(final String repositoryName, final String[] snapshotUuids, final ActionListener listener) { + void deleteSnapshotsByUuid(final String repositoryName, final String[] snapshotUuids, final ActionListener listener) { deleteSnapshots(repositoryName, null, snapshotUuids, null, SnapshotId::getUUID, listener); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java new file mode 100644 index 0000000000000..c4e063a8ca1ce --- /dev/null +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java @@ -0,0 +1,415 @@ +/* + * 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.snapshots; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.RepositoryCleanupInProgress; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotDeletionsPending; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.metadata.Metadata; +import org.elasticsearch.cluster.metadata.RepositoriesMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor.ConflictType; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; +import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; +import static org.elasticsearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class SnapshotDeletionsPendingExecutorTests extends ESTestCase { + + private SnapshotsService snapshotsService; + private DeterministicTaskQueue taskQueue; + private ClusterService clusterService; + private TestThreadPool threadPool; + private SnapshotDeletionsPendingExecutor executor; + + @Before + @Override + @SuppressWarnings("unchecked") + public void setUp() throws Exception { + super.setUp(); + threadPool = new TestThreadPool(getTestName()); + clusterService = createClusterService(threadPool); + snapshotsService = mock(SnapshotsService.class); + doAnswer(invocation -> { + ((ActionListener) invocation.getArgument(2)).onResponse(null); + return null; + }).when(snapshotsService).deleteSnapshotsByUuid(any(), any(), any()); + taskQueue = new DeterministicTaskQueue(); + executor = new SnapshotDeletionsPendingExecutor(snapshotsService, clusterService, taskQueue.getThreadPool(), Settings.EMPTY); + clusterService.addStateApplier(event -> executor.processPendingDeletions(event.state(), event.previousState())); + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); + snapshotsService = null; + terminate(threadPool); + } + + public void testSnapshotDeletionsPendingsAreTriggered() throws Exception { + final RepositoryMetadata repository1 = randomRepository(); + setState(clusterService, stateWithRepository(emptyState(), repository1)); + + final SnapshotId snapshot1 = randomSnapshotId(); + ClusterState state = stateWithNewPending(clusterService.state(), repository1, snapshot1, 1); + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(true)); + + setState(clusterService, state); + + assertThat(executor.isTriggered(snapshot1), is(true)); + assertThat(taskQueue.hasRunnableTasks(), is(true)); + + runAllTasks(); + + assertBusy(() -> assertThat(executor.isTriggered(snapshot1), is(false))); + verify(snapshotsService, times(1)).deleteSnapshotsByUuid(eq(repository1.name()), any(), any()); + assertNoSnapshotDeletionsPendingsInClusterState(); + + final RepositoryMetadata repository2 = randomRepository(); + state = stateWithRepository(clusterService.state(), repository2); + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(false)); + + setState(clusterService, state); + + final SnapshotId snapshot2 = randomSnapshotId(); + state = stateWithNewPending(clusterService.state(), repository2, snapshot2, 2); + final SnapshotId snapshot3 = randomSnapshotId(); + state = stateWithNewPending(state, repository2, snapshot3, 3); + final SnapshotId snapshot4 = randomSnapshotId(); + state = stateWithNewPending(state, repository2, snapshot4, 4); + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(true)); + + setState(clusterService, state); + + assertThat(executor.isTriggered(snapshot2), is(true)); + assertThat(executor.isTriggered(snapshot3), is(true)); + assertThat(executor.isTriggered(snapshot4), is(true)); + assertThat(taskQueue.hasRunnableTasks(), is(true)); + + runAllTasks(); + + assertBusy(() -> assertThat(executor.isTriggered(snapshot2), is(false))); + assertThat(executor.isTriggered(snapshot3), is(false)); + assertThat(executor.isTriggered(snapshot4), is(false)); + verify(snapshotsService, times(3)).deleteSnapshotsByUuid(eq(repository2.name()), any(), any()); + assertNoSnapshotDeletionsPendingsInClusterState(); + } + + public void testSnapshotDeletionsPendingsWithRandomConflict() throws Exception { + final RepositoryMetadata repository = randomRepository(); + ClusterState state = stateWithRepository(emptyState(), repository); + + setState(clusterService, state); + + final SnapshotId snapshot = randomSnapshotId(); + state = stateWithNewPending(state, repository, snapshot, 1L); + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(true)); + assertThat(executor.pendingDeletionsWithConflictsChanged(state, clusterService.state()), is(false)); + + final ConflictType conflict = randomFrom(ConflictType.values()); + state = addConflict(conflict, state, repository, snapshot); + + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(true)); + assertThat(executor.pendingDeletionsWithConflictsChanged(state, clusterService.state()), is(false)); + + setState(clusterService, state); + + assertThat(executor.getConflict(snapshot), equalTo(conflict)); + assertThat(executor.isTriggered(snapshot), is(false)); + assertThat(taskQueue.hasRunnableTasks(), is(false)); + + state = resolveConflict(conflict, state, repository, snapshot); + assertThat(executor.pendingDeletionsChanged(state, clusterService.state()), is(false)); + assertThat(executor.pendingDeletionsWithConflictsChanged(state, clusterService.state()), is(true)); + + setState(clusterService, state); + + assertThat(executor.isTriggered(snapshot), is(true)); + assertThat(taskQueue.hasRunnableTasks(), is(true)); + + runAllTasks(); + + assertBusy(() -> assertThat(executor.isTriggered(snapshot), is(false))); + verify(snapshotsService, times(1)).deleteSnapshotsByUuid(any(), any(), any()); + assertNoSnapshotDeletionsPendingsInClusterState(); + } + + private void runAllTasks() { + taskQueue.runAllTasks(); + assertThat(taskQueue.hasRunnableTasks(), is(false)); + } + + private void assertNoSnapshotDeletionsPendingsInClusterState() { + SnapshotDeletionsPending current = clusterService.state().custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + assertThat(Strings.toString(current), current.isEmpty(), is(true)); + } + + private static ClusterState emptyState() { + return ClusterState.builder(ClusterState.EMPTY_STATE) + .nodes(DiscoveryNodes.builder().localNodeId("_node").masterNodeId("_node")) + .metadata(Metadata.builder().generateClusterUuidIfNeeded()) + .build(); + } + + private static RepositoryMetadata randomRepository() { + return new RepositoryMetadata( + randomAlphaOfLength(10).toLowerCase(Locale.ROOT), + UUIDs.randomBase64UUID(random()), + "fs", + Settings.EMPTY, + RepositoryData.UNKNOWN_REPO_GEN, + RepositoryData.EMPTY_REPO_GEN + ); + } + + private static SnapshotId randomSnapshotId() { + return new SnapshotId(randomAlphaOfLength(10).toLowerCase(Locale.ROOT), UUIDs.randomBase64UUID(random())); + } + + private static ClusterState stateWithRepository(ClusterState previousState, RepositoryMetadata repositoryMetadata) { + final RepositoriesMetadata previousRepos = previousState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + assertThat(previousRepos.repository(repositoryMetadata.name()), nullValue()); + final List newRepos = new ArrayList<>(previousRepos.repositories()); + newRepos.add(repositoryMetadata); + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .metadata(Metadata.builder(previousState.metadata()).putCustom(RepositoriesMetadata.TYPE, new RepositoriesMetadata(newRepos))) + .build(); + } + + private static ClusterState stateWithNewPending( + ClusterState previousState, + RepositoryMetadata repository, + SnapshotId snapshot, + long creationTime + ) { + SnapshotDeletionsPending previousPendings = previousState.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + SnapshotDeletionsPending.Builder pendings = new SnapshotDeletionsPending.Builder(previousPendings, e -> {}); + pendings.add(repository.name(), repository.uuid(), snapshot, creationTime); + + ImmutableOpenMap.Builder customs = ImmutableOpenMap.builder(previousState.getCustoms()); + customs.put(SnapshotDeletionsPending.TYPE, pendings.build(Settings.EMPTY)); + return ClusterState.builder(previousState).version(previousState.version() + 1L).customs(customs.build()).build(); + } + + private static ClusterState stateWithRestore(ClusterState previousState, RepositoryMetadata repository, SnapshotId snapshot) { + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom( + RestoreInProgress.TYPE, + new RestoreInProgress.Builder(previousState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)).add( + new RestoreInProgress.Entry( + UUIDs.randomBase64UUID(random()), + new Snapshot(repository.name(), snapshot), + RestoreInProgress.State.INIT, + List.of(randomAlphaOfLength(10).toLowerCase(Locale.ROOT)), + null + ) + ).build() + ) + .build(); + } + + private static ClusterState stateWithoutRestore(ClusterState previousState, RepositoryMetadata repository, SnapshotId snapshot) { + final RestoreInProgress.Builder builder = new RestoreInProgress.Builder(); + boolean found = false; + for (RestoreInProgress.Entry entry : previousState.custom(RestoreInProgress.TYPE, RestoreInProgress.EMPTY)) { + final Snapshot restored = entry.snapshot(); + if (restored.getRepository().equals(repository.name()) && snapshot.equals(restored.getSnapshotId())) { + found = true; + } else { + builder.add(entry); + } + } + assertThat("Restore not found: " + snapshot, found, is(true)); + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom(RestoreInProgress.TYPE, builder.build()) + .build(); + } + + private static ClusterState stateWithClone(ClusterState previousState, RepositoryMetadata repository, SnapshotId snapshot) { + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom( + SnapshotsInProgress.TYPE, + previousState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY) + .withAddedEntry( + SnapshotsInProgress.startClone( + new Snapshot(repository.name(), randomSnapshotId()), + snapshot, + Map.of( + randomAlphaOfLength(10).toLowerCase(Locale.ROOT), + new IndexId(randomAlphaOfLength(10).toLowerCase(Locale.ROOT), UUIDs.randomBase64UUID(random())) + ), + 0, + 1, + Version.CURRENT + ) + ) + ) + .build(); + } + + private static ClusterState stateWithoutClone(ClusterState previousState, RepositoryMetadata repository, SnapshotId snapshot) { + final SnapshotsInProgress snapshots = previousState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + List entries = new ArrayList<>(); + boolean found = false; + for (SnapshotsInProgress.Entry entry : snapshots.forRepo(repository.name())) { + if (entry.isClone() && snapshot.equals(entry.source())) { + found = true; + } else { + entries.add(entry); + } + } + assertThat("Clone not found: " + snapshot, found, is(true)); + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom(SnapshotsInProgress.TYPE, snapshots.withUpdatedEntriesForRepo(repository.name(), entries)) + .build(); + } + + private static ClusterState stateWithRepositoryCleanUp(ClusterState previousState, RepositoryMetadata repository) { + RepositoriesMetadata repositories = previousState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + assertThat(repositories.repository(repository.name()), notNullValue()); + RepositoryCleanupInProgress cleanUps = previousState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + assertThat(cleanUps.hasCleanupInProgress(), is(false)); + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom( + RepositoryCleanupInProgress.TYPE, + new RepositoryCleanupInProgress(List.of(RepositoryCleanupInProgress.startedEntry(repository.name(), 0L))) + ) + .build(); + } + + private static ClusterState stateWithoutRepositoryCleanUp(ClusterState previousState, RepositoryMetadata repository) { + RepositoriesMetadata repositories = previousState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + assertThat(repositories.repository(repository.name()), notNullValue()); + RepositoryCleanupInProgress cleanUps = previousState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + assertThat(cleanUps.hasCleanupInProgress(), is(true)); + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .putCustom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY) + .build(); + } + + private static ClusterState updateRepository(ClusterState previousState, String repositoryName, RepositoryMetadata repositoryMetadata) { + final RepositoriesMetadata previousRepos = previousState.metadata().custom(RepositoriesMetadata.TYPE, RepositoriesMetadata.EMPTY); + assertThat(previousRepos.repository(repositoryName), notNullValue()); + final List newRepos = new ArrayList<>(); + for (RepositoryMetadata r : previousRepos.repositories()) { + if (r.name().equals(repositoryName)) { + newRepos.add(repositoryMetadata); + } else { + newRepos.add(r); + } + } + return ClusterState.builder(previousState) + .version(previousState.version() + 1L) + .metadata(Metadata.builder(previousState.metadata()).putCustom(RepositoriesMetadata.TYPE, new RepositoriesMetadata(newRepos))) + .build(); + } + + private static ClusterState addConflict( + ConflictType conflict, + ClusterState previousState, + RepositoryMetadata repository, + SnapshotId snapshot + ) { + return switch (conflict) { + case RESTORING -> stateWithRestore(previousState, repository, snapshot); + case CLONING -> stateWithClone(previousState, repository, snapshot); + case REPO_CLEANUP -> stateWithRepositoryCleanUp(previousState, repository); + case REPO_READONLY -> updateRepository( + previousState, + repository.name(), + repository.withSettings(Settings.builder().put(repository.settings()).put(READONLY_SETTING_KEY, true).build()) + ); + case REPO_MISSING -> updateRepository( + previousState, + repository.name(), + new RepositoryMetadata( + "missing", + RepositoryData.MISSING_UUID, + "fs", + repository.settings(), + repository.generation(), + repository.pendingGeneration() + ) + ); + }; + } + + private ClusterState resolveConflict( + ConflictType conflict, + ClusterState previousState, + RepositoryMetadata repository, + SnapshotId snapshot + ) { + return switch (conflict) { + case RESTORING -> stateWithoutRestore(previousState, repository, snapshot); + case CLONING -> stateWithoutClone(previousState, repository, snapshot); + case REPO_CLEANUP -> stateWithoutRepositoryCleanUp(previousState, repository); + case REPO_READONLY -> updateRepository( + previousState, + repository.name(), + repository.withSettings(Settings.builder().put(repository.settings()).put(READONLY_SETTING_KEY, false).build()) + ); + case REPO_MISSING -> updateRepository( + previousState, + "missing", + randomBoolean() + ? repository + : new RepositoryMetadata( + randomAlphaOfLength(10).toLowerCase(Locale.ROOT), + repository.uuid(), + "fs", + repository.settings(), + repository.generation(), + repository.pendingGeneration() + ) + ); + }; + } +} diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index f1fa158b0eab1..5c771165781ab 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException; import org.elasticsearch.snapshots.RestoreInfo; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; @@ -53,6 +54,7 @@ import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; +import static org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.containsString; @@ -100,9 +102,11 @@ public void testSearchableSnapshotIsDeletedWhenRepoIsRecreated() throws Exceptio assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); + assertConflict(snapshot); assertAcked(client().admin().cluster().prepareDeleteRepository(repository)); awaitSnapshotPendingDeletion(snapshot); + assertConflict(snapshot); final String repoName; if (randomBoolean()) { @@ -119,6 +123,7 @@ public void testSearchableSnapshotIsDeletedWhenRepoIsRecreated() throws Exceptio createRepository(repoName, "mock", repositorySettings, true); } awaitNoMoreSnapshotsDeletions(); + assertNoConflict(snapshot); expectThrows( SnapshotMissingException.class, @@ -150,6 +155,7 @@ public void testSearchableSnapshotIsDeletedWithOnGoingRestore() throws Exception assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); assertFalse(restoreFuture.isDone()); + assertConflict(snapshot); unblockNode(repository, masterNode); awaitNoMoreSnapshotsDeletions(); @@ -157,6 +163,7 @@ public void testSearchableSnapshotIsDeletedWithOnGoingRestore() throws Exception final RestoreInfo restoreInfoResponse = restoreFuture.actionGet().getRestoreInfo(); assertThat(restoreInfoResponse.successfulShards(), greaterThan(0)); assertThat(restoreInfoResponse.failedShards(), equalTo(0)); + assertNoConflict(snapshot); expectThrows( SnapshotMissingException.class, @@ -186,10 +193,12 @@ public void testSearchableSnapshotIsDeletedWithOnGoingClone() throws Exception { assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); assertFalse(cloneFuture.isDone()); + assertConflict(snapshot); unblockNode(repository, masterNode); awaitNoMoreSnapshotsDeletions(); assertAcked(cloneFuture.get()); + assertNoConflict(snapshot); expectThrows( SnapshotMissingException.class, @@ -398,6 +407,7 @@ public void testSearchableSnapshotIsDeletedWithOnRepoCleanUp() throws Exception } garbageFuture.get(); + // repository clean up writes a new index-N blob to ensure concurrent operations will fail so we can block on this blockMasterOnWriteIndexFile(repository); final ActionFuture cleanUpFuture = client().admin() @@ -414,9 +424,11 @@ public void testSearchableSnapshotIsDeletedWithOnRepoCleanUp() throws Exception assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); assertFalse(cleanUpFuture.isDone()); + assertConflict(snapshot); unblockNode(repository, masterNode); awaitNoMoreSnapshotsDeletions(); + assertNoConflict(snapshot); final CleanupRepositoryResponse cleanUpResponse = cleanUpFuture.get(); assertThat(cleanUpResponse.result().blobs(), equalTo((long) garbageFiles)); @@ -440,7 +452,7 @@ public void testSearchableSnapshotIsDeletedAfterExpiration() throws Exception { .setTransientSettings( Settings.builder() .put( - SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey(), + SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(randomLongBetween(100L, 1000L)) ) .build() @@ -462,19 +474,23 @@ public void testSearchableSnapshotIsDeletedAfterExpiration() throws Exception { assertAcked(client().admin().indices().prepareDelete(mountedIndex(index))); awaitSnapshotPendingDeletion(snapshot); - final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); - assertBusy(() -> assertTrue(snapshotsService.hasOngoingSnapshotsDeletions(snapshot))); + assertBusy(() -> { + final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); + assertTrue(snapshotsService.isSnapshotPendingDeletionTriggered(snapshot)); + }); assertAcked( clusterAdmin().prepareUpdateSettings() .setTransientSettings( - Settings.builder() - .put(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey(), TimeValue.ZERO) - .build() + Settings.builder().put(PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey(), TimeValue.ZERO).build() ) ); - assertBusy(() -> assertFalse(snapshotsService.hasOngoingSnapshotsDeletions(snapshot))); + assertBusy(() -> { + final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); + assertFalse(snapshotsService.isSnapshotPendingDeletionTriggered(snapshot)); + }); + awaitNoMoreSnapshotsDeletions(); } catch (Exception e) { @@ -484,8 +500,8 @@ public void testSearchableSnapshotIsDeletedAfterExpiration() throws Exception { clusterAdmin().prepareUpdateSettings() .setTransientSettings( Settings.builder() - .putNull(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey()) - .putNull(SnapshotsService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey()) + .putNull(PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey()) + .putNull(SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey()) .build() ) ); @@ -583,4 +599,19 @@ private Settings.Builder getRepositorySettings(String repository) { private String mountedIndex(String index) { return "restored-" + index; } + + private static void assertConflict(final SnapshotId snapshotId) throws Exception { + assertSnapshotPendingDeletionConflict(snapshotId, true); + } + + private static void assertNoConflict(final SnapshotId snapshotId) throws Exception { + assertSnapshotPendingDeletionConflict(snapshotId, false); + } + + private static void assertSnapshotPendingDeletionConflict(final SnapshotId snapshotId, final boolean expected) throws Exception { + assertBusy(() -> { + final SnapshotsService snapshotsService = internalCluster().getCurrentMasterNodeInstance(SnapshotsService.class); + assertThat(snapshotsService.isSnapshotPendingDeletionConflicting(snapshotId), equalTo(expected)); + }); + } } From 39672ed9c25129911ba1895bbb7f663c0f598aa8 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 14 Jan 2022 19:22:35 +0100 Subject: [PATCH 33/42] unbatched --- .../snapshots/SnapshotDeletionsPendingExecutor.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java index a6890855985b6..d22d47cd205d1 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java @@ -13,6 +13,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.RestoreInProgress; @@ -424,7 +425,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS public void onFailure(String source, Exception e) { triggered.removeAll(pendingDeletionsToRemove); } - }); + }, ClusterStateTaskExecutor.unbatched()); } } } From db5fbd63c9bf6a519481c223c689dc8fa2b01a73 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 14 Jan 2022 20:59:04 +0100 Subject: [PATCH 34/42] same error message --- .../java/org/elasticsearch/snapshots/CloneSnapshotIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java index 28f797dfa5ec6..d081f80be921c 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/CloneSnapshotIT.java @@ -287,7 +287,7 @@ public void testDeletePreventsClone() throws Exception { ConcurrentSnapshotExecutionException.class, () -> startClone(repoName, sourceSnapshot, targetSnapshot, indexName).actionGet() ); - assertThat(ex.getMessage(), containsString("cannot clone from snapshot that is being deleted")); + assertThat(ex.getMessage(), containsString("cannot clone a snapshot while a snapshot deletion is in-progress")); unblockNode(repoName, masterName); assertAcked(deleteFuture.get()); From 593cf3e7fd6ce4c48b4e9174954d67e28fecfd57 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 17 Jan 2022 10:38:47 +0100 Subject: [PATCH 35/42] fixes --- .../snapshots/SnapshotDeletionsPendingExecutor.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java index d22d47cd205d1..20cc6cb73a257 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java @@ -422,7 +422,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS } @Override - public void onFailure(String source, Exception e) { + public void onFailure(Exception e) { triggered.removeAll(pendingDeletionsToRemove); } }, ClusterStateTaskExecutor.unbatched()); From c40f7c18d3b9e8b5b63fc0626a44f873fa71eeab Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 17 Jan 2022 10:49:57 +0100 Subject: [PATCH 36/42] remove obsolete doc --- .../snapshots/SnapshotDeletionsPendingExecutor.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java index 20cc6cb73a257..6e1eb50ecbd7f 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java @@ -80,10 +80,6 @@ public class SnapshotDeletionsPendingExecutor { */ private final Map conflicting = new HashMap<>(); - /** - * Counters for the type of conflicts for the current set of conflicting pending snapshots deletions. This is used to look for - * updates in cluster state updates only when it is really needed. - */ enum ConflictType { RESTORING, CLONING, From dd1ced9a409ad9490c9e435b674fa3608eebc58a Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 17 Jan 2022 11:08:17 +0100 Subject: [PATCH 37/42] rename method --- .../SearchableSnapshotsPendingDeletionsIntegTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 5c771165781ab..45a1e717c6ecf 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -444,7 +444,7 @@ public void testSearchableSnapshotIsDeletedWithOnRepoCleanUp() throws Exception }); } - public void testSearchableSnapshotIsDeletedAfterExpiration() throws Exception { + public void testSnapshotDeletionsPendingIsRemovedFromClusterStateAfterExpiration() throws Exception { mountIndexThenExecute((repository, snapshot, index) -> { try { assertAcked( From 8f7baa1cc6a1ac024448218e25bd64664351355e Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 2 Feb 2022 21:46:35 +0100 Subject: [PATCH 38/42] Update docs/changelog/79156.yaml --- docs/changelog/79156.yaml | 6 ++++++ 1 file changed, 6 insertions(+) create mode 100644 docs/changelog/79156.yaml diff --git a/docs/changelog/79156.yaml b/docs/changelog/79156.yaml new file mode 100644 index 0000000000000..69fa260060814 --- /dev/null +++ b/docs/changelog/79156.yaml @@ -0,0 +1,6 @@ +pr: 79156 +summary: Add snapshots pending deletion in cluster state to delete snapshot once index + is deleted +area: Snapshot/Restore +type: enhancement +issues: [] From f13eaf69f8707b7aa962950d53ec848d3397fbdd Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 3 Mar 2022 14:35:19 +0100 Subject: [PATCH 39/42] batch cluster state updates that removes pending deletions --- .../common/settings/ClusterSettings.java | 6 +- ...a => SnapshotDeletionsPendingService.java} | 106 ++++++++++++------ .../snapshots/SnapshotsService.java | 4 +- ...SnapshotDeletionsPendingServiceTests.java} | 18 ++- ...leSnapshotsPendingDeletionsIntegTests.java | 8 +- 5 files changed, 96 insertions(+), 46 deletions(-) rename server/src/main/java/org/elasticsearch/snapshots/{SnapshotDeletionsPendingExecutor.java => SnapshotDeletionsPendingService.java} (86%) rename server/src/test/java/org/elasticsearch/snapshots/{SnapshotDeletionsPendingExecutorTests.java => SnapshotDeletionsPendingServiceTests.java} (96%) diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 075c4da4ce5b0..577c2f40015eb 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -100,7 +100,7 @@ import org.elasticsearch.search.fetch.subphase.highlight.FastVectorHighlighter; import org.elasticsearch.snapshots.InternalSnapshotsInfoService; import org.elasticsearch.snapshots.RestoreService; -import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingService; import org.elasticsearch.snapshots.SnapshotsService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ProxyConnectionStrategy; @@ -504,8 +504,8 @@ public void apply(Settings value, Settings current, Settings previous) { HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, - SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, - SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, + SnapshotDeletionsPendingService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, + SnapshotDeletionsPendingService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING, RestoreService.REFRESH_REPO_UUID_ON_RESTORE_SETTING, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingService.java similarity index 86% rename from server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java rename to server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingService.java index 6e1eb50ecbd7f..105d88b68e4c1 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutor.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingService.java @@ -13,8 +13,9 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskExecutor; -import org.elasticsearch.cluster.ClusterStateUpdateTask; +import org.elasticsearch.cluster.ClusterStateTaskListener; import org.elasticsearch.cluster.RepositoryCleanupInProgress; import org.elasticsearch.cluster.RestoreInProgress; import org.elasticsearch.cluster.SnapshotDeletionsPending; @@ -23,6 +24,7 @@ import org.elasticsearch.cluster.metadata.RepositoryMetadata; import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -41,6 +43,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; @@ -49,9 +52,9 @@ import static org.elasticsearch.snapshots.SnapshotUtils.restoreSources; import static org.elasticsearch.snapshots.SnapshotsService.findRepositoryForPendingDeletion; -public class SnapshotDeletionsPendingExecutor { +public class SnapshotDeletionsPendingService { - private static final Logger logger = LogManager.getLogger(SnapshotDeletionsPendingExecutor.class); + private static final Logger logger = LogManager.getLogger(SnapshotDeletionsPendingService.class); public static final Setting PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING = Setting.timeSetting( "snapshot.snapshot_deletions_pending.retry_interval", @@ -91,11 +94,12 @@ enum ConflictType { private final SnapshotsService snapshotsService; private final ClusterService clusterService; private final ThreadPool threadPool; + private final ClusterStateTaskExecutor removePendingDeletionsExecutor; private volatile TimeValue pendingDeletionsRetryInterval; private volatile TimeValue pendingDeletionsExpirationInterval; - SnapshotDeletionsPendingExecutor( + SnapshotDeletionsPendingService( SnapshotsService snapshotsService, ClusterService clusterService, ThreadPool threadPool, @@ -104,6 +108,7 @@ enum ConflictType { this.snapshotsService = Objects.requireNonNull(snapshotsService); this.clusterService = Objects.requireNonNull(clusterService); this.threadPool = Objects.requireNonNull(threadPool); + this.removePendingDeletionsExecutor = new RemoveSnapshotDeletionsPendingExecutor(); pendingDeletionsRetryInterval = PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.get(settings); clusterService.getClusterSettings() .addSettingsUpdateConsumer(PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING, t -> pendingDeletionsRetryInterval = t); @@ -394,34 +399,12 @@ void finish() { ); } if (pendingDeletionsToRemove.isEmpty() == false) { - clusterService.submitStateUpdateTask("remove-snapshot-deletions-in-pending", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsPending currentPendings = currentState.custom( - SnapshotDeletionsPending.TYPE, - SnapshotDeletionsPending.EMPTY - ); - final SnapshotDeletionsPending updatedPendings = currentPendings.withRemovedSnapshots( - List.copyOf(pendingDeletionsToRemove) - ); - if (currentPendings == updatedPendings) { - return currentState; - } - return ClusterState.builder(currentState) - .putCustom(SnapshotDeletionsPending.TYPE, updatedPendings) - .build(); - } - - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - triggered.removeAll(pendingDeletionsToRemove); - } - - @Override - public void onFailure(Exception e) { - triggered.removeAll(pendingDeletionsToRemove); - } - }, ClusterStateTaskExecutor.unbatched()); + clusterService.submitStateUpdateTask( + "remove-snapshot-deletions-in-pending", + new RemoveSnapshotDeletionsPendingTask(List.copyOf(pendingDeletionsToRemove)), + ClusterStateTaskConfig.build(Priority.NORMAL), + removePendingDeletionsExecutor + ); } } } @@ -448,4 +431,63 @@ public void onFailure(Exception e) { ); } } + + private class RemoveSnapshotDeletionsPendingExecutor implements ClusterStateTaskExecutor { + + @Override + public ClusterState execute(ClusterState currentState, List> taskContexts) + throws Exception { + ClusterState state = currentState; + for (final var taskContext : taskContexts) { + final var task = taskContext.getTask(); + try { + SnapshotDeletionsPending currentPendings = state.custom(SnapshotDeletionsPending.TYPE, SnapshotDeletionsPending.EMPTY); + SnapshotDeletionsPending updatedPendings = currentPendings.withRemovedSnapshots(task.pendingDeletionsToRemove); + if (currentPendings != updatedPendings) { + state = ClusterState.builder(state).putCustom(SnapshotDeletionsPending.TYPE, updatedPendings).build(); + } + taskContext.success(task.newPublicationListener()); + } catch (Exception e) { + taskContext.onFailure(e); + } + } + return state; + } + } + + private class RemoveSnapshotDeletionsPendingTask implements ClusterStateTaskListener { + + private final List pendingDeletionsToRemove; + private final AtomicBoolean removed; + + private RemoveSnapshotDeletionsPendingTask(List pendingDeletionsToRemove) { + this.pendingDeletionsToRemove = pendingDeletionsToRemove; + this.removed = new AtomicBoolean(); + } + + private void removeAllPendingDeletions() { + if (removed.compareAndSet(false, true)) { + pendingDeletionsToRemove.forEach(triggered::remove); + } + } + + @Override + public void onFailure(Exception e) { + removeAllPendingDeletions(); + } + + ActionListener newPublicationListener() { + return new ActionListener<>() { + @Override + public void onResponse(ClusterState clusterState) { + removeAllPendingDeletions(); + } + + @Override + public void onFailure(Exception e) { + removeAllPendingDeletions(); + } + }; + } + } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 2d25a3b5744e3..d43ba632dc9e7 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -178,7 +178,7 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final SystemIndices systemIndices; - private final SnapshotDeletionsPendingExecutor snapshotPendingDeletions; + private final SnapshotDeletionsPendingService snapshotPendingDeletions; /** * Setting that specifies the maximum number of allowed concurrent snapshot create and delete operations in the @@ -226,7 +226,7 @@ public SnapshotsService( .addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, i -> maxConcurrentOperations = i); } this.systemIndices = systemIndices; - this.snapshotPendingDeletions = new SnapshotDeletionsPendingExecutor(this, clusterService, threadPool, settings); + this.snapshotPendingDeletions = new SnapshotDeletionsPendingService(this, clusterService, threadPool, settings); } /** diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingServiceTests.java similarity index 96% rename from server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java rename to server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingServiceTests.java index c4e063a8ca1ce..c502fb6c7459f 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingExecutorTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotDeletionsPendingServiceTests.java @@ -18,6 +18,8 @@ import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.metadata.RepositoriesMetadata; import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.node.DiscoveryNodeRole; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; @@ -27,7 +29,7 @@ import org.elasticsearch.common.util.concurrent.DeterministicTaskQueue; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoryData; -import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor.ConflictType; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingService.ConflictType; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.threadpool.TestThreadPool; import org.junit.After; @@ -38,6 +40,7 @@ import java.util.Locale; import java.util.Map; +import static java.util.Collections.emptyMap; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; import static org.elasticsearch.test.ClusterServiceUtils.setState; @@ -52,13 +55,13 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -public class SnapshotDeletionsPendingExecutorTests extends ESTestCase { +public class SnapshotDeletionsPendingServiceTests extends ESTestCase { private SnapshotsService snapshotsService; private DeterministicTaskQueue taskQueue; private ClusterService clusterService; private TestThreadPool threadPool; - private SnapshotDeletionsPendingExecutor executor; + private SnapshotDeletionsPendingService executor; @Before @Override @@ -73,7 +76,7 @@ public void setUp() throws Exception { return null; }).when(snapshotsService).deleteSnapshotsByUuid(any(), any(), any()); taskQueue = new DeterministicTaskQueue(); - executor = new SnapshotDeletionsPendingExecutor(snapshotsService, clusterService, taskQueue.getThreadPool(), Settings.EMPTY); + executor = new SnapshotDeletionsPendingService(snapshotsService, clusterService, taskQueue.getThreadPool(), Settings.EMPTY); clusterService.addStateApplier(event -> executor.processPendingDeletions(event.state(), event.previousState())); } @@ -186,7 +189,12 @@ private void assertNoSnapshotDeletionsPendingsInClusterState() { private static ClusterState emptyState() { return ClusterState.builder(ClusterState.EMPTY_STATE) - .nodes(DiscoveryNodes.builder().localNodeId("_node").masterNodeId("_node")) + .nodes( + DiscoveryNodes.builder() + .add(new DiscoveryNode("_node", buildNewFakeTransportAddress(), emptyMap(), DiscoveryNodeRole.roles(), Version.CURRENT)) + .localNodeId("_node") + .masterNodeId("_node") + ) .metadata(Metadata.builder().generateClusterUuidIfNeeded()) .build(); } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java index 45a1e717c6ecf..6443ce557aae5 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsPendingDeletionsIntegTests.java @@ -32,7 +32,7 @@ import org.elasticsearch.repositories.fs.FsRepository; import org.elasticsearch.snapshots.ConcurrentSnapshotExecutionException; import org.elasticsearch.snapshots.RestoreInfo; -import org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor; +import org.elasticsearch.snapshots.SnapshotDeletionsPendingService; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotMissingException; @@ -54,7 +54,7 @@ import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.elasticsearch.repositories.blobstore.BlobStoreRepository.READONLY_SETTING_KEY; import static org.elasticsearch.snapshots.SearchableSnapshotsSettings.SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION; -import static org.elasticsearch.snapshots.SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING; +import static org.elasticsearch.snapshots.SnapshotDeletionsPendingService.PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.containsString; @@ -452,7 +452,7 @@ public void testSnapshotDeletionsPendingIsRemovedFromClusterStateAfterExpiration .setTransientSettings( Settings.builder() .put( - SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey(), + SnapshotDeletionsPendingService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey(), TimeValue.timeValueMillis(randomLongBetween(100L, 1000L)) ) .build() @@ -501,7 +501,7 @@ public void testSnapshotDeletionsPendingIsRemovedFromClusterStateAfterExpiration .setTransientSettings( Settings.builder() .putNull(PENDING_SNAPSHOT_DELETIONS_EXPIRATION_INTERVAL_SETTING.getKey()) - .putNull(SnapshotDeletionsPendingExecutor.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey()) + .putNull(SnapshotDeletionsPendingService.PENDING_SNAPSHOT_DELETIONS_RETRY_INTERVAL_SETTING.getKey()) .build() ) ); From 7fdbb907b18de872e13a0e523bb565799be61759 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 3 Mar 2022 15:59:16 +0100 Subject: [PATCH 40/42] nits --- .../cluster/metadata/MetadataDeleteIndexService.java | 11 +++++------ .../org/elasticsearch/snapshots/RestoreService.java | 4 ++-- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index ba5756a226123..7ae1f739c54a9 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -213,8 +213,9 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( final ClusterState state ) { final List deletedIndicesSettings = indicesToDelete.stream() - .map(index -> state.metadata().getIndexSafe(index).getSettings()) - .filter(SearchableSnapshotsSettings::isSearchableSnapshotStore) + .map(index -> state.metadata().getIndexSafe(index)) + .filter(IndexMetadata::isSearchableSnapshot) + .map(IndexMetadata::getSettings) .filter(indexSettings -> indexSettings.getAsBoolean(SEARCHABLE_SNAPSHOTS_DELETE_SNAPSHOT_ON_INDEX_DELETION, false)) .collect(Collectors.toList()); if (deletedIndicesSettings.isEmpty()) { @@ -225,10 +226,8 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( .indices() .stream() .map(Map.Entry::getValue) - .filter(index -> indicesToDelete.contains(index.getIndex()) == false) - .map(IndexMetadata::getSettings) - .filter(SearchableSnapshotsSettings::isSearchableSnapshotStore) - .map(MetadataDeleteIndexService::toSnapshotId) + .filter(index -> index.isSearchableSnapshot() && indicesToDelete.contains(index.getIndex()) == false) + .map(index -> MetadataDeleteIndexService.toSnapshotId(index.getSettings())) .collect(Collectors.toUnmodifiableSet()); final RepositoriesMetadata repositories = state.metadata().custom(RepositoriesMetadata.TYPE); diff --git a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java index 79f6d686b712b..0cf4bd8570fa0 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/RestoreService.java @@ -1674,10 +1674,10 @@ private static void ensureSearchableSnapshotsRestorable( if (other.getIndex().equals(index)) { continue; // do not check the searchable snapshot index against itself } - final Settings otherSettings = other.getSettings(); - if (SearchableSnapshotsSettings.isSearchableSnapshotStore(otherSettings) == false) { + if (other.isSearchableSnapshot() == false) { continue; // other index is not a searchable snapshot index, skip } + final Settings otherSettings = other.getSettings(); final String otherSnapshotUuid = otherSettings.get(SEARCHABLE_SNAPSHOTS_SNAPSHOT_UUID_SETTING_KEY); if (Objects.equals(snapshotUuid, otherSnapshotUuid) == false) { continue; // other index is backed by a different snapshot, skip From 82778ae19dd1fca4512ff8ac6928f0f5557113b1 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 14 Mar 2022 13:51:45 +0100 Subject: [PATCH 41/42] reset (wip) --- .../org/elasticsearch/cluster/SnapshotDeletionsPending.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java index f6a2170253698..bd7d7d27e4864 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsPending.java @@ -55,7 +55,7 @@ public class SnapshotDeletionsPending extends AbstractNamedDiffable impl /** * Version from which a snapshot can be marked as to be deleted after an index is deleted. */ - public static final Version SNAPSHOT_DELETIONS_PENDING_VERSION = Version.V_8_1_0; + public static final Version SNAPSHOT_DELETIONS_PENDING_VERSION = Version.V_8_2_0; /** * Setting for the maximum number of snapshots pending deletion allowed in the cluster state. From fb54e4fbbbb5eb1988174ceab6a240dbfe5e0618 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 14 Mar 2022 14:08:53 +0100 Subject: [PATCH 42/42] update branch --- .../cluster/metadata/MetadataDeleteIndexService.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java index 7ae1f739c54a9..8fbc4979d1189 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetadataDeleteIndexService.java @@ -224,8 +224,8 @@ private SnapshotDeletionsPending updateSnapshotDeletionsPending( final Set activeSearchableSnapshots = state.metadata() .indices() + .values() .stream() - .map(Map.Entry::getValue) .filter(index -> index.isSearchableSnapshot() && indicesToDelete.contains(index.getIndex()) == false) .map(index -> MetadataDeleteIndexService.toSnapshotId(index.getSettings())) .collect(Collectors.toUnmodifiableSet());