From d333dacb4abb14a58b36f3108521547ee374a06a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 10 Jul 2020 15:19:08 +0200 Subject: [PATCH] Enable Fully Concurrent Snapshot Operations (#56911) Enables fully concurrent snapshot operations: * Snapshot create- and delete operations can be started in any order * Delete operations wait for snapshot finalization to finish, are batched as much as possible to improve efficiency and once enqueued in the cluster state prevent new snapshots from starting on data nodes until executed * We could be even more concurrent here in a follow-up by interleaving deletes and snapshots on a per-shard level. I decided not to do this for now since it seemed not worth the added complexity yet. Due to batching+deduplicating of deletes the pain of having a delete stuck behind a long -running snapshot seemed manageable (dropped client connections + resulting retries don't cause issues due to deduplication of delete jobs, batching of deletes allows enqueuing more and more deletes even if a snapshot blocks for a long time that will all be executed in essentially constant time (due to bulk snapshot deletion, deleting multiple snapshots is mostly about as fast as deleting a single one)) * Snapshot creation is completely concurrent across shards, but per shard snapshots are linearized for each repository as are snapshot finalizations See updated JavaDoc and added test cases for more details and illustration on the functionality. Some notes: The queuing of snapshot finalizations and deletes and the related locking/synchronization is a little awkward in this version but can be much simplified with some refactoring. The problem is that snapshot finalizations resolve their listeners on the `SNAPSHOT` pool while deletes resolve the listener on the master update thread. With some refactoring both of these could be moved to the master update thread, effectively removing the need for any synchronization around the `SnapshotService` state. I didn't do this refactoring here because it's a fairly large change and not necessary for the functionality but plan to do so in a follow-up. This change allows for completely removing any trickery around synchronizing deletes and snapshots from SLM and 100% does away with SLM errors from collisions between deletes and snapshots. Snapshotting a single index in parallel to a long running full backup will execute without having to wait for the long running backup as required by the ILM/SLM use case of moving indices to "snapshot tier". Finalizations are linearized but ordered according to which snapshot saw all of its shards complete first --- .../repositories/s3/S3Repository.java | 2 +- .../snapshots/ConcurrentSnapshotsIT.java | 1316 ++++++++++++++++ .../MinThreadsSnapshotRestoreIT.java | 155 -- .../TransportSnapshotsStatusAction.java | 1 + .../cluster/SnapshotDeletionsInProgress.java | 141 +- .../cluster/SnapshotsInProgress.java | 68 +- .../common/settings/ClusterSettings.java | 2 + .../repositories/FilterRepository.java | 2 +- .../repositories/Repository.java | 2 +- .../repositories/RepositoryData.java | 14 +- .../blobstore/BlobStoreRepository.java | 66 +- .../snapshots/SnapshotShardsService.java | 7 +- .../snapshots/SnapshotsService.java | 1358 ++++++++++++++--- .../elasticsearch/snapshots/package-info.java | 37 + .../ClusterSerializationTests.java | 8 +- .../discovery/AbstractDisruptionTestCase.java | 2 +- .../RepositoriesServiceTests.java | 2 +- .../snapshots/SnapshotResiliencyTests.java | 97 +- .../index/shard/RestoreOnlyRepository.java | 2 +- .../AbstractSnapshotIntegTestCase.java | 18 +- .../snapshots/mockstore/MockRepository.java | 33 +- .../xpack/ccr/repository/CcrRepository.java | 2 +- .../xpack/slm/SnapshotRetentionTaskTests.java | 3 +- 23 files changed, 2867 insertions(+), 471 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java delete mode 100644 server/src/internalClusterTest/java/org/elasticsearch/snapshots/MinThreadsSnapshotRestoreIT.java diff --git a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java index 58d3bf21594ab..51032981a9eef 100644 --- a/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/elasticsearch/repositories/s3/S3Repository.java @@ -249,7 +249,7 @@ public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryS @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { if (SnapshotsService.useShardGenerations(repositoryMetaVersion) == false) { listener = delayedListener(listener); } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java new file mode 100644 index 0000000000000..6e0c84f4732a9 --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/ConcurrentSnapshotsIT.java @@ -0,0 +1,1316 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.snapshots; + +import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.StepListener; +import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; +import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotStatus; +import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; +import org.elasticsearch.action.support.GroupedActionListener; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.SnapshotDeletionsInProgress; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.UncategorizedExecutionException; +import org.elasticsearch.discovery.AbstractDisruptionTestCase; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.RepositoryException; +import org.elasticsearch.repositories.ShardGenerations; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.snapshots.mockstore.MockRepository; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.disruption.NetworkDisruption; +import org.elasticsearch.test.transport.MockTransportService; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.TimeUnit; +import java.util.function.Predicate; + +import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; +import static org.elasticsearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.not; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) +public class ConcurrentSnapshotsIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(MockTransportService.TestPlugin.class, MockRepository.Plugin.class); + } + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)) + .put(AbstractDisruptionTestCase.DEFAULT_SETTINGS) + .build(); + } + + public void testLongRunningSnapshotAllowsConcurrentSnapshot() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-slow"); + + final ActionFuture createSlowFuture = + startFullSnapshotBlockedOnDataNode("slow-snapshot", repoName, dataNode); + + final String dataNode2 = internalCluster().startDataOnlyNode(); + ensureStableCluster(3); + final String indexFast = "index-fast"; + createIndexWithContent(indexFast, dataNode2, dataNode); + + assertSuccessful(client().admin().cluster().prepareCreateSnapshot(repoName, "fast-snapshot") + .setIndices(indexFast).setWaitForCompletion(true).execute()); + + assertThat(createSlowFuture.isDone(), is(false)); + unblockNode(repoName, dataNode); + + assertSuccessful(createSlowFuture); + } + + public void testDeletesAreBatched() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + + createIndex("foo"); + ensureGreen(); + + final int numSnapshots = randomIntBetween(1, 4); + final PlainActionFuture> allSnapshotsDone = PlainActionFuture.newFuture(); + final ActionListener snapshotsListener = new GroupedActionListener<>(allSnapshotsDone, numSnapshots); + final Collection snapshotNames = new HashSet<>(); + for (int i = 0; i < numSnapshots; i++) { + final String snapshot = "snap-" + i; + snapshotNames.add(snapshot); + client().admin().cluster().prepareCreateSnapshot(repoName, snapshot).setWaitForCompletion(true) + .execute(snapshotsListener); + } + final Collection snapshotResponses = allSnapshotsDone.get(); + for (CreateSnapshotResponse snapshotResponse : snapshotResponses) { + assertThat(snapshotResponse.getSnapshotInfo().state(), is(SnapshotState.SUCCESS)); + } + + createIndexWithContent("index-slow"); + + final ActionFuture createSlowFuture = + startFullSnapshotBlockedOnDataNode("blocked-snapshot", repoName, dataNode); + + final Collection> deleteFutures = new ArrayList<>(); + while (snapshotNames.isEmpty() == false) { + final Collection toDelete = randomSubsetOf(snapshotNames); + if (toDelete.isEmpty()) { + continue; + } + snapshotNames.removeAll(toDelete); + final StepListener future = new StepListener<>(); + client().admin().cluster().prepareDeleteSnapshot(repoName, toDelete.toArray(Strings.EMPTY_ARRAY)).execute(future); + deleteFutures.add(future); + } + + assertThat(createSlowFuture.isDone(), is(false)); + + final long repoGenAfterInitialSnapshots = getRepositoryData(repoName).getGenId(); + assertThat(repoGenAfterInitialSnapshots, is(numSnapshots - 1L)); + unblockNode(repoName, dataNode); + + final SnapshotInfo slowSnapshotInfo = assertSuccessful(createSlowFuture); + + logger.info("--> waiting for batched deletes to finish"); + final PlainActionFuture> allDeletesDone = new PlainActionFuture<>(); + final ActionListener deletesListener = new GroupedActionListener<>(allDeletesDone, deleteFutures.size()); + for (StepListener deleteFuture : deleteFutures) { + deleteFuture.whenComplete(deletesListener::onResponse, deletesListener::onFailure); + } + allDeletesDone.get(); + + logger.info("--> verifying repository state"); + final RepositoryData repositoryDataAfterDeletes = getRepositoryData(repoName); + // One increment for snapshot, one for all the deletes + assertThat(repositoryDataAfterDeletes.getGenId(), is(repoGenAfterInitialSnapshots + 2)); + assertThat(repositoryDataAfterDeletes.getSnapshotIds(), contains(slowSnapshotInfo.snapshotId())); + } + + public void testBlockedRepoDoesNotBlockOtherRepos() throws Exception { + internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String blockedRepoName = "test-repo-blocked"; + final String otherRepoName = "test-repo"; + createRepository(blockedRepoName, "mock"); + createRepository(otherRepoName, "fs"); + createIndex("foo"); + ensureGreen(); + createIndexWithContent("index-slow"); + + final ActionFuture createSlowFuture = + startAndBlockFailingFullSnapshot(blockedRepoName, "blocked-snapshot"); + + client().admin().cluster().prepareCreateSnapshot(otherRepoName, "snapshot") + .setIndices("does-not-exist-*") + .setWaitForCompletion(false).get(); + + unblockNode(blockedRepoName, internalCluster().getMasterName()); + expectThrows(SnapshotException.class, createSlowFuture::actionGet); + + assertBusy(() -> assertThat(currentSnapshots(otherRepoName), empty()), 30L, TimeUnit.SECONDS); + } + + public void testMultipleReposAreIndependent() throws Exception { + internalCluster().startMasterOnlyNode(); + // We're blocking a some of the snapshot threads when we block the first repo below so we have to make sure we have enough threads + // left for the second concurrent snapshot. + final String dataNode = startDataNodeWithLargeSnapshotPool(); + final String blockedRepoName = "test-repo-blocked"; + final String otherRepoName = "test-repo"; + createRepository(blockedRepoName, "mock"); + createRepository(otherRepoName, "fs"); + createIndexWithContent("test-index"); + + final ActionFuture createSlowFuture = + startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + + logger.info("--> waiting for concurrent snapshot(s) to finish"); + createNSnapshots(otherRepoName, randomIntBetween(1, 5)); + + unblockNode(blockedRepoName, dataNode); + assertSuccessful(createSlowFuture); + } + + public void testMultipleReposAreIndependent2() throws Exception { + internalCluster().startMasterOnlyNode(); + // We're blocking a some of the snapshot threads when we block the first repo below so we have to make sure we have enough threads + // left for the second repository's concurrent operations. + final String dataNode = startDataNodeWithLargeSnapshotPool(); + final String blockedRepoName = "test-repo-blocked"; + final String otherRepoName = "test-repo"; + createRepository(blockedRepoName, "mock"); + createRepository(otherRepoName, "fs"); + createIndexWithContent("test-index"); + + final ActionFuture createSlowFuture = + startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + + logger.info("--> waiting for concurrent snapshot(s) to finish"); + createNSnapshots(otherRepoName, randomIntBetween(1, 5)); + assertAcked(startDelete(otherRepoName, "*").get()); + + unblockNode(blockedRepoName, dataNode); + assertSuccessful(createSlowFuture); + } + + public void testMultipleReposAreIndependent3() throws Exception { + final String masterNode = internalCluster().startMasterOnlyNode(LARGE_SNAPSHOT_POOL_SETTINGS); + internalCluster().startDataOnlyNode(); + final String blockedRepoName = "test-repo-blocked"; + final String otherRepoName = "test-repo"; + createRepository(blockedRepoName, "mock"); + createRepository(otherRepoName, "fs"); + createIndexWithContent("test-index"); + + createFullSnapshot( blockedRepoName, "blocked-snapshot"); + blockNodeOnAnyFiles(blockedRepoName, masterNode); + final ActionFuture slowDeleteFuture = startDelete(blockedRepoName, "*"); + + logger.info("--> waiting for concurrent snapshot(s) to finish"); + createNSnapshots(otherRepoName, randomIntBetween(1, 5)); + assertAcked(startDelete(otherRepoName, "*").get()); + + unblockNode(blockedRepoName, masterNode); + assertAcked(slowDeleteFuture.actionGet()); + } + + public void testSnapshotRunsAfterInProgressDelete() throws Exception { + final String masterNode = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + + ensureGreen(); + createIndexWithContent("index-test"); + + final String firstSnapshot = "first-snapshot"; + createFullSnapshot(repoName, firstSnapshot); + + blockMasterFromFinalizingSnapshotOnIndexFile(repoName); + final ActionFuture deleteFuture = startDelete(repoName, firstSnapshot); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + final ActionFuture snapshotFuture = startFullSnapshot(repoName, "second-snapshot"); + + unblockNode(repoName, masterNode); + final UncategorizedExecutionException ex = expectThrows(UncategorizedExecutionException.class, deleteFuture::actionGet); + assertThat(ex.getRootCause(), instanceOf(IOException.class)); + + assertSuccessful(snapshotFuture); + } + + public void testAbortOneOfMultipleSnapshots() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + final String firstIndex = "index-one"; + createIndexWithContent(firstIndex); + + final String firstSnapshot = "snapshot-one"; + final ActionFuture firstSnapshotResponse = + startFullSnapshotBlockedOnDataNode(firstSnapshot, repoName, dataNode); + + final String dataNode2 = internalCluster().startDataOnlyNode(); + ensureStableCluster(3); + final String secondIndex = "index-two"; + createIndexWithContent(secondIndex, dataNode2, dataNode); + + final String secondSnapshot = "snapshot-two"; + final ActionFuture secondSnapshotResponse = startFullSnapshot(repoName, secondSnapshot); + + logger.info("--> wait for snapshot on second data node to finish"); + awaitClusterState(state -> { + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + return snapshotsInProgress.entries().size() == 2 && snapshotHasCompletedShard(secondSnapshot, snapshotsInProgress); + }); + + final ActionFuture deleteSnapshotsResponse = startDelete(repoName, firstSnapshot); + awaitNDeletionsInProgress(1); + + logger.info("--> start third snapshot"); + final ActionFuture thirdSnapshotResponse = client().admin().cluster() + .prepareCreateSnapshot(repoName, "snapshot-three").setIndices(secondIndex).setWaitForCompletion(true).execute(); + + assertThat(firstSnapshotResponse.isDone(), is(false)); + assertThat(secondSnapshotResponse.isDone(), is(false)); + + unblockNode(repoName, dataNode); + final SnapshotInfo firstSnapshotInfo = firstSnapshotResponse.get().getSnapshotInfo(); + assertThat(firstSnapshotInfo.state(), is(SnapshotState.FAILED)); + assertThat(firstSnapshotInfo.reason(), is("Snapshot was aborted by deletion")); + + final SnapshotInfo secondSnapshotInfo = assertSuccessful(secondSnapshotResponse); + final SnapshotInfo thirdSnapshotInfo = assertSuccessful(thirdSnapshotResponse); + + assertThat(deleteSnapshotsResponse.get().isAcknowledged(), is(true)); + + logger.info("--> verify that the first snapshot is gone"); + assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), + containsInAnyOrder(secondSnapshotInfo, thirdSnapshotInfo)); + } + + public void testCascadedAborts() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + + final String firstSnapshot = "snapshot-one"; + final ActionFuture firstSnapshotResponse = + startFullSnapshotBlockedOnDataNode(firstSnapshot, repoName, dataNode); + + final String dataNode2 = internalCluster().startDataOnlyNode(); + ensureStableCluster(3); + createIndexWithContent("index-two", dataNode2, dataNode); + + final String secondSnapshot = "snapshot-two"; + final ActionFuture secondSnapshotResponse = startFullSnapshot(repoName, secondSnapshot); + + logger.info("--> wait for snapshot on second data node to finish"); + awaitClusterState(state -> { + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + return snapshotsInProgress.entries().size() == 2 && snapshotHasCompletedShard(secondSnapshot, snapshotsInProgress); + }); + + final ActionFuture deleteSnapshotsResponse = startDelete(repoName, firstSnapshot); + awaitNDeletionsInProgress(1); + + final ActionFuture thirdSnapshotResponse = startFullSnapshot(repoName, "snapshot-three"); + + assertThat(firstSnapshotResponse.isDone(), is(false)); + assertThat(secondSnapshotResponse.isDone(), is(false)); + + logger.info("--> waiting for all three snapshots to show up as in-progress"); + assertBusy(() -> assertThat(currentSnapshots(repoName), hasSize(3)), 30L, TimeUnit.SECONDS); + + final ActionFuture allDeletedResponse = startDelete(repoName, "*"); + + logger.info("--> waiting for second and third snapshot to finish"); + assertBusy(() -> { + assertThat(currentSnapshots(repoName), hasSize(1)); + final SnapshotsInProgress snapshotsInProgress = clusterService().state().custom(SnapshotsInProgress.TYPE); + assertThat(snapshotsInProgress.entries().get(0).state(), is(SnapshotsInProgress.State.ABORTED)); + }, 30L, TimeUnit.SECONDS); + + unblockNode(repoName, dataNode); + + logger.info("--> verify all snapshots were aborted"); + assertThat(firstSnapshotResponse.get().getSnapshotInfo().state(), is(SnapshotState.FAILED)); + assertThat(secondSnapshotResponse.get().getSnapshotInfo().state(), is(SnapshotState.FAILED)); + assertThat(thirdSnapshotResponse.get().getSnapshotInfo().state(), is(SnapshotState.FAILED)); + + logger.info("--> verify both deletes have completed"); + assertAcked(deleteSnapshotsResponse.get()); + assertAcked(allDeletedResponse.get()); + + logger.info("--> verify that all snapshots are gone"); + assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty()); + } + + public void testMasterFailOverWithQueuedDeletes() throws Exception { + internalCluster().startMasterOnlyNodes(3); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + + final String firstIndex = "index-one"; + createIndexWithContent(firstIndex); + + final String firstSnapshot = "snapshot-one"; + blockDataNode(repoName, dataNode); + final ActionFuture firstSnapshotResponse = startFullSnapshotFromNonMasterClient(repoName, firstSnapshot); + waitForBlock(dataNode, repoName, TimeValue.timeValueSeconds(30L)); + + final String dataNode2 = internalCluster().startDataOnlyNode(); + ensureStableCluster(5); + final String secondIndex = "index-two"; + createIndexWithContent(secondIndex, dataNode2, dataNode); + + final String secondSnapshot = "snapshot-two"; + final ActionFuture secondSnapshotResponse = startFullSnapshot(repoName, secondSnapshot); + + logger.info("--> wait for snapshot on second data node to finish"); + awaitClusterState(state -> { + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + return snapshotsInProgress.entries().size() == 2 && snapshotHasCompletedShard(secondSnapshot, snapshotsInProgress); + }); + + final ActionFuture firstDeleteFuture = startDeleteFromNonMasterClient(repoName, firstSnapshot); + awaitNDeletionsInProgress(1); + + blockDataNode(repoName, dataNode2); + final ActionFuture snapshotThreeFuture = startFullSnapshotFromNonMasterClient(repoName, "snapshot-three"); + waitForBlock(dataNode2, repoName, TimeValue.timeValueSeconds(30L)); + + assertThat(firstSnapshotResponse.isDone(), is(false)); + assertThat(secondSnapshotResponse.isDone(), is(false)); + + logger.info("--> waiting for all three snapshots to show up as in-progress"); + assertBusy(() -> assertThat(currentSnapshots(repoName), hasSize(3)), 30L, TimeUnit.SECONDS); + + final ActionFuture deleteAllSnapshots = startDeleteFromNonMasterClient(repoName, "*"); + logger.info("--> wait for delete to be enqueued in cluster state"); + awaitClusterState(state -> { + final SnapshotDeletionsInProgress deletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + return deletionsInProgress.getEntries().size() == 1 && deletionsInProgress.getEntries().get(0).getSnapshots().size() == 3; + }); + + logger.info("--> waiting for second snapshot to finish and the other two snapshots to become aborted"); + assertBusy(() -> { + assertThat(currentSnapshots(repoName), hasSize(2)); + for (SnapshotsInProgress.Entry entry + : clusterService().state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) { + assertThat(entry.state(), is(SnapshotsInProgress.State.ABORTED)); + assertThat(entry.snapshot().getSnapshotId().getName(), not(secondSnapshot)); + } + }, 30L, TimeUnit.SECONDS); + + logger.info("--> stopping current master node"); + internalCluster().stopCurrentMasterNode(); + + unblockNode(repoName, dataNode); + unblockNode(repoName, dataNode2); + + assertAcked(firstDeleteFuture.get()); + assertAcked(deleteAllSnapshots.get()); + expectThrows(SnapshotException.class, snapshotThreeFuture::actionGet); + + logger.info("--> verify that all snapshots are gone and no more work is left in the cluster state"); + assertBusy(() -> { + assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty()); + final ClusterState state = clusterService().state(); + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE); + assertThat(snapshotsInProgress.entries(), empty()); + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + assertThat(snapshotDeletionsInProgress.getEntries(), empty()); + }, 30L, TimeUnit.SECONDS); + } + + public void testAssertMultipleSnapshotsAndPrimaryFailOver() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + + final String testIndex = "index-one"; + createIndex(testIndex, Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).put(SETTING_NUMBER_OF_REPLICAS, 1).build()); + ensureYellow(testIndex); + indexDoc(testIndex, "some_id", "foo", "bar"); + + blockDataNode(repoName, dataNode); + final ActionFuture firstSnapshotResponse = startFullSnapshotFromMasterClient(repoName, "snapshot-one"); + waitForBlock(dataNode, repoName, TimeValue.timeValueSeconds(30L)); + + internalCluster().startDataOnlyNode(); + ensureStableCluster(3); + ensureGreen(testIndex); + + final String secondSnapshot = "snapshot-two"; + final ActionFuture secondSnapshotResponse = startFullSnapshotFromMasterClient(repoName, secondSnapshot); + + internalCluster().restartNode(dataNode, InternalTestCluster.EMPTY_CALLBACK); + + assertThat(firstSnapshotResponse.get().getSnapshotInfo().state(), is(SnapshotState.PARTIAL)); + assertThat(secondSnapshotResponse.get().getSnapshotInfo().state(), is(SnapshotState.PARTIAL)); + } + + public void testQueuedDeletesWithFailures() throws Exception { + final String masterNode = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + blockMasterFromFinalizingSnapshotOnIndexFile(repoName); + final ActionFuture firstDeleteFuture = startDelete(repoName, "*"); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + final ActionFuture snapshotFuture = startFullSnapshot(repoName, "snapshot-queued"); + awaitNSnapshotsInProgress(1); + + final ActionFuture secondDeleteFuture = startDelete(repoName, "*"); + awaitNDeletionsInProgress(2); + + unblockNode(repoName, masterNode); + expectThrows(UncategorizedExecutionException.class, firstDeleteFuture::actionGet); + + // Second delete works out cleanly since the repo is unblocked now + assertThat(secondDeleteFuture.get().isAcknowledged(), is(true)); + // Snapshot should have been aborted + assertThat(snapshotFuture.get().getSnapshotInfo().state(), is(SnapshotState.FAILED)); + + assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty()); + } + + public void testQueuedDeletesWithOverlap() throws Exception { + final String masterNode = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final ActionFuture firstDeleteFuture = startAndBlockOnDeleteSnapshot(repoName, "*"); + final ActionFuture snapshotFuture = startFullSnapshot(repoName, "snapshot-queued"); + awaitNSnapshotsInProgress(1); + + final ActionFuture secondDeleteFuture = startDelete(repoName, "*"); + awaitNDeletionsInProgress(2); + + unblockNode(repoName, masterNode); + assertThat(firstDeleteFuture.get().isAcknowledged(), is(true)); + + // Second delete works out cleanly since the repo is unblocked now + assertThat(secondDeleteFuture.get().isAcknowledged(), is(true)); + // Snapshot should have been aborted + assertThat(snapshotFuture.get().getSnapshotInfo().state(), is(SnapshotState.FAILED)); + + assertThat(client().admin().cluster().prepareGetSnapshots(repoName).get().getSnapshots(repoName), empty()); + } + + public void testQueuedOperationsOnMasterRestart() throws Exception { + internalCluster().startMasterOnlyNodes(3); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + startAndBlockOnDeleteSnapshot(repoName, "*"); + + client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-three").setWaitForCompletion(false).get(); + + startDelete(repoName, "*"); + awaitNDeletionsInProgress(2); + + internalCluster().stopCurrentMasterNode(); + ensureStableCluster(3); + + awaitNoMoreRunningOperations(); + } + + public void testQueuedOperationsOnMasterDisconnect() throws Exception { + internalCluster().startMasterOnlyNodes(3); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final String masterNode = internalCluster().getMasterName(); + final NetworkDisruption networkDisruption = isolateMasterDisruption(NetworkDisruption.DISCONNECT); + internalCluster().setDisruptionScheme(networkDisruption); + + blockNodeOnAnyFiles(repoName, masterNode); + ActionFuture firstDeleteFuture = client(masterNode).admin().cluster() + .prepareDeleteSnapshot(repoName, "*").execute(); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + final ActionFuture createThirdSnapshot = client(masterNode).admin().cluster() + .prepareCreateSnapshot(repoName, "snapshot-three").setWaitForCompletion(true).execute(); + awaitNSnapshotsInProgress(1); + + final ActionFuture secondDeleteFuture = + client(masterNode).admin().cluster().prepareDeleteSnapshot(repoName, "*").execute(); + awaitNDeletionsInProgress(2); + + networkDisruption.startDisrupting(); + ensureStableCluster(3, dataNode); + unblockNode(repoName, masterNode); + networkDisruption.stopDisrupting(); + + logger.info("--> make sure all failing requests get a response"); + expectThrows(RepositoryException.class, firstDeleteFuture::actionGet); + expectThrows(RepositoryException.class, secondDeleteFuture::actionGet); + expectThrows(SnapshotException.class, createThirdSnapshot::actionGet); + + awaitNoMoreRunningOperations(); + } + + public void testQueuedOperationsOnMasterDisconnectAndRepoFailure() throws Exception { + internalCluster().startMasterOnlyNodes(3); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final String masterNode = internalCluster().getMasterName(); + final NetworkDisruption networkDisruption = isolateMasterDisruption(NetworkDisruption.DISCONNECT); + internalCluster().setDisruptionScheme(networkDisruption); + + blockMasterFromFinalizingSnapshotOnIndexFile(repoName); + final ActionFuture firstFailedSnapshotFuture = + startFullSnapshotFromMasterClient(repoName, "failing-snapshot-1"); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + final ActionFuture secondFailedSnapshotFuture = + startFullSnapshotFromMasterClient(repoName, "failing-snapshot-2"); + awaitNSnapshotsInProgress(2); + + final ActionFuture failedDeleteFuture = + client(masterNode).admin().cluster().prepareDeleteSnapshot(repoName, "*").execute(); + awaitNDeletionsInProgress(1); + + networkDisruption.startDisrupting(); + ensureStableCluster(3, dataNode); + unblockNode(repoName, masterNode); + networkDisruption.stopDisrupting(); + + logger.info("--> make sure all failing requests get a response"); + expectThrows(SnapshotException.class, firstFailedSnapshotFuture::actionGet); + expectThrows(SnapshotException.class, secondFailedSnapshotFuture::actionGet); + expectThrows(RepositoryException.class, failedDeleteFuture::actionGet); + + awaitNoMoreRunningOperations(); + } + + public void testQueuedOperationsAndBrokenRepoOnMasterFailOver() throws Exception { + disableRepoConsistencyCheck("This test corrupts the repository on purpose"); + + internalCluster().startMasterOnlyNodes(3); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final long generation = getRepositoryData(repoName).getGenId(); + + startAndBlockOnDeleteSnapshot(repoName, "*"); + + corruptIndexN(repoPath, generation); + + client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-three").setWaitForCompletion(false).get(); + + final ActionFuture deleteFuture = startDeleteFromNonMasterClient(repoName, "*"); + awaitNDeletionsInProgress(2); + + internalCluster().stopCurrentMasterNode(); + ensureStableCluster(3); + + awaitNoMoreRunningOperations(); + expectThrows(RepositoryException.class, deleteFuture::actionGet); + } + + public void testQueuedSnapshotOperationsAndBrokenRepoOnMasterFailOver() throws Exception { + disableRepoConsistencyCheck("This test corrupts the repository on purpose"); + + internalCluster().startMasterOnlyNodes(3); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final long generation = getRepositoryData(repoName).getGenId(); + final String masterNode = internalCluster().getMasterName(); + blockNodeOnAnyFiles(repoName, masterNode); + final ActionFuture snapshotThree = startFullSnapshotFromNonMasterClient(repoName, "snapshot-three"); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + corruptIndexN(repoPath, generation); + + final ActionFuture snapshotFour = startFullSnapshotFromNonMasterClient(repoName, "snapshot-four"); + internalCluster().stopCurrentMasterNode(); + ensureStableCluster(3); + + awaitNoMoreRunningOperations(); + expectThrows(ElasticsearchException.class, snapshotThree::actionGet); + expectThrows(ElasticsearchException.class, snapshotFour::actionGet); + } + + public void testQueuedSnapshotOperationsAndBrokenRepoOnMasterFailOver2() throws Exception { + disableRepoConsistencyCheck("This test corrupts the repository on purpose"); + + internalCluster().startMasterOnlyNodes(3); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final long generation = getRepositoryData(repoName).getGenId(); + final String masterNode = internalCluster().getMasterName(); + blockMasterFromFinalizingSnapshotOnIndexFile(repoName); + final ActionFuture snapshotThree = startFullSnapshotFromNonMasterClient(repoName, "snapshot-three"); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + corruptIndexN(repoPath, generation); + + final ActionFuture snapshotFour = startFullSnapshotFromNonMasterClient(repoName, "snapshot-four"); + awaitNSnapshotsInProgress(2); + + final NetworkDisruption networkDisruption = isolateMasterDisruption(NetworkDisruption.DISCONNECT); + internalCluster().setDisruptionScheme(networkDisruption); + networkDisruption.startDisrupting(); + ensureStableCluster(3, dataNode); + unblockNode(repoName, masterNode); + networkDisruption.stopDisrupting(); + awaitNoMoreRunningOperations(); + expectThrows(ElasticsearchException.class, snapshotThree::actionGet); + expectThrows(ElasticsearchException.class, snapshotFour::actionGet); + } + + public void testQueuedSnapshotOperationsAndBrokenRepoOnMasterFailOverMultipleRepos() throws Exception { + disableRepoConsistencyCheck("This test corrupts the repository on purpose"); + + internalCluster().startMasterOnlyNodes(3, LARGE_SNAPSHOT_POOL_SETTINGS); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + + final String masterNode = internalCluster().getMasterName(); + + final String blockedRepoName = "repo-blocked"; + createRepository(blockedRepoName, "mock"); + createNSnapshots(blockedRepoName, randomIntBetween(1, 5)); + blockNodeOnAnyFiles(blockedRepoName, masterNode); + final ActionFuture deleteFuture = startDeleteFromNonMasterClient(blockedRepoName, "*"); + waitForBlock(masterNode, blockedRepoName, TimeValue.timeValueSeconds(30L)); + final ActionFuture createBlockedSnapshot = + startFullSnapshotFromNonMasterClient(blockedRepoName, "queued-snapshot"); + + final long generation = getRepositoryData(repoName).getGenId(); + blockNodeOnAnyFiles(repoName, masterNode); + final ActionFuture snapshotThree = startFullSnapshotFromNonMasterClient(repoName, "snapshot-three"); + waitForBlock(masterNode, repoName, TimeValue.timeValueSeconds(30L)); + + corruptIndexN(repoPath, generation); + + final ActionFuture snapshotFour = startFullSnapshotFromNonMasterClient(repoName, "snapshot-four"); + internalCluster().stopCurrentMasterNode(); + ensureStableCluster(3); + + awaitNoMoreRunningOperations(); + expectThrows(ElasticsearchException.class, snapshotThree::actionGet); + expectThrows(ElasticsearchException.class, snapshotFour::actionGet); + assertAcked(deleteFuture.get()); + expectThrows(ElasticsearchException.class, createBlockedSnapshot::actionGet); + } + + public void testMultipleSnapshotsQueuedAfterDelete() throws Exception { + final String masterNode = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(1, 5)); + + final ActionFuture deleteFuture = startAndBlockOnDeleteSnapshot(repoName, "*"); + final ActionFuture snapshotThree = startFullSnapshot(repoName, "snapshot-three"); + final ActionFuture snapshotFour = startFullSnapshot(repoName, "snapshot-four"); + + unblockNode(repoName, masterNode); + + assertSuccessful(snapshotThree); + assertSuccessful(snapshotFour); + assertAcked(deleteFuture.get()); + } + + public void testQueuedSnapshotsWaitingForShardReady() throws Exception { + internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNodes(2); + final String repoName = "test-repo"; + createRepository(repoName, "fs"); + + final String testIndex = "test-idx"; + // Create index on two nodes and make sure each node has a primary by setting no replicas + assertAcked(prepareCreate(testIndex, 2, indexSettingsNoReplicas(between(2, 10)))); + + ensureGreen(testIndex); + + logger.info("--> indexing some data"); + for (int i = 0; i < 100; i++) { + indexDoc(testIndex, Integer.toString(i), "foo", "bar" + i); + } + refresh(); + assertThat(client().prepareSearch(testIndex).setSize(0).get().getHits().getTotalHits().value, equalTo(100L)); + + logger.info("--> start relocations"); + allowNodes(testIndex, 1); + + logger.info("--> wait for relocations to start"); + assertBusy(() -> assertThat( + client().admin().cluster().prepareHealth(testIndex).execute().actionGet().getRelocatingShards(), greaterThan(0)), + 1L, TimeUnit.MINUTES); + + logger.info("--> start two snapshots"); + final String snapshotOne = "snap-1"; + final String snapshotTwo = "snap-2"; + final ActionFuture snapOneResponse = client().admin().cluster() + .prepareCreateSnapshot(repoName, snapshotOne).setWaitForCompletion(false).setIndices(testIndex).execute(); + final ActionFuture snapTwoResponse = client().admin().cluster() + .prepareCreateSnapshot(repoName, snapshotTwo).setWaitForCompletion(false).setIndices(testIndex).execute(); + + snapOneResponse.get(); + snapTwoResponse.get(); + logger.info("--> wait for snapshot to complete"); + for (String snapshot : Arrays.asList(snapshotOne, snapshotTwo)) { + SnapshotInfo snapshotInfo = waitForCompletion(repoName, snapshot, TimeValue.timeValueSeconds(600)); + assertThat(snapshotInfo.state(), equalTo(SnapshotState.SUCCESS)); + assertThat(snapshotInfo.shardFailures().size(), equalTo(0)); + } + } + + public void testBackToBackQueuedDeletes() throws Exception { + final String masterName = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + final List snapshots = createNSnapshots(repoName, 2); + final String snapshotOne = snapshots.get(0); + final String snapshotTwo = snapshots.get(1); + + final ActionFuture deleteSnapshotOne = startAndBlockOnDeleteSnapshot(repoName, snapshotOne); + final ActionFuture deleteSnapshotTwo = startDelete(repoName, snapshotTwo); + awaitNDeletionsInProgress(2); + + unblockNode(repoName, masterName); + assertAcked(deleteSnapshotOne.get()); + assertAcked(deleteSnapshotTwo.get()); + + final RepositoryData repositoryData = getRepositoryData(repoName); + assertThat(repositoryData.getSnapshotIds(), empty()); + // Two snapshots and two distinct delete operations move us 4 steps from -1 to 3 + assertThat(repositoryData.getGenId(), is(3L)); + } + + public void testQueuedOperationsAfterFinalizationFailure() throws Exception { + internalCluster().startMasterOnlyNodes(3); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + + final List snapshotNames = createNSnapshots(repoName, randomIntBetween(2, 5)); + + final ActionFuture snapshotThree = startAndBlockFailingFullSnapshot(repoName, "snap-other"); + + final String masterName = internalCluster().getMasterName(); + + final String snapshotOne = snapshotNames.get(0); + final ActionFuture deleteSnapshotOne = startDelete(repoName, snapshotOne); + awaitNDeletionsInProgress(1); + + unblockNode(repoName, masterName); + + expectThrows(SnapshotException.class, snapshotThree::actionGet); + assertAcked(deleteSnapshotOne.get()); + } + + public void testStartDeleteDuringFinalizationCleanup() throws Exception { + final String masterName = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + createNSnapshots(repoName, randomIntBetween(1, 5)); + final String snapshotName = "snap-name"; + blockMasterFromDeletingIndexNFile(repoName); + final ActionFuture snapshotFuture = startFullSnapshot(repoName, snapshotName); + waitForBlock(masterName, repoName, TimeValue.timeValueSeconds(30L)); + final ActionFuture deleteFuture = startDelete(repoName, snapshotName); + awaitNDeletionsInProgress(1); + unblockNode(repoName, masterName); + assertSuccessful(snapshotFuture); + assertAcked(deleteFuture.get(30L, TimeUnit.SECONDS)); + } + + public void testEquivalentDeletesAreDeduplicated() throws Exception { + final String masterName = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + createNSnapshots(repoName, randomIntBetween(1, 5)); + + blockMasterFromDeletingIndexNFile(repoName); + final int deletes = randomIntBetween(2, 10); + final List> deleteResponses = new ArrayList<>(deletes); + for (int i = 0; i < deletes; ++i) { + deleteResponses.add(client().admin().cluster().prepareDeleteSnapshot(repoName, "*").execute()); + } + waitForBlock(masterName, repoName, TimeValue.timeValueSeconds(30L)); + awaitNDeletionsInProgress(1); + for (ActionFuture deleteResponse : deleteResponses) { + assertFalse(deleteResponse.isDone()); + } + awaitNDeletionsInProgress(1); + unblockNode(repoName, masterName); + for (ActionFuture deleteResponse : deleteResponses) { + assertAcked(deleteResponse.get()); + } + } + + public void testMasterFailoverOnFinalizationLoop() throws Exception { + internalCluster().startMasterOnlyNodes(3); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + final NetworkDisruption networkDisruption = isolateMasterDisruption(NetworkDisruption.DISCONNECT); + internalCluster().setDisruptionScheme(networkDisruption); + + final List snapshotNames = createNSnapshots(repoName, randomIntBetween(2, 5)); + final String masterName = internalCluster().getMasterName(); + blockMasterFromDeletingIndexNFile(repoName); + final ActionFuture snapshotThree = startFullSnapshotFromMasterClient(repoName, "snap-other"); + waitForBlock(masterName, repoName, TimeValue.timeValueSeconds(30L)); + + final String snapshotOne = snapshotNames.get(0); + final ActionFuture deleteSnapshotOne = startDelete(repoName, snapshotOne); + awaitNDeletionsInProgress(1); + networkDisruption.startDisrupting(); + ensureStableCluster(3, dataNode); + + unblockNode(repoName, masterName); + networkDisruption.stopDisrupting(); + ensureStableCluster(4); + + assertSuccessful(snapshotThree); + try { + deleteSnapshotOne.actionGet(); + } catch (RepositoryException re) { + // ignored + } + awaitNoMoreRunningOperations(); + } + + public void testStatusMultipleSnapshotsMultipleRepos() throws Exception { + internalCluster().startMasterOnlyNode(); + // We're blocking a some of the snapshot threads when we block the first repo below so we have to make sure we have enough threads + // left for the second concurrent snapshot. + final String dataNode = startDataNodeWithLargeSnapshotPool(); + final String blockedRepoName = "test-repo-blocked-1"; + final String otherBlockedRepoName = "test-repo-blocked-2"; + createRepository(blockedRepoName, "mock"); + createRepository(otherBlockedRepoName, "mock"); + createIndexWithContent("test-index"); + + final ActionFuture createSlowFuture1 = + startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + final ActionFuture createSlowFuture2 = + startFullSnapshotBlockedOnDataNode("blocked-snapshot-2", blockedRepoName, dataNode); + final ActionFuture createSlowFuture3 = + startFullSnapshotBlockedOnDataNode("other-blocked-snapshot", otherBlockedRepoName, dataNode); + awaitNSnapshotsInProgress(3); + + assertSnapshotStatusCountOnRepo("_all", 3); + assertSnapshotStatusCountOnRepo(blockedRepoName, 2); + assertSnapshotStatusCountOnRepo(otherBlockedRepoName, 1); + + unblockNode(blockedRepoName, dataNode); + awaitNSnapshotsInProgress(1); + assertSnapshotStatusCountOnRepo("_all", 1); + assertSnapshotStatusCountOnRepo(blockedRepoName, 0); + assertSnapshotStatusCountOnRepo(otherBlockedRepoName, 1); + + unblockNode(otherBlockedRepoName, dataNode); + assertSuccessful(createSlowFuture1); + assertSuccessful(createSlowFuture2); + assertSuccessful(createSlowFuture3); + } + + public void testInterleavedAcrossMultipleRepos() throws Exception { + internalCluster().startMasterOnlyNode(); + // We're blocking a some of the snapshot threads when we block the first repo below so we have to make sure we have enough threads + // left for the second concurrent snapshot. + final String dataNode = startDataNodeWithLargeSnapshotPool(); + final String blockedRepoName = "test-repo-blocked-1"; + final String otherBlockedRepoName = "test-repo-blocked-2"; + createRepository(blockedRepoName, "mock"); + createRepository(otherBlockedRepoName, "mock"); + createIndexWithContent("test-index"); + + final ActionFuture createSlowFuture1 = + startFullSnapshotBlockedOnDataNode("blocked-snapshot", blockedRepoName, dataNode); + final ActionFuture createSlowFuture2 = + startFullSnapshotBlockedOnDataNode("blocked-snapshot-2", blockedRepoName, dataNode); + final ActionFuture createSlowFuture3 = + startFullSnapshotBlockedOnDataNode("other-blocked-snapshot", otherBlockedRepoName, dataNode); + awaitNSnapshotsInProgress(3); + unblockNode(blockedRepoName, dataNode); + unblockNode(otherBlockedRepoName, dataNode); + + assertSuccessful(createSlowFuture1); + assertSuccessful(createSlowFuture2); + assertSuccessful(createSlowFuture3); + } + + public void testMasterFailoverAndMultipleQueuedUpSnapshotsAcrossTwoRepos() throws Exception { + disableRepoConsistencyCheck("This test corrupts the repository on purpose"); + + internalCluster().startMasterOnlyNodes(3, LARGE_SNAPSHOT_POOL_SETTINGS); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final String otherRepoName = "other-test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", repoPath); + createRepository(otherRepoName, "mock"); + createIndexWithContent("index-one"); + createNSnapshots(repoName, randomIntBetween(2, 5)); + final int countOtherRepo = randomIntBetween(2, 5); + createNSnapshots(otherRepoName, countOtherRepo); + + corruptIndexN(repoPath, getRepositoryData(repoName).getGenId()); + + blockMasterFromFinalizingSnapshotOnIndexFile(repoName); + blockMasterFromFinalizingSnapshotOnIndexFile(otherRepoName); + + client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-blocked-1").setWaitForCompletion(false).get(); + client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-blocked-2").setWaitForCompletion(false).get(); + client().admin().cluster().prepareCreateSnapshot(otherRepoName, "snapshot-other-blocked-1").setWaitForCompletion(false).get(); + client().admin().cluster().prepareCreateSnapshot(otherRepoName, "snapshot-other-blocked-2").setWaitForCompletion(false).get(); + + awaitNSnapshotsInProgress(4); + final String initialMaster = internalCluster().getMasterName(); + waitForBlock(initialMaster, repoName, TimeValue.timeValueSeconds(30L)); + waitForBlock(initialMaster, otherRepoName, TimeValue.timeValueSeconds(30L)); + + internalCluster().stopCurrentMasterNode(); + ensureStableCluster(3, dataNode); + awaitNoMoreRunningOperations(); + + final RepositoryData repositoryData = getRepositoryData(otherRepoName); + assertThat(repositoryData.getSnapshotIds(), hasSize(countOtherRepo + 2)); + } + + public void testConcurrentOperationsLimit() throws Exception { + final String masterName = internalCluster().startMasterOnlyNode(); + internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + createRepository(repoName, "mock"); + createIndexWithContent("index-test"); + + final int limitToTest = randomIntBetween(1, 3); + assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(Settings.builder().put( + SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.getKey(), limitToTest).build()).get()); + + final List snapshotNames = createNSnapshots(repoName, limitToTest + 1); + blockNodeOnAnyFiles(repoName, masterName); + int blockedSnapshots = 0; + boolean blockedDelete = false; + final List> snapshotFutures = new ArrayList<>(); + ActionFuture deleteFuture = null; + for (int i = 0; i < limitToTest; ++i) { + if (blockedDelete || randomBoolean()) { + snapshotFutures.add(startFullSnapshot(repoName, "snap-" + i)); + ++blockedSnapshots; + } else { + blockedDelete = true; + deleteFuture = startDelete(repoName, randomFrom(snapshotNames)); + } + } + awaitNSnapshotsInProgress(blockedSnapshots); + if (blockedDelete) { + awaitNDeletionsInProgress(1); + } + waitForBlock(masterName, repoName, TimeValue.timeValueSeconds(30L)); + + final String expectedFailureMessage = "Cannot start another operation, already running [" + limitToTest + + "] operations and the current limit for concurrent snapshot operations is set to [" + limitToTest + "]"; + final ConcurrentSnapshotExecutionException csen1 = expectThrows(ConcurrentSnapshotExecutionException.class, + () -> client().admin().cluster().prepareCreateSnapshot(repoName, "expected-to-fail").execute().actionGet()); + assertThat(csen1.getMessage(), containsString(expectedFailureMessage)); + if (blockedDelete == false || limitToTest == 1) { + final ConcurrentSnapshotExecutionException csen2 = expectThrows(ConcurrentSnapshotExecutionException.class, + () -> client().admin().cluster().prepareDeleteSnapshot(repoName, "*").execute().actionGet()); + assertThat(csen2.getMessage(), containsString(expectedFailureMessage)); + } + + unblockNode(repoName, masterName); + if (deleteFuture != null) { + assertAcked(deleteFuture.get()); + } + for (ActionFuture snapshotFuture : snapshotFutures) { + assertSuccessful(snapshotFuture); + } + } + + public void testConcurrentSnapshotWorksWithOldVersionRepo() throws Exception { + internalCluster().startMasterOnlyNode(); + final String dataNode = internalCluster().startDataOnlyNode(); + final String repoName = "test-repo"; + final Path repoPath = randomRepoPath(); + createRepository(repoName, "mock", Settings.builder().put(BlobStoreRepository.CACHE_REPOSITORY_DATA.getKey(), false) + .put("location", repoPath)); + initWithSnapshotVersion(repoName, repoPath, SnapshotsService.OLD_SNAPSHOT_FORMAT); + + createIndexWithContent("index-slow"); + + final ActionFuture createSlowFuture = + startFullSnapshotBlockedOnDataNode("slow-snapshot", repoName, dataNode); + + final String dataNode2 = internalCluster().startDataOnlyNode(); + ensureStableCluster(3); + final String indexFast = "index-fast"; + createIndexWithContent(indexFast, dataNode2, dataNode); + + final ActionFuture createFastSnapshot = + client().admin().cluster().prepareCreateSnapshot(repoName, "fast-snapshot").setWaitForCompletion(true).execute(); + + assertThat(createSlowFuture.isDone(), is(false)); + unblockNode(repoName, dataNode); + + assertSuccessful(createFastSnapshot); + assertSuccessful(createSlowFuture); + + final RepositoryData repositoryData = getRepositoryData(repoName); + assertThat(repositoryData.shardGenerations(), is(ShardGenerations.EMPTY)); + } + + private static String startDataNodeWithLargeSnapshotPool() { + return internalCluster().startDataOnlyNode(LARGE_SNAPSHOT_POOL_SETTINGS); + } + + private static void assertSnapshotStatusCountOnRepo(String otherBlockedRepoName, int count) { + final SnapshotsStatusResponse snapshotsStatusResponse = + client().admin().cluster().prepareSnapshotStatus(otherBlockedRepoName).get(); + final List snapshotStatuses = snapshotsStatusResponse.getSnapshots(); + assertThat(snapshotStatuses, hasSize(count)); + } + + private List createNSnapshots(String repoName, int count) { + final List snapshotNames = new ArrayList<>(count); + final String prefix = "snap-" + UUIDs.randomBase64UUID(random()).toLowerCase(Locale.ROOT) + "-"; + for (int i = 0; i < count; i++) { + final String name = prefix + i; + createFullSnapshot(repoName, name); + snapshotNames.add(name); + } + logger.info("--> created {} in [{}]", snapshotNames, repoName); + return snapshotNames; + } + + private void awaitNoMoreRunningOperations() throws Exception { + awaitNoMoreRunningOperations(internalCluster().getMasterName()); + } + + private ActionFuture startDeleteFromNonMasterClient(String repoName, String snapshotName) { + logger.info("--> deleting snapshot [{}] from repo [{}] from non master client", snapshotName, repoName); + return internalCluster().nonMasterClient().admin().cluster().prepareDeleteSnapshot(repoName, snapshotName).execute(); + } + + private ActionFuture startDelete(String repoName, String snapshotName) { + logger.info("--> deleting snapshot [{}] from repo [{}]", snapshotName, repoName); + return client().admin().cluster().prepareDeleteSnapshot(repoName, snapshotName).execute(); + } + + private ActionFuture startFullSnapshotFromNonMasterClient(String repoName, String snapshotName) { + logger.info("--> creating full snapshot [{}] to repo [{}] from non master client", snapshotName, repoName); + return internalCluster().nonMasterClient().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true).execute(); + } + + private ActionFuture startFullSnapshotFromMasterClient(String repoName, String snapshotName) { + logger.info("--> creating full snapshot [{}] to repo [{}] from master client", snapshotName, repoName); + return internalCluster().masterClient().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true).execute(); + } + + private ActionFuture startFullSnapshot(String repoName, String snapshotName) { + logger.info("--> creating full snapshot [{}] to repo [{}]", snapshotName, repoName); + return client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName).setWaitForCompletion(true).execute(); + } + + private void awaitClusterState(Predicate statePredicate) throws Exception { + awaitClusterState(internalCluster().getMasterName(), statePredicate); + } + + // Large snapshot pool settings to set up nodes for tests involving multiple repositories that need to have enough + // threads so that blocking some threads on one repository doesn't block other repositories from doing work + private static final Settings LARGE_SNAPSHOT_POOL_SETTINGS = Settings.builder() + .put("thread_pool.snapshot.core", 5).put("thread_pool.snapshot.max", 5).build(); + + private static final Settings SINGLE_SHARD_NO_REPLICA = indexSettingsNoReplicas(1).build(); + + private void createIndexWithContent(String indexName) { + createIndexWithContent(indexName, SINGLE_SHARD_NO_REPLICA); + } + + private void createIndexWithContent(String indexName, String nodeInclude, String nodeExclude) { + createIndexWithContent(indexName, indexSettingsNoReplicas(1) + .put("index.routing.allocation.include._name", nodeInclude) + .put("index.routing.allocation.exclude._name", nodeExclude).build()); + } + + private void createIndexWithContent(String indexName, Settings indexSettings) { + logger.info("--> creating index [{}]", indexName); + createIndex(indexName, indexSettings); + ensureGreen(indexName); + indexDoc(indexName, "some_id", "foo", "bar"); + } + + private static boolean snapshotHasCompletedShard(String snapshot, SnapshotsInProgress snapshotsInProgress) { + for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { + if (entry.snapshot().getSnapshotId().getName().equals(snapshot)) { + for (ObjectCursor shard : entry.shards().values()) { + if (shard.value.state().completed()) { + return true; + } + } + } + } + return false; + } + + private static SnapshotInfo assertSuccessful(ActionFuture future) throws Exception { + final SnapshotInfo snapshotInfo = future.get().getSnapshotInfo(); + assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS)); + return snapshotInfo; + } + + private void corruptIndexN(Path repoPath, long generation) throws IOException { + logger.info("--> corrupting [index-{}] in [{}]", generation, repoPath); + Path indexNBlob = repoPath.resolve(BlobStoreRepository.INDEX_FILE_PREFIX + generation); + assertFileExists(indexNBlob); + Files.write(indexNBlob, randomByteArrayOfLength(1), StandardOpenOption.TRUNCATE_EXISTING); + } + + private void awaitNDeletionsInProgress(int count) throws Exception { + logger.info("--> wait for [{}] deletions to show up in the cluster state", count); + awaitClusterState(state -> + state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries().size() == count); + } + + private void awaitNSnapshotsInProgress(int count) throws Exception { + logger.info("--> wait for [{}] snapshots to show up in the cluster state", count); + awaitClusterState(state -> + state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().size() == count); + } + + private static List currentSnapshots(String repoName) { + return client().admin().cluster().prepareGetSnapshots(repoName).setSnapshots(GetSnapshotsRequest.CURRENT_SNAPSHOT) + .get().getSnapshots(repoName); + } + + private ActionFuture startAndBlockOnDeleteSnapshot(String repoName, String snapshotName) + throws InterruptedException { + final String masterName = internalCluster().getMasterName(); + blockNodeOnAnyFiles(repoName, masterName); + final ActionFuture fut = startDelete(repoName, snapshotName); + waitForBlock(masterName, repoName, TimeValue.timeValueSeconds(30L)); + return fut; + } + + private ActionFuture startAndBlockFailingFullSnapshot(String blockedRepoName, String snapshotName) + throws InterruptedException { + blockMasterFromFinalizingSnapshotOnIndexFile(blockedRepoName); + final ActionFuture fut = startFullSnapshot(blockedRepoName, snapshotName); + waitForBlock(internalCluster().getMasterName(), blockedRepoName, TimeValue.timeValueSeconds(30L)); + return fut; + } + + private ActionFuture startFullSnapshotBlockedOnDataNode(String snapshotName, String repoName, String dataNode) + throws InterruptedException { + blockDataNode(repoName, dataNode); + final ActionFuture fut = startFullSnapshot(repoName, snapshotName); + waitForBlock(dataNode, repoName, TimeValue.timeValueSeconds(30L)); + return fut; + } +} diff --git a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MinThreadsSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MinThreadsSnapshotRestoreIT.java deleted file mode 100644 index 9f437cdf70d35..0000000000000 --- a/server/src/internalClusterTest/java/org/elasticsearch/snapshots/MinThreadsSnapshotRestoreIT.java +++ /dev/null @@ -1,155 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.snapshots; - -import org.elasticsearch.action.ActionFuture; -import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.repositories.RepositoriesService; -import org.elasticsearch.snapshots.mockstore.MockRepository; - -import java.util.Collection; -import java.util.Collections; - -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.hamcrest.Matchers.containsString; - -/** - * Tests for snapshot/restore that require at least 2 threads available - * in the thread pool (for example, tests that use the mock repository that - * block on master). - */ -public class MinThreadsSnapshotRestoreIT extends AbstractSnapshotIntegTestCase { - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("thread_pool.snapshot.core", 2) - .put("thread_pool.snapshot.max", 2) - .build(); - } - - @Override - protected Collection> nodePlugins() { - return Collections.singleton(MockRepository.Plugin.class); - } - - public void testConcurrentSnapshotDeletionsNotAllowed() throws Exception { - logger.info("--> creating repository"); - final String repo = "test-repo"; - assertAcked(client().admin().cluster().preparePutRepository(repo).setType("mock").setSettings( - Settings.builder() - .put("location", randomRepoPath()) - .put("random", randomAlphaOfLength(10)) - .put("wait_after_unblock", 200)).get()); - - logger.info("--> snapshot twice"); - final String index = "test-idx1"; - assertAcked(prepareCreate(index, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); - for (int i = 0; i < 10; i++) { - indexDoc(index, Integer.toString(i), "foo", "bar" + i); - } - refresh(); - final String snapshot1 = "test-snap1"; - client().admin().cluster().prepareCreateSnapshot(repo, snapshot1).setWaitForCompletion(true).get(); - final String index2 = "test-idx2"; - assertAcked(prepareCreate(index2, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); - for (int i = 0; i < 10; i++) { - indexDoc(index2, Integer.toString(i), "foo", "bar" + i); - } - refresh(); - final String snapshot2 = "test-snap2"; - client().admin().cluster().prepareCreateSnapshot(repo, snapshot2).setWaitForCompletion(true).get(); - - String blockedNode = internalCluster().getMasterName(); - ((MockRepository)internalCluster().getInstance(RepositoriesService.class, blockedNode).repository(repo)).blockOnDataFiles(true); - logger.info("--> start deletion of first snapshot"); - ActionFuture future = - client().admin().cluster().prepareDeleteSnapshot(repo, snapshot2).execute(); - logger.info("--> waiting for block to kick in on node [{}]", blockedNode); - waitForBlock(blockedNode, repo, TimeValue.timeValueSeconds(10)); - - logger.info("--> try deleting the second snapshot, should fail because the first deletion is in progress"); - try { - client().admin().cluster().prepareDeleteSnapshot(repo, snapshot1).get(); - fail("should not be able to delete snapshots concurrently"); - } catch (ConcurrentSnapshotExecutionException e) { - assertThat(e.getMessage(), containsString("cannot delete - another snapshot is currently being deleted")); - } - - logger.info("--> unblocking blocked node [{}]", blockedNode); - unblockNode(repo, blockedNode); - - logger.info("--> wait until first snapshot is finished"); - assertAcked(future.actionGet()); - - logger.info("--> delete second snapshot, which should now work"); - client().admin().cluster().prepareDeleteSnapshot(repo, snapshot1).get(); - assertTrue(client().admin().cluster().prepareGetSnapshots(repo).setSnapshots("_all").get().getSnapshots(repo).isEmpty()); - } - - public void testSnapshottingWithInProgressDeletionNotAllowed() throws Exception { - logger.info("--> creating repository"); - final String repo = "test-repo"; - assertAcked(client().admin().cluster().preparePutRepository(repo).setType("mock").setSettings( - Settings.builder() - .put("location", randomRepoPath()) - .put("random", randomAlphaOfLength(10)) - .put("wait_after_unblock", 200)).get()); - - logger.info("--> snapshot"); - final String index = "test-idx"; - assertAcked(prepareCreate(index, 1, Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0))); - for (int i = 0; i < 10; i++) { - indexDoc(index, Integer.toString(i), "foo", "bar" + i); - } - refresh(); - final String snapshot1 = "test-snap1"; - client().admin().cluster().prepareCreateSnapshot(repo, snapshot1).setWaitForCompletion(true).get(); - - String blockedNode = internalCluster().getMasterName(); - ((MockRepository)internalCluster().getInstance(RepositoriesService.class, blockedNode).repository(repo)).blockOnDataFiles(true); - logger.info("--> start deletion of snapshot"); - ActionFuture future = client().admin().cluster().prepareDeleteSnapshot(repo, snapshot1).execute(); - logger.info("--> waiting for block to kick in on node [{}]", blockedNode); - waitForBlock(blockedNode, repo, TimeValue.timeValueSeconds(10)); - - logger.info("--> try creating a second snapshot, should fail because the deletion is in progress"); - final String snapshot2 = "test-snap2"; - try { - client().admin().cluster().prepareCreateSnapshot(repo, snapshot2).setWaitForCompletion(true).get(); - fail("should not be able to create a snapshot while another is being deleted"); - } catch (ConcurrentSnapshotExecutionException e) { - assertThat(e.getMessage(), containsString("cannot snapshot while a snapshot deletion is in-progress")); - } - - logger.info("--> unblocking blocked node [{}]", blockedNode); - unblockNode(repo, blockedNode); - - logger.info("--> wait until snapshot deletion is finished"); - assertAcked(future.actionGet()); - - logger.info("--> creating second snapshot, which should now work"); - client().admin().cluster().prepareCreateSnapshot(repo, snapshot2).setWaitForCompletion(true).get(); - assertEquals(1, client().admin().cluster().prepareGetSnapshots(repo).setSnapshots("_all").get().getSnapshots(repo).size()); - } -} diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 678c7ca426009..b3fcb36dad24c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -196,6 +196,7 @@ private void buildResponse(SnapshotsInProgress snapshotsInProgress, SnapshotsSta break; case INIT: case WAITING: + case QUEUED: stage = SnapshotIndexShardStage.STARTED; break; case SUCCESS: diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java index 0d8c2c8a58822..364c9b623e59f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotDeletionsInProgress.java @@ -21,21 +21,25 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterState.Custom; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.UUIDs; 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.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.repositories.RepositoryData; import org.elasticsearch.repositories.RepositoryOperation; import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.snapshots.SnapshotsService; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; /** * A class that represents the snapshot deletions that are in progress in the cluster. @@ -51,6 +55,8 @@ public class SnapshotDeletionsInProgress extends AbstractNamedDiffable i private SnapshotDeletionsInProgress(List entries) { this.entries = entries; + assert entries.size() == entries.stream().map(Entry::uuid).distinct().count() : "Found duplicate UUIDs in entries " + entries; + assert assertNoConcurrentDeletionsForSameRepository(entries); } public static SnapshotDeletionsInProgress of(List entries) { @@ -61,7 +67,18 @@ public static SnapshotDeletionsInProgress of(List entries) { + final Set activeRepositories = new HashSet<>(); + for (Entry entry : entries) { + if (entry.state() == State.STARTED) { + final boolean added = activeRepositories.add(entry.repository()); + assert added : "Found multiple running deletes for a single repository in " + entries; + } + } + return true; } /** @@ -83,13 +100,20 @@ public SnapshotDeletionsInProgress withAddedEntry(Entry entry) { } /** - * Returns a new instance of {@link SnapshotDeletionsInProgress} which removes - * the given entry from the invoking instance. + * Returns a new instance of {@link SnapshotDeletionsInProgress} that has the entry with the given {@code deleteUUID} removed from its + * entries. */ - public SnapshotDeletionsInProgress withRemovedEntry(Entry entry) { - List entries = new ArrayList<>(getEntries()); - entries.remove(entry); - return SnapshotDeletionsInProgress.of(entries); + public SnapshotDeletionsInProgress withRemovedEntry(String deleteUUID) { + List updatedEntries = new ArrayList<>(entries.size() - 1); + boolean removed = false; + for (Entry entry : entries) { + if (entry.uuid().equals(deleteUUID)) { + removed = true; + } else { + updatedEntries.add(entry); + } + } + return removed ? SnapshotDeletionsInProgress.of(updatedEntries) : this; } /** @@ -183,17 +207,23 @@ public String toString() { public static final class Entry implements Writeable, RepositoryOperation { private final List snapshots; private final String repoName; + private final State state; private final long startTime; private final long repositoryStateId; + private final String uuid; + + public Entry(List snapshots, String repoName, long startTime, long repositoryStateId, State state) { + this(snapshots, repoName, startTime, repositoryStateId, state, UUIDs.randomBase64UUID()); + } - public Entry(List snapshots, String repoName, long startTime, long repositoryStateId) { + private Entry(List snapshots, String repoName, long startTime, long repositoryStateId, State state, String uuid) { this.snapshots = snapshots; assert snapshots.size() == new HashSet<>(snapshots).size() : "Duplicate snapshot ids in " + snapshots; this.repoName = repoName; this.startTime = startTime; this.repositoryStateId = repositoryStateId; - assert repositoryStateId > RepositoryData.EMPTY_REPO_GEN : - "Can't delete based on an empty or unknown repository generation but saw [" + repositoryStateId + "]"; + this.state = state; + this.uuid = uuid; } public Entry(StreamInput in) throws IOException { @@ -201,6 +231,43 @@ public Entry(StreamInput in) throws IOException { this.snapshots = in.readList(SnapshotId::new); this.startTime = in.readVLong(); this.repositoryStateId = in.readLong(); + if (in.getVersion().onOrAfter(SnapshotsService.FULL_CONCURRENCY_VERSION)) { + this.state = State.readFrom(in); + this.uuid = in.readString(); + } else { + this.state = State.STARTED; + this.uuid = IndexMetadata.INDEX_UUID_NA_VALUE; + } + } + + public Entry started() { + assert state == State.WAITING; + return new Entry(snapshots, repository(), startTime, repositoryStateId, State.STARTED, uuid); + } + + public Entry withAddedSnapshots(Collection newSnapshots) { + assert state == State.WAITING; + final Collection updatedSnapshots = new HashSet<>(snapshots); + if (updatedSnapshots.addAll(newSnapshots) == false) { + return this; + } + return new Entry(List.copyOf(updatedSnapshots), repository(), startTime, repositoryStateId, State.WAITING, uuid); + } + + public Entry withSnapshots(Collection snapshots) { + return new Entry(List.copyOf(snapshots), repository(), startTime, repositoryStateId, state, uuid); + } + + public Entry withRepoGen(long repoGen) { + return new Entry(snapshots, repository(), startTime, repoGen, state, uuid); + } + + public State state() { + return state; + } + + public String uuid() { + return uuid; } public List getSnapshots() { @@ -226,12 +293,14 @@ public boolean equals(Object o) { return repoName.equals(that.repoName) && snapshots.equals(that.snapshots) && startTime == that.startTime - && repositoryStateId == that.repositoryStateId; + && repositoryStateId == that.repositoryStateId + && state == that.state + && uuid.equals(that.uuid); } @Override public int hashCode() { - return Objects.hash(snapshots, repoName, startTime, repositoryStateId); + return Objects.hash(snapshots, repoName, startTime, repositoryStateId, state, uuid); } @Override @@ -240,6 +309,10 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(snapshots); out.writeVLong(startTime); out.writeLong(repositoryStateId); + if (out.getVersion().onOrAfter(SnapshotsService.FULL_CONCURRENCY_VERSION)) { + state.writeTo(out); + out.writeString(uuid); + } } @Override @@ -251,5 +324,47 @@ public String repository() { public long repositoryStateId() { return repositoryStateId; } + + @Override + public String toString() { + return "SnapshotDeletionsInProgress.Entry[[" + uuid + "][" + state + "]" + snapshots + "]"; + } + } + + public enum State implements Writeable { + + /** + * Delete is waiting to execute because there are snapshots and or a delete operation that has to complete before this delete may + * run. + */ + WAITING((byte) 0), + + /** + * Delete is physically executing on the repository. + */ + STARTED((byte) 1); + + private final byte value; + + State(byte value) { + this.value = value; + } + + public static State readFrom(StreamInput in) throws IOException { + final byte value = in.readByte(); + switch (value) { + case 0: + return WAITING; + case 1: + return STARTED; + default: + throw new IllegalArgumentException("No snapshot delete state for value [" + value + "]"); + } + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeByte(value); + } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java index 71e49ee88498c..0c9a7cc91e97d 100644 --- a/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/elasticsearch/cluster/SnapshotsInProgress.java @@ -41,6 +41,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -151,6 +152,18 @@ public Entry(Entry entry, ImmutableOpenMap shards) this(entry, entry.state, shards, entry.failure); } + public Entry withRepoGen(long newRepoGen) { + assert newRepoGen > repositoryStateId : "Updated repository generation [" + newRepoGen + + "] must be higher than current generation [" + repositoryStateId + "]"; + return new Entry(snapshot, includeGlobalState, partial, state, indices, dataStreams, startTime, newRepoGen, shards, failure, + userMetadata, version); + } + + public Entry withShards(ImmutableOpenMap shards) { + return new Entry(snapshot, includeGlobalState, partial, state, indices, dataStreams, startTime, repositoryStateId, shards, + failure, userMetadata, version); + } + @Override public String repository() { return snapshot.getRepository(); @@ -308,7 +321,16 @@ public static boolean completed(ObjectContainer shards) { } public static class ShardSnapshotStatus { + + /** + * Shard snapshot status for shards that are waiting for another operation to finish before they can be assigned to a node. + */ + public static final ShardSnapshotStatus UNASSIGNED_QUEUED = + new SnapshotsInProgress.ShardSnapshotStatus(null, ShardState.QUEUED, null); + private final ShardState state; + + @Nullable private final String nodeId; @Nullable @@ -321,17 +343,23 @@ public ShardSnapshotStatus(String nodeId, String generation) { this(nodeId, ShardState.INIT, generation); } - public ShardSnapshotStatus(String nodeId, ShardState state, String generation) { + public ShardSnapshotStatus(@Nullable String nodeId, ShardState state, @Nullable String generation) { this(nodeId, state, null, generation); } - public ShardSnapshotStatus(String nodeId, ShardState state, String reason, String generation) { + public ShardSnapshotStatus(@Nullable String nodeId, ShardState state, String reason, @Nullable String generation) { this.nodeId = nodeId; this.state = state; this.reason = reason; this.generation = generation; + assert assertConsistent(); + } + + private boolean assertConsistent() { // If the state is failed we have to have a reason for this failure assert state.failed() == false || reason != null; + assert (state != ShardState.INIT && state != ShardState.WAITING) || nodeId != null : "Null node id for state [" + state + "]"; + return true; } public ShardSnapshotStatus(StreamInput in) throws IOException { @@ -349,10 +377,12 @@ public ShardState state() { return state; } + @Nullable public String nodeId() { return nodeId; } + @Nullable public String generation() { return this.generation; } @@ -361,6 +391,15 @@ public String reason() { return reason; } + /** + * Checks if this shard snapshot is actively executing. + * A shard is defined as actively executing if it either is in a state that may write to the repository + * ({@link ShardState#INIT} or {@link ShardState#ABORTED}) or about to write to it in state {@link ShardState#WAITING}. + */ + public boolean isActive() { + return state == ShardState.INIT || state == ShardState.ABORTED || state == ShardState.WAITING; + } + public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(nodeId); out.writeByte(state.value); @@ -438,6 +477,19 @@ public static State fromValue(byte value) { private final List entries; + private static boolean assertConsistentEntries(List entries) { + final Map> assignedShardsByRepo = new HashMap<>(); + for (Entry entry : entries) { + for (ObjectObjectCursor shard : entry.shards()) { + if (shard.value.isActive()) { + assert assignedShardsByRepo.computeIfAbsent(entry.repository(), k -> new HashSet<>()).add(shard.key) : + "Found duplicate shard assignments in " + entries; + } + } + } + return true; + } + public static SnapshotsInProgress of(List entries) { if (entries.isEmpty()) { return EMPTY; @@ -447,6 +499,7 @@ public static SnapshotsInProgress of(List entries) { private SnapshotsInProgress(List entries) { this.entries = entries; + assert assertConsistentEntries(entries); } public List entries() { @@ -581,7 +634,14 @@ public enum ShardState { FAILED((byte) 3, true, true), ABORTED((byte) 4, false, true), MISSING((byte) 5, true, true), - WAITING((byte) 6, false, false); + /** + * Shard snapshot is waiting for the primary to snapshot to become available. + */ + WAITING((byte) 6, false, false), + /** + * Shard snapshot is waiting for another shard snapshot for the same shard and to the same repository to finish. + */ + QUEUED((byte) 7, false, false); private final byte value; @@ -617,6 +677,8 @@ public static ShardState fromValue(byte value) { return MISSING; case 6: return WAITING; + case 7: + return QUEUED; default: throw new IllegalArgumentException("No shard snapshot state for value [" + value + "]"); } 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 39bf7847b10bd..1f16a6cec36d8 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -109,6 +109,7 @@ import org.elasticsearch.search.SearchService; import org.elasticsearch.search.aggregations.MultiBucketConsumerService; import org.elasticsearch.search.fetch.subphase.highlight.FastVectorHighlighter; +import org.elasticsearch.snapshots.SnapshotsService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ProxyConnectionStrategy; import org.elasticsearch.transport.RemoteClusterService; @@ -484,6 +485,7 @@ public void apply(Settings value, Settings current, Settings previous) { LagDetector.CLUSTER_FOLLOWER_LAG_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_CONNECT_TIMEOUT_SETTING, HandshakingTransportAddressConnector.PROBE_HANDSHAKE_TIMEOUT_SETTING, + SnapshotsService.MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, FsHealthService.ENABLED_SETTING, FsHealthService.REFRESH_INTERVAL_SETTING, FsHealthService.SLOW_PATH_LOGGING_THRESHOLD_SETTING, diff --git a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java index ce4eab29bc978..a27301724ac87 100644 --- a/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/FilterRepository.java @@ -86,7 +86,7 @@ public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryS @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { in.deleteSnapshots(snapshotIds, repositoryStateId, repositoryMetaVersion, listener); } diff --git a/server/src/main/java/org/elasticsearch/repositories/Repository.java b/server/src/main/java/org/elasticsearch/repositories/Repository.java index b15fa5fa77763..ec0fb5b561ad1 100644 --- a/server/src/main/java/org/elasticsearch/repositories/Repository.java +++ b/server/src/main/java/org/elasticsearch/repositories/Repository.java @@ -140,7 +140,7 @@ void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryStateId, * @param listener completion listener */ void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener); + ActionListener listener); /** * Returns snapshot throttle time in nanoseconds */ diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java index d660088b374fc..6f13ae232422b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoryData.java @@ -410,14 +410,18 @@ public List resolveIndices(final List indices) { /** * Resolve the given index names to index ids, creating new index ids for * new indices in the repository. + * + * @param indicesToResolve names of indices to resolve + * @param inFlightIds name to index mapping for currently in-flight snapshots not yet in the repository data to fall back to */ - public List resolveNewIndices(final List indicesToResolve) { + public List resolveNewIndices(List indicesToResolve, Map inFlightIds) { List snapshotIndices = new ArrayList<>(); for (String index : indicesToResolve) { - final IndexId indexId; - if (indices.containsKey(index)) { - indexId = indices.get(index); - } else { + IndexId indexId = indices.get(index); + if (indexId == null) { + indexId = inFlightIds.get(index); + } + if (indexId == null) { indexId = new IndexId(index, UUIDs.randomBase64UUID()); } snapshotIndices.add(indexId); 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 7665068f27d78..c2e7588e575b1 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -427,7 +427,6 @@ public void updateState(ClusterState state) { private long bestGeneration(Collection operations) { final String repoName = metadata.name(); - assert operations.size() <= 1 : "Assumed one or no operations but received " + operations; return operations.stream().filter(e -> e.repository().equals(repoName)).mapToLong(RepositoryOperation::repositoryStateId) .max().orElse(RepositoryData.EMPTY_REPO_GEN); } @@ -544,7 +543,7 @@ public RepositoryStats stats() { @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { if (isReadOnly()) { listener.onFailure(new RepositoryException(metadata.name(), "cannot delete snapshot from a readonly repository")); } else { @@ -618,7 +617,7 @@ private RepositoryData safeRepositoryData(long repositoryStateId, Map snapshotIds, long repositoryStateId, Map foundIndices, Map rootBlobs, RepositoryData repositoryData, Version repoMetaVersion, - ActionListener listener) { + ActionListener listener) { if (SnapshotsService.useShardGenerations(repoMetaVersion)) { // First write the new shard state metadata (with the removed snapshot) and compute deletion targets @@ -639,13 +638,13 @@ private void doDeleteShardSnapshots(Collection snapshotIds, long rep } final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, builder.build()); writeIndexGen(updatedRepoData, repositoryStateId, repoMetaVersion, Function.identity(), - ActionListener.wrap(v -> writeUpdatedRepoDataStep.onResponse(updatedRepoData), listener::onFailure)); + ActionListener.wrap(writeUpdatedRepoDataStep::onResponse, listener::onFailure)); }, listener::onFailure); // Once we have updated the repository, run the clean-ups writeUpdatedRepoDataStep.whenComplete(updatedRepoData -> { // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = - new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(null)), 2); + new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(updatedRepoData)), 2); asyncCleanupUnlinkedRootAndIndicesBlobs(snapshotIds, foundIndices, rootBlobs, updatedRepoData, afterCleanupsListener); asyncCleanupUnlinkedShardLevelBlobs(repositoryData, snapshotIds, writeShardMetaDataAndComputeDeletesStep.result(), afterCleanupsListener); @@ -653,11 +652,11 @@ private void doDeleteShardSnapshots(Collection snapshotIds, long rep } else { // Write the new repository data first (with the removed snapshot), using no shard generations final RepositoryData updatedRepoData = repositoryData.removeSnapshots(snapshotIds, ShardGenerations.EMPTY); - writeIndexGen(updatedRepoData, repositoryStateId, repoMetaVersion, Function.identity(), ActionListener.wrap(v -> { + writeIndexGen(updatedRepoData, repositoryStateId, repoMetaVersion, Function.identity(), ActionListener.wrap(newRepoData -> { // Run unreferenced blobs cleanup in parallel to shard-level snapshot deletion final ActionListener afterCleanupsListener = - new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(null)), 2); - asyncCleanupUnlinkedRootAndIndicesBlobs(snapshotIds, foundIndices, rootBlobs, updatedRepoData, afterCleanupsListener); + new GroupedActionListener<>(ActionListener.wrap(() -> listener.onResponse(newRepoData)), 2); + asyncCleanupUnlinkedRootAndIndicesBlobs(snapshotIds, foundIndices, rootBlobs, newRepoData, afterCleanupsListener); final StepListener> writeMetaAndComputeDeletesStep = new StepListener<>(); writeUpdatedShardMetaDataAndComputeDeletes(snapshotIds, repositoryData, false, writeMetaAndComputeDeletesStep); writeMetaAndComputeDeletesStep.whenComplete(deleteResults -> @@ -1571,10 +1570,10 @@ public ClusterState execute(ClusterState currentState) { "Tried to update from unexpected pending repo generation [" + meta.pendingGeneration() + "] after write to generation [" + newGen + "]"); } - return stateFilter.apply(ClusterState.builder(currentState).metadata(Metadata.builder(currentState.getMetadata()) - .putCustom(RepositoriesMetadata.TYPE, - currentState.metadata().custom(RepositoriesMetadata.TYPE).withUpdatedGeneration( - metadata.name(), newGen, newGen))).build()); + return updateRepositoryGenerationsIfNecessary(stateFilter.apply(ClusterState.builder(currentState) + .metadata(Metadata.builder(currentState.getMetadata()).putCustom(RepositoriesMetadata.TYPE, + currentState.metadata().custom(RepositoriesMetadata.TYPE) + .withUpdatedGeneration(metadata.name(), newGen, newGen))).build()), expectedGen, newGen); } @Override @@ -1639,6 +1638,45 @@ public void onFailure(Exception e) { return true; } + /** + * Updates the repository generation that running deletes and snapshot finalizations will be based on for this repository if any such + * operations are found in the cluster state while setting the safe repository generation. + * + * @param state cluster state to update + * @param oldGen previous safe repository generation + * @param newGen new safe repository generation + * @return updated cluster state + */ + private ClusterState updateRepositoryGenerationsIfNecessary(ClusterState state, long oldGen, long newGen) { + final String repoName = metadata.name(); + final SnapshotsInProgress updatedSnapshotsInProgress; + boolean changedSnapshots = false; + final List snapshotEntries = new ArrayList<>(); + for (SnapshotsInProgress.Entry entry : state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) { + if (entry.repository().equals(repoName) && entry.repositoryStateId() == oldGen) { + snapshotEntries.add(entry.withRepoGen(newGen)); + changedSnapshots = true; + } else { + snapshotEntries.add(entry); + } + } + updatedSnapshotsInProgress = changedSnapshots ? SnapshotsInProgress.of(snapshotEntries) : null; + final SnapshotDeletionsInProgress updatedDeletionsInProgress; + boolean changedDeletions = false; + final List deletionEntries = new ArrayList<>(); + for (SnapshotDeletionsInProgress.Entry entry : + state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { + if (entry.repository().equals(repoName) && entry.repositoryStateId() == oldGen) { + deletionEntries.add(entry.withRepoGen(newGen)); + changedDeletions = true; + } else { + deletionEntries.add(entry); + } + } + updatedDeletionsInProgress = changedDeletions ? SnapshotDeletionsInProgress.of(deletionEntries) : null; + return SnapshotsService.updateWithSnapshots(state, updatedSnapshotsInProgress, updatedDeletionsInProgress); + } + private RepositoryMetadata getRepoMetadata(ClusterState state) { final RepositoryMetadata repositoryMetadata = state.getMetadata().custom(RepositoriesMetadata.TYPE).repository(metadata.name()); @@ -1721,7 +1759,7 @@ public void snapshotShard(Store store, MapperService mapperService, SnapshotId s final ShardId shardId = store.shardId(); final long startTime = threadPool.absoluteTimeInMillis(); try { - final String generation = ShardGenerations.fixShardGeneration(snapshotStatus.generation()); + final String generation = snapshotStatus.generation(); logger.debug("[{}] [{}] snapshot to [{}] [{}] ...", shardId, snapshotId, metadata.name(), generation); final BlobContainer shardContainer = shardContainer(indexId, shardId); final Set blobs; @@ -2200,8 +2238,6 @@ public BlobStoreIndexShardSnapshot loadShardSnapshot(BlobContainer shardContaine private Tuple buildBlobStoreIndexShardSnapshots(Set blobs, BlobContainer shardContainer, @Nullable String generation) throws IOException { - assert ShardGenerations.fixShardGeneration(generation) == generation - : "Generation must not be numeric but received [" + generation + "]"; if (generation != null) { if (generation.equals(ShardGenerations.NEW_SHARD_GEN)) { return new Tuple<>(BlobStoreIndexShardSnapshots.EMPTY, ShardGenerations.NEW_SHARD_GEN); diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index bc65e17779581..5079498725bf7 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -53,6 +53,7 @@ import org.elasticsearch.repositories.IndexId; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.ShardGenerations; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequestDeduplicator; @@ -250,8 +251,10 @@ private void startNewShards(SnapshotsInProgress.Entry entry, Map() { @Override diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index f288b3c4b7f5e..4c8e2afe675a9 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -69,6 +69,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; @@ -89,12 +90,16 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Deque; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; @@ -103,6 +108,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static java.util.Collections.emptySet; import static java.util.Collections.unmodifiableList; import static org.elasticsearch.cluster.SnapshotsInProgress.completed; @@ -113,6 +119,8 @@ */ public class SnapshotsService extends AbstractLifecycleComponent implements ClusterStateApplier { + public static final Version FULL_CONCURRENCY_VERSION = Version.V_8_0_0; + public static final Version SHARD_GEN_IN_REPO_DATA_VERSION = Version.V_7_6_0; public static final Version INDEX_GEN_IN_REPO_DATA_VERSION = Version.V_8_0_0; @@ -134,6 +142,14 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final Map>>> snapshotCompletionListeners = new ConcurrentHashMap<>(); + /** + * Listeners for snapshot deletion keyed by delete uuid as returned from {@link SnapshotDeletionsInProgress.Entry#uuid()} + */ + private final Map>> snapshotDeletionListeners = new HashMap<>(); + + //Set of repositories currently running either a snapshot finalization or a snapshot delete. + private final Set currentlyFinalizing = Collections.synchronizedSet(new HashSet<>()); + // Set of snapshots that are currently being ended by this node private final Set endingSnapshots = Collections.synchronizedSet(new HashSet<>()); @@ -142,6 +158,18 @@ public class SnapshotsService extends AbstractLifecycleComponent implements Clus private final TransportService transportService; + private final OngoingRepositoryOperations repositoryOperations = new OngoingRepositoryOperations(); + + /** + * 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 the sizes of + * {@link SnapshotsInProgress#entries()} and {@link SnapshotDeletionsInProgress#getEntries()}. + */ + public static final Setting MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING = + Setting.intSetting("snapshot.max_concurrent_operations", 1000, 1, Setting.Property.NodeScope, Setting.Property.Dynamic); + + private volatile int maxConcurrentOperations; + public SnapshotsService(Settings settings, ClusterService clusterService, IndexNameExpressionResolver indexNameExpressionResolver, RepositoriesService repositoriesService, TransportService transportService, ActionFilters actionFilters) { this.clusterService = clusterService; @@ -156,6 +184,9 @@ public SnapshotsService(Settings settings, ClusterService clusterService, IndexN if (DiscoveryNode.isMasterNode(settings)) { // addLowPriorityApplier to make sure that Repository will be created before snapshot clusterService.addLowPriorityApplier(this); + maxConcurrentOperations = MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING.get(settings); + clusterService.getClusterSettings().addSettingsUpdateConsumer(MAX_CONCURRENT_SNAPSHOT_OPERATIONS_SETTING, + i -> maxConcurrentOperations = i); } } @@ -204,26 +235,36 @@ public ClusterState execute(ClusterState currentState) { throw new InvalidSnapshotNameException( repository.getMetadata().name(), snapshotName, "snapshot with the same name already exists"); } + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List runningSnapshots = snapshots.entries(); + if (runningSnapshots.stream().anyMatch(s -> { + final Snapshot running = s.snapshot(); + return running.getRepository().equals(repositoryName) && running.getSnapshotId().getName().equals(snapshotName); + })) { + throw new InvalidSnapshotNameException( + repository.getMetadata().name(), snapshotName, "snapshot with the same name is already in-progress"); + } validate(repositoryName, snapshotName, currentState); + final boolean concurrentOperationsAllowed = currentState.nodes().getMinNodeVersion().onOrAfter(FULL_CONCURRENCY_VERSION); final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - if (deletionsInProgress.hasDeletionsInProgress()) { + currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + if (deletionsInProgress.hasDeletionsInProgress() && concurrentOperationsAllowed == false) { throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, "cannot snapshot while a snapshot deletion is in-progress in [" + deletionsInProgress + "]"); } final RepositoryCleanupInProgress repositoryCleanupInProgress = - currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); if (repositoryCleanupInProgress.hasCleanupInProgress()) { throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, "cannot snapshot while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]"); } - final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); // Fail if there are any concurrently running snapshots. The only exception to this being a snapshot in INIT state from a // previous master that we can simply ignore and remove from the cluster state because we would clean it up from the // cluster state anyway in #applyClusterState. - if (snapshots.entries().stream().anyMatch(entry -> entry.state() != State.INIT)) { + if (concurrentOperationsAllowed == false && runningSnapshots.stream().anyMatch(entry -> entry.state() != State.INIT)) { throw new ConcurrentSnapshotExecutionException(repositoryName, snapshotName, " a snapshot is already running"); } + ensureBelowConcurrencyLimit(repositoryName, snapshotName, snapshots, deletionsInProgress); // Store newSnapshot here to be processed in clusterStateProcessed List indices = Arrays.asList(indexNameExpressionResolver.concreteIndexNames(currentState, request)); @@ -232,10 +273,13 @@ public ClusterState execute(ClusterState currentState) { logger.trace("[{}][{}] creating snapshot for indices [{}]", repositoryName, snapshotName, indices); - final List indexIds = repositoryData.resolveNewIndices(indices); + final List indexIds = repositoryData.resolveNewIndices( + indices, runningSnapshots.stream().filter(entry -> entry.repository().equals(repositoryName)) + .flatMap(entry -> entry.indices().stream()).distinct() + .collect(Collectors.toMap(IndexId::getName, Function.identity()))); final Version version = minCompatibleVersion(currentState.nodes().getMinNodeVersion(), repositoryData, null); - ImmutableOpenMap shards = - shards(currentState, indexIds, useShardGenerations(version), repositoryData); + ImmutableOpenMap shards = shards(snapshots, deletionsInProgress, currentState.metadata(), + currentState.routingTable(), indexIds, useShardGenerations(version), repositoryData, repositoryName); if (request.partial() == false) { Set missing = new HashSet<>(); for (ObjectObjectCursor entry : shards) { @@ -249,11 +293,13 @@ public ClusterState execute(ClusterState currentState) { } } newEntry = new SnapshotsInProgress.Entry( - new Snapshot(repositoryName, snapshotId), request.includeGlobalState(), request.partial(), - State.STARTED, indexIds, dataStreams, threadPool.absoluteTimeInMillis(), repositoryData.getGenId(), shards, - null, userMeta, version); + new Snapshot(repositoryName, snapshotId), request.includeGlobalState(), request.partial(), + State.STARTED, indexIds, dataStreams, threadPool.absoluteTimeInMillis(), repositoryData.getGenId(), shards, + null, userMeta, version); + final List newEntries = new ArrayList<>(runningSnapshots); + newEntries.add(newEntry); return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, - SnapshotsInProgress.of(List.of(newEntry))).build(); + SnapshotsInProgress.of(List.copyOf(newEntries))).build(); } @Override @@ -269,7 +315,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, final Cl listener.onResponse(snapshot); } finally { if (newEntry.state().completed() || newEntry.shards().isEmpty()) { - endSnapshot(newEntry, newState.metadata()); + endSnapshot(newEntry, newState.metadata(), repositoryData); } } } @@ -281,6 +327,17 @@ public TimeValue timeout() { }, "create_snapshot [" + snapshotName + ']', listener::onFailure); } + private void ensureBelowConcurrencyLimit(String repository, String name, SnapshotsInProgress snapshotsInProgress, + SnapshotDeletionsInProgress deletionsInProgress) { + final int inProgressOperations = snapshotsInProgress.entries().size() + deletionsInProgress.getEntries().size(); + final int maxOps = maxConcurrentOperations; + if (inProgressOperations >= maxOps) { + throw new ConcurrentSnapshotExecutionException(repository, name, + "Cannot start another operation, already running [" + inProgressOperations + "] operations and the current" + + " limit for concurrent snapshot operations is set to [" + maxOps + "]"); + } + } + /** * Validates snapshot request * @@ -381,9 +438,9 @@ private static Metadata metadataForSnapshot(SnapshotsInProgress.Entry snapshot, * @param snapshots list of snapshots that will be used as a filter, empty list means no snapshots are filtered * @return list of metadata for currently running snapshots */ - public static List currentSnapshots(SnapshotsInProgress snapshotsInProgress, String repository, + public static List currentSnapshots(@Nullable SnapshotsInProgress snapshotsInProgress, String repository, List snapshots) { - if (snapshotsInProgress.entries().isEmpty()) { + if (snapshotsInProgress == null || snapshotsInProgress.entries().isEmpty()) { return Collections.emptyList(); } if ("_all".equals(repository)) { @@ -431,16 +488,10 @@ public void applyClusterState(ClusterChangedEvent event) { try { if (event.localNodeMaster()) { // We don't remove old master when master flips anymore. So, we need to check for change in master - final SnapshotsInProgress snapshotsInProgress = - event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + SnapshotsInProgress snapshotsInProgress = event.state().custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); final boolean newMaster = event.previousState().nodes().isLocalNodeElectedMaster() == false; - if (snapshotsInProgress.entries().isEmpty() == false) { - processExternalChanges(newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), + processExternalChanges(newMaster || removedNodesCleanupNeeded(snapshotsInProgress, event.nodesDelta().removedNodes()), event.routingTableChanged() && waitingShardsStartedOrUnassigned(snapshotsInProgress, event)); - } - if (newMaster) { - finalizeSnapshotDeletionFromPreviousMaster(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 @@ -456,11 +507,12 @@ public void applyClusterState(ClusterChangedEvent event) { logger.warn("Failed to update snapshot state ", e); } assert assertConsistentWithClusterState(event.state()); + assert assertNoDanglingSnapshots(event.state()); } private boolean assertConsistentWithClusterState(ClusterState state) { - final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE); - if (snapshotsInProgress != null && snapshotsInProgress.entries().isEmpty() == false) { + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + if (snapshotsInProgress.entries().isEmpty() == false) { synchronized (endingSnapshots) { final Set runningSnapshots = Stream.concat( snapshotsInProgress.entries().stream().map(SnapshotsInProgress.Entry::snapshot), @@ -471,29 +523,44 @@ private boolean assertConsistentWithClusterState(ClusterState state) { + snapshotListenerKeys + " but running snapshots are " + runningSnapshots; } } + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = + state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + if (snapshotDeletionsInProgress.hasDeletionsInProgress()) { + synchronized (repositoryOperations.runningDeletions) { + final Set runningDeletes = Stream.concat( + snapshotDeletionsInProgress.getEntries().stream().map(SnapshotDeletionsInProgress.Entry::uuid), + repositoryOperations.runningDeletions.stream()) + .collect(Collectors.toSet()); + final Set deleteListenerKeys = snapshotDeletionListeners.keySet(); + assert runningDeletes.containsAll(deleteListenerKeys) : "Saw deletions listeners for unknown uuids in " + + deleteListenerKeys + " but running deletes are " + runningDeletes; + } + } return true; } - /** - * Finalizes a snapshot deletion in progress if the current node is the master but it - * was not master in the previous cluster state and there is still a lingering snapshot - * deletion in progress in the cluster state. This means that the old master failed - * before it could clean up an in-progress snapshot deletion. We attempt to delete the - * snapshot files and remove the deletion from the cluster state. It is possible that the - * old master was in a state of long GC and then it resumes and tries to delete the snapshot - * that has already been deleted by the current master. This is acceptable however, since - * the old master's snapshot deletion will just respond with an error but in actuality, the - * snapshot was deleted and a call to GET snapshots would reveal that the snapshot no longer exists. - */ - private void finalizeSnapshotDeletionFromPreviousMaster(ClusterState state) { - SnapshotDeletionsInProgress deletionsInProgress = - state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - if (deletionsInProgress.hasDeletionsInProgress()) { - assert deletionsInProgress.getEntries().size() == 1 : "only one in-progress deletion allowed per cluster"; - SnapshotDeletionsInProgress.Entry entry = deletionsInProgress.getEntries().get(0); - deleteSnapshotsFromRepository(entry.repository(), entry.getSnapshots(), null, entry.repositoryStateId(), - state.nodes().getMinNodeVersion()); + // Assert that there are no snapshots that have a shard that is waiting to be assigned even though the cluster state would allow for it + // to be assigned + private static boolean assertNoDanglingSnapshots(ClusterState state) { + final SnapshotsInProgress snapshotsInProgress = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = + state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final Set reposWithRunningDelete = snapshotDeletionsInProgress.getEntries().stream() + .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED) + .map(SnapshotDeletionsInProgress.Entry::repository).collect(Collectors.toSet()); + final Set reposSeen = new HashSet<>(); + for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { + if (reposSeen.add(entry.repository())) { + for (ObjectCursor value : entry.shards().values()) { + if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { + assert reposWithRunningDelete.contains(entry.repository()) + : "Found shard snapshot waiting to be assigned in [" + entry + + "] but it is not blocked by any running delete"; + } + } + } } + return true; } /** @@ -515,11 +582,12 @@ private void processExternalChanges(boolean changedNodes, boolean startShards) { private final Collection finishedSnapshots = new ArrayList<>(); + private final Collection deletionsToExecute = new ArrayList<>(); + @Override public ClusterState execute(ClusterState currentState) { RoutingTable routingTable = currentState.routingTable(); - final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); - assert snapshots != null : "We only submit this kind of update if there have been snapshots before"; + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); DiscoveryNodes nodes = currentState.nodes(); boolean changed = false; final EnumSet statesToUpdate; @@ -531,11 +599,17 @@ public ClusterState execute(ClusterState currentState) { // We are reacting to shards that started only so which only affects the individual shard states of started snapshots statesToUpdate = EnumSet.of(State.STARTED); } - ArrayList entries = new ArrayList<>(); + ArrayList updatedSnapshotEntries = new ArrayList<>(); + + // We keep a cache of shards that failed in this map. If we fail a shardId for a given repository because of + // a node leaving or shard becoming unassigned for one snapshot, we will also fail it for all subsequent enqueued snapshots + // for the same repository + final Map> knownFailures = new HashMap<>(); + for (final SnapshotsInProgress.Entry snapshot : snapshots.entries()) { if (statesToUpdate.contains(snapshot.state())) { - ImmutableOpenMap shards = - processWaitingShardsAndRemovedNodes(snapshot.shards(), routingTable, nodes); + ImmutableOpenMap shards = processWaitingShardsAndRemovedNodes(snapshot.shards(), + routingTable, nodes, knownFailures.computeIfAbsent(snapshot.repository(), k -> new HashMap<>())); if (shards != null) { final SnapshotsInProgress.Entry updatedSnapshot; changed = true; @@ -545,9 +619,9 @@ public ClusterState execute(ClusterState currentState) { } else { updatedSnapshot = new SnapshotsInProgress.Entry(snapshot, shards); } - entries.add(updatedSnapshot); + updatedSnapshotEntries.add(updatedSnapshot); } else { - entries.add(snapshot); + updatedSnapshotEntries.add(snapshot); } } else if (snapshot.repositoryStateId() == RepositoryData.UNKNOWN_REPO_GEN) { // BwC path, older versions could create entries with unknown repo GEN in INIT or ABORTED state that did not yet @@ -559,14 +633,20 @@ public ClusterState execute(ClusterState currentState) { if (snapshot.state().completed() || completed(snapshot.shards().values())) { finishedSnapshots.add(snapshot); } - entries.add(snapshot); + updatedSnapshotEntries.add(snapshot); } } - if (changed) { - return ClusterState.builder(currentState) - .putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(entries)).build(); + final ClusterState res = readyDeletions( + changed ? ClusterState.builder(currentState).putCustom( + SnapshotsInProgress.TYPE, SnapshotsInProgress.of(unmodifiableList(updatedSnapshotEntries))).build() : + currentState).v1(); + for (SnapshotDeletionsInProgress.Entry delete + : res.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { + if (delete.state() == SnapshotDeletionsInProgress.State.STARTED) { + deletionsToExecute.add(delete); + } } - return currentState; + return res; } @Override @@ -577,19 +657,53 @@ public void onFailure(String source, Exception e) { @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - finishedSnapshots.forEach(entry -> endSnapshot(entry, newState.metadata())); + final SnapshotDeletionsInProgress snapshotDeletionsInProgress = + newState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + if (finishedSnapshots.isEmpty() == false) { + // If we found snapshots that should be finalized as a result of the CS update we try to initiate finalization for them + // unless there is an executing snapshot delete already. If there is an executing snapshot delete we don't have to + // enqueue the snapshot finalizations here because the ongoing delete will take care of that when removing the delete + // from the cluster state + final Set reposWithRunningDeletes = snapshotDeletionsInProgress.getEntries().stream() + .filter(entry -> entry.state() == SnapshotDeletionsInProgress.State.STARTED) + .map(SnapshotDeletionsInProgress.Entry::repository).collect(Collectors.toSet()); + for (SnapshotsInProgress.Entry entry : finishedSnapshots) { + if (reposWithRunningDeletes.contains(entry.repository()) == false) { + endSnapshot(entry, newState.metadata(), null); + } + } + } + // run newly ready deletes + for (SnapshotDeletionsInProgress.Entry entry : deletionsToExecute) { + if (tryEnterRepoLoop(entry.repository())) { + deleteSnapshotsFromRepository(entry, newState.nodes().getMinNodeVersion()); + } + } } }); } private static ImmutableOpenMap processWaitingShardsAndRemovedNodes( - ImmutableOpenMap snapshotShards, RoutingTable routingTable, DiscoveryNodes nodes) { + ImmutableOpenMap snapshotShards, RoutingTable routingTable, DiscoveryNodes nodes, + Map knownFailures) { boolean snapshotChanged = false; ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); for (ObjectObjectCursor shardEntry : snapshotShards) { ShardSnapshotStatus shardStatus = shardEntry.value; ShardId shardId = shardEntry.key; - if (shardStatus.state() == ShardState.WAITING) { + if (shardStatus.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED)) { + // this shard snapshot is waiting for a previous snapshot to finish execution for this shard + final ShardSnapshotStatus knownFailure = knownFailures.get(shardId); + if (knownFailure == null) { + // if no failure is known for the shard we keep waiting + shards.put(shardId, shardStatus); + } else { + // If a failure is known for an execution we waited on for this shard then we fail with the same exception here + // as well + snapshotChanged = true; + shards.put(shardId, knownFailure); + } + } else if (shardStatus.state() == ShardState.WAITING) { IndexRoutingTable indexShardRoutingTable = routingTable.index(shardId.getIndex()); if (indexShardRoutingTable != null) { IndexShardRoutingTable shardRouting = indexShardRoutingTable.shard(shardId.id()); @@ -599,7 +713,7 @@ private static ImmutableOpenMap processWaitingShar snapshotChanged = true; logger.trace("starting shard that we were waiting for [{}] on node [{}]", shardId, shardStatus.nodeId()); shards.put(shardId, - new ShardSnapshotStatus(shardRouting.primaryShard().currentNodeId(), shardStatus.generation())); + new ShardSnapshotStatus(shardRouting.primaryShard().currentNodeId(), shardStatus.generation())); continue; } else if (shardRouting.primaryShard().initializing() || shardRouting.primaryShard().relocating()) { // Shard that we were waiting for hasn't started yet or still relocating - will continue to wait @@ -611,8 +725,10 @@ private static ImmutableOpenMap processWaitingShar // Shard that we were waiting for went into unassigned state or disappeared - giving up snapshotChanged = true; logger.warn("failing snapshot of shard [{}] on unassigned shard [{}]", shardId, shardStatus.nodeId()); - shards.put(shardId, new ShardSnapshotStatus( - shardStatus.nodeId(), ShardState.FAILED, "shard is unassigned", shardStatus.generation())); + final ShardSnapshotStatus failedState = new ShardSnapshotStatus(shardStatus.nodeId(), ShardState.FAILED, + "shard is unassigned", shardStatus.generation()); + shards.put(shardId, failedState); + knownFailures.put(shardId, failedState); } else if (shardStatus.state().completed() == false && shardStatus.nodeId() != null) { if (nodes.nodeExists(shardStatus.nodeId())) { shards.put(shardId, shardStatus); @@ -621,8 +737,10 @@ private static ImmutableOpenMap processWaitingShar snapshotChanged = true; logger.warn("failing snapshot of shard [{}] on closed node [{}]", shardId, shardStatus.nodeId()); - shards.put(shardId, - new ShardSnapshotStatus(shardStatus.nodeId(), ShardState.FAILED, "node shutdown", shardStatus.generation())); + final ShardSnapshotStatus failedState = new ShardSnapshotStatus(shardStatus.nodeId(), ShardState.FAILED, + "node shutdown", shardStatus.generation()); + shards.put(shardId, failedState); + knownFailures.put(shardId, failedState); } } else { shards.put(shardId, shardStatus); @@ -685,19 +803,63 @@ private static boolean removedNodesCleanupNeeded(SnapshotsInProgress snapshotsIn } /** - * Finalizes the shard in repository and then removes it from cluster state - *

- * This is non-blocking method that runs on a thread from SNAPSHOT thread pool + * Finalizes the snapshot in the repository. * * @param entry snapshot */ - private void endSnapshot(SnapshotsInProgress.Entry entry, Metadata metadata) { - if (endingSnapshots.add(entry.snapshot()) == false) { - return; + private void endSnapshot(SnapshotsInProgress.Entry entry, Metadata metadata, @Nullable RepositoryData repositoryData) { + final boolean newFinalization = endingSnapshots.add(entry.snapshot()); + final String repoName = entry.repository(); + if (tryEnterRepoLoop(repoName)) { + if (repositoryData == null) { + repositoriesService.repository(repoName).getRepositoryData(new ActionListener<>() { + @Override + public void onResponse(RepositoryData repositoryData) { + finalizeSnapshotEntry(entry, metadata, repositoryData); + } + + @Override + public void onFailure(Exception e) { + clusterService.submitStateUpdateTask("fail repo tasks for [" + repoName + "]", + new FailPendingRepoTasksTask(repoName, e)); + } + }); + } else { + finalizeSnapshotEntry(entry, metadata, repositoryData); + } + } else { + if (newFinalization) { + repositoryOperations.addFinalization(entry, metadata); + } } - final Snapshot snapshot = entry.snapshot(); + } + + /** + * Try starting to run a snapshot finalization or snapshot delete for the given repository. If this method returns + * {@code true} then snapshot finalizations and deletions for the repo may be executed. Once no more operations are + * ready for the repository {@link #leaveRepoLoop(String)} should be invoked so that a subsequent state change that + * causes another operation to become ready can execute. + * + * @return true if a finalization or snapshot delete may be started at this point + */ + private boolean tryEnterRepoLoop(String repository) { + return currentlyFinalizing.add(repository); + } + + /** + * Stop polling for ready snapshot finalizations or deletes in state {@link SnapshotDeletionsInProgress.State#STARTED} to execute + * for the given repository. + */ + private void leaveRepoLoop(String repository) { + final boolean removed = currentlyFinalizing.remove(repository); + assert removed; + } + + private void finalizeSnapshotEntry(SnapshotsInProgress.Entry entry, Metadata metadata, RepositoryData repositoryData) { + assert currentlyFinalizing.contains(entry.repository()); try { final String failure = entry.failure(); + final Snapshot snapshot = entry.snapshot(); logger.trace("[{}] finalizing snapshot in repository, state: [{}], failure[{}]", snapshot, entry.state(), failure); ArrayList shardFailures = new ArrayList<>(); for (ObjectObjectCursor shardStatus : entry.shards()) { @@ -713,6 +875,7 @@ private void endSnapshot(SnapshotsInProgress.Entry entry, Metadata metadata) { } } final ShardGenerations shardGenerations = buildGenerations(entry, metadata); + final String repository = snapshot.getRepository(); final SnapshotInfo snapshotInfo = new SnapshotInfo(snapshot.getSnapshotId(), shardGenerations.indices().stream().map(IndexId::getName).collect(Collectors.toList()), entry.dataStreams(), @@ -721,31 +884,36 @@ private void endSnapshot(SnapshotsInProgress.Entry entry, Metadata metadata) { entry.includeGlobalState(), entry.userMetadata()); repositoriesService.repository(snapshot.getRepository()).finalizeSnapshot( shardGenerations, - entry.repositoryStateId(), + repositoryData.getGenId(), metadataForSnapshot(entry, metadata), snapshotInfo, entry.version(), state -> stateWithoutSnapshot(state, snapshot), ActionListener.wrap(newRepoData -> { - final List>> completionListeners = - snapshotCompletionListeners.remove(snapshot); - if (completionListeners != null) { - final Tuple result = Tuple.tuple(newRepoData, snapshotInfo); - try { - ActionListener.onResponse(completionListeners, result); - } catch (Exception e) { - logger.warn("Failed to notify listeners", e); - } - } endingSnapshots.remove(snapshot); + completeListenersIgnoringException( + snapshotCompletionListeners.remove(snapshot), Tuple.tuple(newRepoData, snapshotInfo)); logger.info("snapshot [{}] completed with state [{}]", snapshot, snapshotInfo.state()); - }, e -> handleFinalizationFailure(e, entry))); + runNextQueuedOperation(newRepoData, repository, true); + }, e -> handleFinalizationFailure(e, entry, repositoryData))); } catch (Exception e) { - handleFinalizationFailure(e, entry); + assert false : new AssertionError(e); + handleFinalizationFailure(e, entry, repositoryData); } } - private void handleFinalizationFailure(Exception e, SnapshotsInProgress.Entry entry) { + /** + * Handles failure to finalize a snapshot. If the exception indicates that this node was unable to publish a cluster state and stopped + * being the master node, then fail all snapshot create and delete listeners executing on this node by delegating to + * {@link #failAllListenersOnMasterFailOver}. Otherwise, i.e. as a result of failing to write to the snapshot repository for some + * reason, remove the snapshot's {@link SnapshotsInProgress.Entry} from the cluster state and move on with other queued snapshot + * operations if there are any. + * + * @param e exception encountered + * @param entry snapshot entry that failed to finalize + * @param repositoryData current repository data for the snapshot's repository + */ + private void handleFinalizationFailure(Exception e, SnapshotsInProgress.Entry entry, RepositoryData repositoryData) { Snapshot snapshot = entry.snapshot(); if (ExceptionsHelper.unwrap(e, NotMasterException.class, FailedToCommitClusterStateException.class) != null) { // Failure due to not being master any more, don't try to remove snapshot from cluster state the next master @@ -753,15 +921,127 @@ private void handleFinalizationFailure(Exception e, SnapshotsInProgress.Entry en logger.debug(() -> new ParameterizedMessage( "[{}] failed to update cluster state during snapshot finalization", snapshot), e); failSnapshotCompletionListeners(snapshot, - new SnapshotException(snapshot, "Failed to update cluster state during snapshot finalization", e)); + new SnapshotException(snapshot, "Failed to update cluster state during snapshot finalization", e)); + failAllListenersOnMasterFailOver(e); } else { logger.warn(() -> new ParameterizedMessage("[{}] failed to finalize snapshot", snapshot), e); - removeSnapshotFromClusterState(snapshot, e); + removeFailedSnapshotFromClusterState(snapshot, e, repositoryData); + } + } + + /** + * Run the next queued up repository operation for the given repository name. + * + * @param repositoryData current repository data + * @param repository repository name + * @param attemptDelete whether to try and run delete operations that are ready in the cluster state if no + * snapshot create operations remain to execute + */ + private void runNextQueuedOperation(RepositoryData repositoryData, String repository, boolean attemptDelete) { + assert currentlyFinalizing.contains(repository); + final Tuple nextFinalization = repositoryOperations.pollFinalization(repository); + if (nextFinalization == null) { + if (attemptDelete) { + runReadyDeletions(repositoryData, repository); + } else { + leaveRepoLoop(repository); + } + } else { + logger.trace("Moving on to finalizing next snapshot [{}]", nextFinalization); + finalizeSnapshotEntry(nextFinalization.v1(), nextFinalization.v2(), repositoryData); + } + } + + /** + * Runs a cluster state update that checks whether we have outstanding snapshot deletions that can be executed and executes them. + * + * TODO: optimize this to execute in a single CS update together with finalizing the latest snapshot + */ + private void runReadyDeletions(RepositoryData repositoryData, String repository) { + clusterService.submitStateUpdateTask("Run ready deletions", new ClusterStateUpdateTask() { + + private SnapshotDeletionsInProgress.Entry deletionToRun; + + @Override + public ClusterState execute(ClusterState currentState) { + assert readyDeletions(currentState).v1() == currentState : + "Deletes should have been set to ready by finished snapshot deletes and finalizations"; + for (SnapshotDeletionsInProgress.Entry entry : + currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).getEntries()) { + if (entry.repository().equals(repository) && entry.state() == SnapshotDeletionsInProgress.State.STARTED) { + deletionToRun = entry; + break; + } + } + return currentState; + } + + @Override + public void onFailure(String source, Exception e) { + logger.warn("Failed to run ready delete operations", e); + failAllListenersOnMasterFailOver(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + if (deletionToRun == null) { + runNextQueuedOperation(repositoryData, repository, false); + } else { + deleteSnapshotsFromRepository(deletionToRun, repositoryData, newState.nodes().getMinNodeVersion()); + } + } + }); + } + + /** + * Finds snapshot delete operations that are ready to execute in the given {@link ClusterState} and computes a new cluster state that + * has all executable deletes marked as executing. Returns a {@link Tuple} of the updated cluster state and all executable deletes. + * This can either be {@link SnapshotDeletionsInProgress.Entry} that were already in state + * {@link SnapshotDeletionsInProgress.State#STARTED} or waiting entries in state {@link SnapshotDeletionsInProgress.State#WAITING} + * that were moved to {@link SnapshotDeletionsInProgress.State#STARTED} in the returned updated cluster state. + * + * @param currentState current cluster state + * @return tuple of an updated cluster state and currently executable snapshot delete operations + */ + private static Tuple> readyDeletions(ClusterState currentState) { + final SnapshotDeletionsInProgress deletions = + currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + if (deletions.hasDeletionsInProgress() == false) { + return Tuple.tuple(currentState, List.of()); } + final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE); + assert snapshotsInProgress != null; + final Set repositoriesSeen = new HashSet<>(); + boolean changed = false; + final ArrayList readyDeletions = new ArrayList<>(); + final List newDeletes = new ArrayList<>(); + for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) { + final String repo = entry.repository(); + if (repositoriesSeen.add(entry.repository()) && entry.state() == SnapshotDeletionsInProgress.State.WAITING + && snapshotsInProgress.entries().stream() + .filter(se -> se.repository().equals(repo)).noneMatch(SnapshotsService::isWritingToRepository)) { + changed = true; + final SnapshotDeletionsInProgress.Entry newEntry = entry.started(); + readyDeletions.add(newEntry); + newDeletes.add(newEntry); + } else { + newDeletes.add(entry); + } + } + return Tuple.tuple(changed ? ClusterState.builder(currentState).putCustom( + SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.of(newDeletes)).build() : currentState, readyDeletions); } + /** + * Computes the cluster state resulting from removing a given snapshot create operation from the given state. + * + * @param state current cluster state + * @param snapshot snapshot for which to remove the snapshot operation + * @return updated cluster state + */ private static ClusterState stateWithoutSnapshot(ClusterState state, Snapshot snapshot) { SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + ClusterState result = state; boolean changed = false; ArrayList entries = new ArrayList<>(); for (SnapshotsInProgress.Entry entry : snapshots.entries()) { @@ -772,17 +1052,21 @@ private static ClusterState stateWithoutSnapshot(ClusterState state, Snapshot sn } } if (changed) { - return ClusterState.builder(state).putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(entries)).build(); + result = ClusterState.builder(state).putCustom( + SnapshotsInProgress.TYPE, SnapshotsInProgress.of(unmodifiableList(entries))).build(); } - return state; + return readyDeletions(result).v1(); } /** - * Removes record of running snapshot from cluster state and notifies the listener when this action is complete - * @param snapshot snapshot - * @param failure exception if snapshot failed + * Removes record of running snapshot from cluster state and notifies the listener when this action is complete. This method is only + * used when the snapshot fails for some reason. During normal operation the snapshot repository will remove the + * {@link SnapshotsInProgress.Entry} from the cluster state once it's done finalizing the snapshot. + * + * @param snapshot snapshot that failed + * @param failure exception that failed the snapshot */ - private void removeSnapshotFromClusterState(final Snapshot snapshot, Exception failure) { + private void removeFailedSnapshotFromClusterState(Snapshot snapshot, Exception failure, RepositoryData repositoryData) { assert failure != null : "Failure must be supplied"; clusterService.submitStateUpdateTask("remove snapshot metadata", new ClusterStateUpdateTask() { @@ -795,32 +1079,29 @@ public ClusterState execute(ClusterState currentState) { public void onFailure(String source, Exception e) { logger.warn(() -> new ParameterizedMessage("[{}] failed to remove snapshot metadata", snapshot), e); failSnapshotCompletionListeners( - snapshot, new SnapshotException(snapshot, "Failed to remove snapshot from cluster state", e)); + snapshot, new SnapshotException(snapshot, "Failed to remove snapshot from cluster state", e)); + failAllListenersOnMasterFailOver(e); } @Override public void onNoLongerMaster(String source) { - failSnapshotCompletionListeners( - snapshot, ExceptionsHelper.useOrSuppress(failure, new SnapshotException(snapshot, "no longer master"))); + failure.addSuppressed(new SnapshotException(snapshot, "no longer master")); + failSnapshotCompletionListeners(snapshot, failure); + failAllListenersOnMasterFailOver(new NotMasterException(source)); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { failSnapshotCompletionListeners(snapshot, failure); + runNextQueuedOperation(repositoryData, snapshot.getRepository(), true); } }); } private void failSnapshotCompletionListeners(Snapshot snapshot, Exception e) { - final List>> completionListeners = snapshotCompletionListeners.remove(snapshot); - if (completionListeners != null) { - try { - ActionListener.onFailure(completionListeners, e); - } catch (Exception ex) { - logger.warn("Failed to notify listeners", ex); - } - } endingSnapshots.remove(snapshot); + failListenersIgnoringException(snapshotCompletionListeners.remove(snapshot), e); + assert repositoryOperations.assertNotQueued(snapshot); } /** @@ -849,17 +1130,18 @@ public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionLis @Override public ClusterState execute(ClusterState currentState) throws Exception { + final Version minNodeVersion = currentState.nodes().getMinNodeVersion(); final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - final SnapshotsInProgress.Entry snapshotEntry = findInProgressSnapshot(snapshots, snapshotNames, repositoryName); + final List snapshotEntries = findInProgressSnapshots(snapshots, snapshotNames, repositoryName); final List snapshotIds = matchingSnapshotIds( - snapshotEntry == null ? null : snapshotEntry.snapshot().getSnapshotId(), - repositoryData, snapshotNames, repositoryName); - if (snapshotEntry == null) { - deleteFromRepoTask = - createDeleteStateUpdate(snapshotIds, repositoryName, repositoryData.getGenId(), Priority.NORMAL, listener); + snapshotEntries.stream().map(e -> e.snapshot().getSnapshotId()).collect(Collectors.toList()), repositoryData, + snapshotNames, repositoryName); + if (snapshotEntries.isEmpty() || minNodeVersion.onOrAfter(SnapshotsService.FULL_CONCURRENCY_VERSION)) { + deleteFromRepoTask = createDeleteStateUpdate(snapshotIds, repositoryName, repositoryData, Priority.NORMAL, listener); return deleteFromRepoTask.execute(currentState); } - + assert snapshotEntries.size() == 1 : "Expected just a single running snapshot but saw " + snapshotEntries; + final SnapshotsInProgress.Entry snapshotEntry = snapshotEntries.get(0); runningSnapshot = snapshotEntry.snapshot(); final ImmutableOpenMap shards; @@ -879,16 +1161,7 @@ public ClusterState execute(ClusterState currentState) throws Exception { abortedDuringInit = true; } else if (state == State.STARTED) { // snapshot is started - mark every non completed shard as aborted - final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); - for (ObjectObjectCursor shardEntry : snapshotEntry.shards()) { - ShardSnapshotStatus status = shardEntry.value; - if (status.state().completed() == false) { - status = new ShardSnapshotStatus( - status.nodeId(), ShardState.ABORTED, "aborted by snapshot deletion", status.generation()); - } - shardsBuilder.put(shardEntry.key, status); - } - shards = shardsBuilder.build(); + shards = abortEntry(snapshotEntry); failure = "Snapshot was aborted by deletion"; } else { boolean hasUncompletedShards = false; @@ -915,14 +1188,14 @@ public ClusterState execute(ClusterState currentState) throws Exception { } return ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, SnapshotsInProgress.of(snapshots.entries().stream() - // remove init state snapshot we found from a previous master if there was one - .filter(existing -> abortedDuringInit == false || existing.equals(snapshotEntry) == false) - .map(existing -> { - if (existing.equals(snapshotEntry)) { - return new SnapshotsInProgress.Entry(snapshotEntry, State.ABORTED, shards, failure); - } - return existing; - }).collect(Collectors.toUnmodifiableList()))).build(); + // remove init state snapshot we found from a previous master if there was one + .filter(existing -> abortedDuringInit == false || existing.equals(snapshotEntry) == false) + .map(existing -> { + if (existing.equals(snapshotEntry)) { + return new SnapshotsInProgress.Entry(snapshotEntry, State.ABORTED, shards, failure); + } + return existing; + }).collect(Collectors.toUnmodifiableList()))).build(); } @Override @@ -944,8 +1217,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS listener.onResponse(null); } else { clusterService.submitStateUpdateTask("delete snapshot", - createDeleteStateUpdate(outstandingDeletes, repositoryName, repositoryData.getGenId(), - Priority.IMMEDIATE, listener)); + createDeleteStateUpdate(outstandingDeletes, repositoryName, repositoryData, Priority.IMMEDIATE, listener)); } return; } @@ -954,8 +1226,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS result -> { logger.debug("deleted snapshot completed - deleting files"); clusterService.submitStateUpdateTask("delete snapshot", - createDeleteStateUpdate(outstandingDeletes, repositoryName, - result.v1().getGenId(), Priority.IMMEDIATE, listener)); + createDeleteStateUpdate(outstandingDeletes, repositoryName, result.v1(), Priority.IMMEDIATE, listener)); }, e -> { if (ExceptionsHelper.unwrap(e, NotMasterException.class, FailedToCommitClusterStateException.class) != null) { @@ -978,11 +1249,11 @@ public TimeValue timeout() { }, "delete snapshot", listener::onFailure); } - private static List matchingSnapshotIds(@Nullable SnapshotId inProgress, RepositoryData repositoryData, + private static List matchingSnapshotIds(List inProgress, RepositoryData repositoryData, String[] snapshotsOrPatterns, String repositoryName) { final Map allSnapshotIds = repositoryData.getSnapshotIds().stream().collect( Collectors.toMap(SnapshotId::getName, Function.identity())); - final Set foundSnapshots = new HashSet<>(); + final Set foundSnapshots = new HashSet<>(inProgress); for (String snapshotOrPattern : snapshotsOrPatterns) { if (Regex.isSimpleMatchPattern(snapshotOrPattern)) { for (Map.Entry entry : allSnapshotIds.entrySet()) { @@ -993,7 +1264,7 @@ private static List matchingSnapshotIds(@Nullable SnapshotId inProgr } else { final SnapshotId foundId = allSnapshotIds.get(snapshotOrPattern); if (foundId == null) { - if (inProgress == null || inProgress.getName().equals(snapshotOrPattern) == false) { + if (inProgress.stream().noneMatch(snapshotId -> snapshotId.getName().equals(snapshotOrPattern))) { throw new SnapshotMissingException(repositoryName, snapshotOrPattern); } } else { @@ -1004,22 +1275,20 @@ private static List matchingSnapshotIds(@Nullable SnapshotId inProgr return List.copyOf(foundSnapshots); } - // Return in-progress snapshot entry by name and repository in the given cluster state or null if none is found - @Nullable - private static SnapshotsInProgress.Entry findInProgressSnapshot(SnapshotsInProgress snapshots, String[] snapshotNames, - String repositoryName) { - SnapshotsInProgress.Entry snapshotEntry = null; + // Return in-progress snapshot entries by name and repository in the given cluster state or null if none is found + private static List findInProgressSnapshots(SnapshotsInProgress snapshots, String[] snapshotNames, + String repositoryName) { + List entries = new ArrayList<>(); for (SnapshotsInProgress.Entry entry : snapshots.entries()) { if (entry.repository().equals(repositoryName) - && Regex.simpleMatch(snapshotNames, entry.snapshot().getSnapshotId().getName())) { - snapshotEntry = entry; - break; + && Regex.simpleMatch(snapshotNames, entry.snapshot().getSnapshotId().getName())) { + entries.add(entry); } } - return snapshotEntry; + return entries; } - private ClusterStateUpdateTask createDeleteStateUpdate(List snapshotIds, String repoName, long repositoryStateId, + private ClusterStateUpdateTask createDeleteStateUpdate(List snapshotIds, String repoName, RepositoryData repositoryData, Priority priority, ActionListener listener) { // Short circuit to noop state update if there isn't anything to delete if (snapshotIds.isEmpty()) { @@ -1041,45 +1310,99 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS }; } return new ClusterStateUpdateTask(priority) { + + private SnapshotDeletionsInProgress.Entry newDelete; + + private boolean reusedExistingDelete = false; + + private final Collection completedSnapshots = new ArrayList<>(); + @Override public ClusterState execute(ClusterState currentState) { final SnapshotDeletionsInProgress deletionsInProgress = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - if (deletionsInProgress.hasDeletionsInProgress()) { - throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), - "cannot delete - another snapshot is currently being deleted in [" + deletionsInProgress + "]"); + currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + final Version minNodeVersion = currentState.nodes().getMinNodeVersion(); + if (minNodeVersion.before(FULL_CONCURRENCY_VERSION)) { + if (deletionsInProgress.hasDeletionsInProgress()) { + throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), + "cannot delete - another snapshot is currently being deleted in [" + deletionsInProgress + "]"); + } } final RepositoryCleanupInProgress repositoryCleanupInProgress = - currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); + currentState.custom(RepositoryCleanupInProgress.TYPE, RepositoryCleanupInProgress.EMPTY); if (repositoryCleanupInProgress.hasCleanupInProgress()) { throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), - "cannot delete snapshots while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]"); + "cannot delete snapshots while a repository cleanup is in-progress in [" + repositoryCleanupInProgress + "]"); } 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 // and the files the restore depends on would all be gone + for (RestoreInProgress.Entry entry : restoreInProgress) { if (repoName.equals(entry.snapshot().getRepository()) && snapshotIds.contains(entry.snapshot().getSnapshotId())) { throw new ConcurrentSnapshotExecutionException(new Snapshot(repoName, snapshotIds.get(0)), - "cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]"); + "cannot delete snapshot during a restore in progress in [" + restoreInProgress + "]"); } } - SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); - if (snapshots.entries().isEmpty() == false) { - // However other snapshots are running - cannot continue - throw new ConcurrentSnapshotExecutionException( - repoName, snapshotIds.toString(), "another snapshot is currently running cannot delete"); + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final SnapshotsInProgress updatedSnapshots; + if (minNodeVersion.onOrAfter(FULL_CONCURRENCY_VERSION)) { + updatedSnapshots = SnapshotsInProgress.of(snapshots.entries().stream() + .map(existing -> { + // snapshot is started - mark every non completed shard as aborted + if (existing.state() == State.STARTED && snapshotIds.contains(existing.snapshot().getSnapshotId())) { + final ImmutableOpenMap abortedShards = abortEntry(existing); + final boolean isCompleted = completed(abortedShards.values()); + final SnapshotsInProgress.Entry abortedEntry = new SnapshotsInProgress.Entry( + existing, isCompleted ? State.SUCCESS : State.ABORTED, abortedShards, + "Snapshot was aborted by deletion"); + if (isCompleted) { + completedSnapshots.add(abortedEntry); + } + return abortedEntry; + } + return existing; + }).collect(Collectors.toUnmodifiableList())); + } else { + if (snapshots.entries().isEmpty() == false) { + // However other snapshots are running - cannot continue + throw new ConcurrentSnapshotExecutionException( + repoName, snapshotIds.toString(), "another snapshot is currently running cannot delete"); + } + updatedSnapshots = snapshots; } // add the snapshot deletion to the cluster state - SnapshotDeletionsInProgress.Entry entry = new SnapshotDeletionsInProgress.Entry( + final SnapshotDeletionsInProgress.Entry replacedEntry = deletionsInProgress.getEntries().stream().filter(entry -> + entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.WAITING) + .findFirst().orElse(null); + if (replacedEntry == null) { + final Optional foundDuplicate = + deletionsInProgress.getEntries().stream().filter(entry -> + entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.STARTED + && entry.getSnapshots().containsAll(snapshotIds)).findFirst(); + if (foundDuplicate.isPresent()) { + newDelete = foundDuplicate.get(); + reusedExistingDelete = true; + return currentState; + } + ensureBelowConcurrencyLimit(repoName, snapshotIds.get(0).getName(), snapshots, deletionsInProgress); + newDelete = new SnapshotDeletionsInProgress.Entry( snapshotIds, repoName, threadPool.absoluteTimeInMillis(), - repositoryStateId - ); - return ClusterState.builder(currentState).putCustom( - SnapshotDeletionsInProgress.TYPE, deletionsInProgress.withAddedEntry(entry)).build(); + repositoryData.getGenId(), + updatedSnapshots.entries().stream().filter(entry -> repoName.equals(entry.repository())).noneMatch( + SnapshotsService::isWritingToRepository) + && deletionsInProgress.getEntries().stream().noneMatch(entry -> + repoName.equals(entry.repository()) && entry.state() == SnapshotDeletionsInProgress.State.STARTED) + ? SnapshotDeletionsInProgress.State.STARTED : SnapshotDeletionsInProgress.State.WAITING); + } else { + newDelete = replacedEntry.withAddedSnapshots(snapshotIds); + } + return updateWithSnapshots(currentState, updatedSnapshots, + (replacedEntry == null ? deletionsInProgress : deletionsInProgress.withRemovedEntry(replacedEntry.uuid())) + .withAddedEntry(newDelete)); } @Override @@ -1089,11 +1412,64 @@ public void onFailure(String source, Exception e) { @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - deleteSnapshotsFromRepository(repoName, snapshotIds, listener, repositoryStateId, newState.nodes().getMinNodeVersion()); + addDeleteListener(newDelete.uuid(), listener); + if (reusedExistingDelete) { + return; + } + if (newDelete.state() == SnapshotDeletionsInProgress.State.STARTED) { + if (tryEnterRepoLoop(repoName)) { + deleteSnapshotsFromRepository(newDelete, repositoryData, newState.nodes().getMinNodeVersion()); + } else { + logger.trace("Delete [{}] could not execute directly and was queued", newDelete); + } + } else { + for (SnapshotsInProgress.Entry completedSnapshot : completedSnapshots) { + endSnapshot(completedSnapshot, newState.metadata(), repositoryData); + } + } } }; } + /** + * Checks if the given {@link SnapshotsInProgress.Entry} is currently writing to the repository. + * + * @param entry snapshot entry + * @return true if entry is currently writing to the repository + */ + private static boolean isWritingToRepository(SnapshotsInProgress.Entry entry) { + if (entry.state().completed()) { + // Entry is writing to the repo because it's finalizing on master + return true; + } + for (ObjectCursor value : entry.shards().values()) { + if (value.value.isActive()) { + // Entry is writing to the repo because it's writing to a shard on a data node or waiting to do so for a concrete shard + return true; + } + } + return false; + } + + private ImmutableOpenMap abortEntry(SnapshotsInProgress.Entry existing) { + final ImmutableOpenMap.Builder shardsBuilder = + ImmutableOpenMap.builder(); + for (ObjectObjectCursor shardEntry : existing.shards()) { + ShardSnapshotStatus status = shardEntry.value; + if (status.state().completed() == false) { + final String nodeId = status.nodeId(); + status = new ShardSnapshotStatus(nodeId, nodeId == null ? ShardState.FAILED : ShardState.ABORTED, + "aborted by snapshot deletion", status.generation()); + } + shardsBuilder.put(shardEntry.key, status); + } + return shardsBuilder.build(); + } + + private void addDeleteListener(String deleteUUID, ActionListener listener) { + snapshotDeletionListeners.computeIfAbsent(deleteUUID, k -> new CopyOnWriteArrayList<>()).add(listener); + } + /** * Determines the minimum {@link Version} that the snapshot repository must be compatible with from the current nodes in the cluster * and the contents of the repository. The minimum version is determined as the lowest version found across all snapshots in the @@ -1146,82 +1522,361 @@ public static boolean useIndexGenerations(Version repositoryMetaVersion) { /** Deletes snapshot from repository * - * @param repoName repository name - * @param snapshotIds snapshot ids - * @param listener listener - * @param repositoryStateId the unique id representing the state of the repository at the time the deletion began + * @param deleteEntry delete entry in cluster state + * @param minNodeVersion minimum node version in the cluster + */ + private void deleteSnapshotsFromRepository(SnapshotDeletionsInProgress.Entry deleteEntry, Version minNodeVersion) { + final long expectedRepoGen = deleteEntry.repositoryStateId(); + repositoriesService.getRepositoryData(deleteEntry.repository(), new ActionListener<>() { + @Override + public void onResponse(RepositoryData repositoryData) { + assert repositoryData.getGenId() == expectedRepoGen : + "Repository generation should not change as long as a ready delete is found in the cluster state but found [" + + expectedRepoGen + "] in cluster state and [" + repositoryData.getGenId() + "] in the repository"; + deleteSnapshotsFromRepository(deleteEntry, repositoryData, minNodeVersion); + } + + @Override + public void onFailure(Exception e) { + clusterService.submitStateUpdateTask("fail repo tasks for [" + deleteEntry.repository() + "]", + new FailPendingRepoTasksTask(deleteEntry.repository(), e)); + } + }); + } + + /** Deletes snapshot from repository + * + * @param deleteEntry delete entry in cluster state + * @param repositoryData the {@link RepositoryData} of the repository to delete from * @param minNodeVersion minimum node version in the cluster */ - private void deleteSnapshotsFromRepository(String repoName, Collection snapshotIds, @Nullable ActionListener listener, - long repositoryStateId, Version minNodeVersion) { - Repository repository = repositoriesService.repository(repoName); - repository.getRepositoryData(ActionListener.wrap(repositoryData -> repository.deleteSnapshots( + private void deleteSnapshotsFromRepository(SnapshotDeletionsInProgress.Entry deleteEntry, + RepositoryData repositoryData, Version minNodeVersion) { + if (repositoryOperations.startDeletion(deleteEntry.uuid())) { + assert currentlyFinalizing.contains(deleteEntry.repository()); + final List snapshotIds = deleteEntry.getSnapshots(); + assert deleteEntry.state() == SnapshotDeletionsInProgress.State.STARTED : + "incorrect state for entry [" + deleteEntry + "]"; + repositoriesService.repository(deleteEntry.repository()).deleteSnapshots( snapshotIds, - repositoryStateId, + repositoryData.getGenId(), minCompatibleVersion(minNodeVersion, repositoryData, snapshotIds), - ActionListener.wrap(v -> { - logger.info("snapshots {} deleted", snapshotIds); - removeSnapshotDeletionFromClusterState(snapshotIds, null, listener); - }, ex -> removeSnapshotDeletionFromClusterState(snapshotIds, ex, listener) - )), ex -> removeSnapshotDeletionFromClusterState(snapshotIds, ex, listener))); + ActionListener.wrap(updatedRepoData -> { + logger.info("snapshots {} deleted", snapshotIds); + removeSnapshotDeletionFromClusterState(deleteEntry, null, updatedRepoData); + }, ex -> removeSnapshotDeletionFromClusterState(deleteEntry, ex, repositoryData) + )); + } } /** - * Removes the snapshot deletion from {@link SnapshotDeletionsInProgress} in the cluster state. + * Removes a {@link SnapshotDeletionsInProgress.Entry} from {@link SnapshotDeletionsInProgress} in the cluster state after it executed + * on the repository. + * + * @param deleteEntry delete entry to remove from the cluster state + * @param failure failure encountered while executing the delete on the repository or {@code null} if the delete executed + * successfully + * @param repositoryData current {@link RepositoryData} for the repository we just ran the delete on. */ - private void removeSnapshotDeletionFromClusterState(final Collection snapshotIds, @Nullable final Exception failure, - @Nullable final ActionListener listener) { - clusterService.submitStateUpdateTask("remove snapshot deletion metadata", new ClusterStateUpdateTask() { - @Override - public ClusterState execute(ClusterState currentState) { - final SnapshotDeletionsInProgress deletions = - currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); - if (deletions.hasDeletionsInProgress()) { - assert deletions.getEntries().size() == 1 : "should have exactly one deletion in progress"; - SnapshotDeletionsInProgress.Entry entry = deletions.getEntries().get(0); - return ClusterState.builder(currentState).putCustom( - SnapshotDeletionsInProgress.TYPE, deletions.withRemovedEntry(entry)).build(); + private void removeSnapshotDeletionFromClusterState(final SnapshotDeletionsInProgress.Entry deleteEntry, + @Nullable final Exception failure, final RepositoryData repositoryData) { + final ClusterStateUpdateTask clusterStateUpdateTask; + if (failure == null) { + // If we didn't have a failure during the snapshot delete we will remove all snapshot ids that the delete successfully removed + // from the repository from enqueued snapshot delete entries during the cluster state update. After the cluster state update we + // resolve the delete listeners with the latest repository data from after the delete. + clusterStateUpdateTask = new RemoveSnapshotDeletionAndContinueTask(deleteEntry, repositoryData) { + @Override + protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgress deletions) { + boolean changed = false; + List updatedEntries = new ArrayList<>(deletions.getEntries().size()); + for (SnapshotDeletionsInProgress.Entry entry : deletions.getEntries()) { + if (entry.repository().equals(deleteEntry.repository())) { + final List updatedSnapshotIds = new ArrayList<>(entry.getSnapshots()); + if (updatedSnapshotIds.removeAll(deleteEntry.getSnapshots())) { + changed = true; + updatedEntries.add(entry.withSnapshots(updatedSnapshotIds)); + } else { + updatedEntries.add(entry); + } + } else { + updatedEntries.add(entry); + } + } + return changed ? SnapshotDeletionsInProgress.of(updatedEntries) : deletions; + } + + @Override + protected void handleListeners(List> deleteListeners) { + assert repositoryData.getSnapshotIds().stream().noneMatch(deleteEntry.getSnapshots()::contains) + : "Repository data contained snapshot ids " + repositoryData.getSnapshotIds() + + " that should should been deleted by [" + deleteEntry + "]"; + completeListenersIgnoringException(deleteListeners, null); + } + }; + } else { + // The delete failed to execute on the repository. We remove it from the cluster state and then fail all listeners associated + // with it. + clusterStateUpdateTask = new RemoveSnapshotDeletionAndContinueTask(deleteEntry, repositoryData) { + @Override + protected void handleListeners(List> deleteListeners) { + failListenersIgnoringException(deleteListeners, failure); + } + }; + } + clusterService.submitStateUpdateTask("remove snapshot deletion metadata", clusterStateUpdateTask); + } + + /** + * Handle snapshot or delete failure due to not being master any more so we don't try to do run additional cluster state updates. + * The next master will try handling the missing operations. All we can do is fail all the listeners on this master node so that + * transport requests return and we don't leak listeners. + * + * @param e exception that caused us to realize we are not master any longer + */ + private void failAllListenersOnMasterFailOver(Exception e) { + logger.debug("Failing all snapshot operation listeners because this node is not master any longer", e); + synchronized (currentlyFinalizing) { + if (ExceptionsHelper.unwrap(e, NotMasterException.class, FailedToCommitClusterStateException.class) != null) { + repositoryOperations.clear(); + for (Snapshot snapshot : Set.copyOf(snapshotCompletionListeners.keySet())) { + failSnapshotCompletionListeners(snapshot, new SnapshotException(snapshot, "no longer master")); } + final Exception wrapped = + new RepositoryException("_all", "Failed to update cluster state during repository operation", e); + for (Iterator>> iterator = snapshotDeletionListeners.values().iterator(); + iterator.hasNext(); ) { + final List> listeners = iterator.next(); + iterator.remove(); + failListenersIgnoringException(listeners, wrapped); + } + assert snapshotDeletionListeners.isEmpty() : + "No new listeners should have been added but saw " + snapshotDeletionListeners; + } else { + assert false : + new AssertionError("Modifying snapshot state should only ever fail because we failed to publish new state", e); + logger.error("Unexpected failure during cluster state update", e); + } + currentlyFinalizing.clear(); + } + } + + /** + * A cluster state update that will remove a given {@link SnapshotDeletionsInProgress.Entry} from the cluster state + * and trigger running the next snapshot-delete or -finalization operation available to execute if there is one + * ready in the cluster state as a result of this state update. + */ + private abstract class RemoveSnapshotDeletionAndContinueTask extends ClusterStateUpdateTask { + + // Snapshots that can be finalized after the delete operation has been removed from the cluster state + protected final List newFinalizations = new ArrayList<>(); + + private List readyDeletions = Collections.emptyList(); + + protected final SnapshotDeletionsInProgress.Entry deleteEntry; + + private final RepositoryData repositoryData; + + RemoveSnapshotDeletionAndContinueTask(SnapshotDeletionsInProgress.Entry deleteEntry, RepositoryData repositoryData) { + this.deleteEntry = deleteEntry; + this.repositoryData = repositoryData; + } + + @Override + public ClusterState execute(ClusterState currentState) { + final SnapshotDeletionsInProgress deletions = currentState.custom(SnapshotDeletionsInProgress.TYPE); + assert deletions != null : "We only run this if there were deletions in the cluster state before"; + final SnapshotDeletionsInProgress updatedDeletions = deletions.withRemovedEntry(deleteEntry.uuid()); + if (updatedDeletions == deletions) { return currentState; } + final SnapshotDeletionsInProgress newDeletions = filterDeletions(updatedDeletions); + final Tuple> res = readyDeletions( + updateWithSnapshots(currentState, updatedSnapshotsInProgress(currentState, newDeletions), newDeletions)); + readyDeletions = res.v2(); + return res.v1(); + } - @Override - public void onFailure(String source, Exception e) { - logger.warn(() -> new ParameterizedMessage("{} failed to remove snapshot deletion metadata", snapshotIds), e); - if (listener != null) { - listener.onFailure(e); + @Override + public void onFailure(String source, Exception e) { + logger.warn(() -> new ParameterizedMessage("{} failed to remove snapshot deletion metadata", deleteEntry), e); + repositoryOperations.finishDeletion(deleteEntry.uuid()); + failAllListenersOnMasterFailOver(e); + } + + protected SnapshotDeletionsInProgress filterDeletions(SnapshotDeletionsInProgress deletions) { + return deletions; + } + + @Override + public final void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + final List> deleteListeners; + repositoryOperations.finishDeletion(deleteEntry.uuid()); + deleteListeners = snapshotDeletionListeners.remove(deleteEntry.uuid()); + handleListeners(deleteListeners); + if (newFinalizations.isEmpty()) { + if (readyDeletions.isEmpty()) { + leaveRepoLoop(deleteEntry.repository()); + } else { + for (SnapshotDeletionsInProgress.Entry readyDeletion : readyDeletions) { + deleteSnapshotsFromRepository(readyDeletion, repositoryData, newState.nodes().getMinNodeVersion()); + } + } + } else { + leaveRepoLoop(deleteEntry.repository()); + assert readyDeletions.stream().noneMatch(entry -> entry.repository().equals(deleteEntry.repository())) + : "New finalizations " + newFinalizations + " added even though deletes " + readyDeletions + " are ready"; + for (SnapshotsInProgress.Entry entry : newFinalizations) { + endSnapshot(entry, newState.metadata(), repositoryData); } } + } - @Override - public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - if (listener != null) { - if (failure != null) { - listener.onFailure(failure); + /** + * Invoke snapshot delete listeners for {@link #deleteEntry}. + * + * @param deleteListeners delete snapshot listeners or {@code null} if there weren't any for {@link #deleteEntry}. + */ + protected abstract void handleListeners(@Nullable List> deleteListeners); + + /** + * Computes an updated {@link SnapshotsInProgress} that takes into account an updated version of + * {@link SnapshotDeletionsInProgress} that has a {@link SnapshotDeletionsInProgress.Entry} removed from it + * relative to the {@link SnapshotDeletionsInProgress} found in {@code currentState}. + * The removal of a delete from the cluster state can trigger two possible actions on in-progress snapshots: + *

    + *
  • Snapshots that had unfinished shard snapshots in state {@link ShardSnapshotStatus#UNASSIGNED_QUEUED} that + * could not be started because the delete was running can have those started.
  • + *
  • Snapshots that had all their shards reach a completed state while a delete was running (e.g. as a result of + * nodes dropping out of the cluster or another incoming delete aborting them) need not be updated in the cluster + * state but need to have their finalization triggered now that it's possible with the removal of the delete + * from the state.
  • + *
+ * + * @param currentState current cluster state + * @param updatedDeletions deletions with removed entry + * @return updated snapshot in progress instance or {@code null} if there are no changes to it + */ + @Nullable + private SnapshotsInProgress updatedSnapshotsInProgress(ClusterState currentState, + SnapshotDeletionsInProgress updatedDeletions) { + final SnapshotsInProgress snapshotsInProgress = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List snapshotEntries = new ArrayList<>(); + + // Keep track of shardIds that we started snapshots for as a result of removing this delete so we don't assign + // them to multiple snapshots by accident + final Set reassignedShardIds = new HashSet<>(); + + boolean changed = false; + + final String repoName = deleteEntry.repository(); + for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { + if (entry.repository().equals(repoName)) { + if (entry.state().completed() == false) { + // Collect waiting shards that in entry that we can assign now that we are done with the deletion + final List canBeUpdated = new ArrayList<>(); + for (ObjectObjectCursor value : entry.shards()) { + if (value.value.equals(ShardSnapshotStatus.UNASSIGNED_QUEUED) + && reassignedShardIds.contains(value.key) == false) { + canBeUpdated.add(value.key); + } + } + if (canBeUpdated.isEmpty()) { + // No shards can be updated in this snapshot so we just add it as is again + snapshotEntries.add(entry); + } else { + final ImmutableOpenMap shardAssignments = shards(snapshotsInProgress, + updatedDeletions, currentState.metadata(), currentState.routingTable(), entry.indices(), + entry.version().onOrAfter(SHARD_GEN_IN_REPO_DATA_VERSION), repositoryData, repoName); + final ImmutableOpenMap.Builder updatedAssignmentsBuilder = + ImmutableOpenMap.builder(entry.shards()); + for (ShardId shardId : canBeUpdated) { + final boolean added = reassignedShardIds.add(shardId); + assert added; + updatedAssignmentsBuilder.put(shardId, shardAssignments.get(shardId)); + } + snapshotEntries.add(entry.withShards(updatedAssignmentsBuilder.build())); + changed = true; + } } else { - logger.info("Successfully deleted snapshots {}", snapshotIds); - listener.onResponse(null); + // Entry is already completed so we will finalize it now that the delete doesn't block us after + // this CS update finishes + newFinalizations.add(entry); + snapshotEntries.add(entry); } + } else { + // Entry is for another repository we just keep it as is + snapshotEntries.add(entry); } } - }); + return changed ? SnapshotsInProgress.of(snapshotEntries) : null; + } + } + + /** + * Shortcut to build new {@link ClusterState} from the current state and updated values of {@link SnapshotsInProgress} and + * {@link SnapshotDeletionsInProgress}. + * + * @param state current cluster state + * @param snapshotsInProgress new value for {@link SnapshotsInProgress} or {@code null} if it's unchanged + * @param snapshotDeletionsInProgress new value for {@link SnapshotDeletionsInProgress} or {@code null} if it's unchanged + * @return updated cluster state + */ + public static ClusterState updateWithSnapshots(ClusterState state, + @Nullable SnapshotsInProgress snapshotsInProgress, + @Nullable SnapshotDeletionsInProgress snapshotDeletionsInProgress) { + if (snapshotsInProgress == null && snapshotDeletionsInProgress == null) { + return state; + } + ClusterState.Builder builder = ClusterState.builder(state); + if (snapshotsInProgress != null) { + builder.putCustom(SnapshotsInProgress.TYPE, snapshotsInProgress); + } + if (snapshotDeletionsInProgress != null) { + builder.putCustom(SnapshotDeletionsInProgress.TYPE, snapshotDeletionsInProgress); + } + return builder.build(); + } + + private static void failListenersIgnoringException(@Nullable List> listeners, Exception failure) { + if (listeners != null) { + try { + ActionListener.onFailure(listeners, failure); + } catch (Exception ex) { + assert false : new AssertionError(ex); + logger.warn("Failed to notify listeners", ex); + } + } + } + + private static void completeListenersIgnoringException(@Nullable List> listeners, T result) { + if (listeners != null) { + try { + ActionListener.onResponse(listeners, result); + } catch (Exception ex) { + assert false : new AssertionError(ex); + logger.warn("Failed to notify listeners", ex); + } + } } /** - * Calculates the list of shards that should be included into the current snapshot + * Calculates the assignment of shards to data nodes for a new snapshot based on the given cluster state and the + * indices that should be included in the snapshot. * - * @param clusterState cluster state * @param indices Indices to snapshot * @param useShardGenerations whether to write {@link ShardGenerations} during the snapshot * @return list of shard to be included into current snapshot */ - private static ImmutableOpenMap shards(ClusterState clusterState, - List indices, - boolean useShardGenerations, - RepositoryData repositoryData) { + private static ImmutableOpenMap shards( + @Nullable SnapshotsInProgress snapshotsInProgress, @Nullable SnapshotDeletionsInProgress deletionsInProgress, + Metadata metadata, RoutingTable routingTable, List indices, boolean useShardGenerations, + RepositoryData repositoryData, String repoName) { ImmutableOpenMap.Builder builder = ImmutableOpenMap.builder(); - Metadata metadata = clusterState.metadata(); final ShardGenerations shardGenerations = repositoryData.shardGenerations(); + final Set inProgressShards = busyShardsForRepo(repoName, snapshotsInProgress); + final boolean readyToExecute = deletionsInProgress == null || deletionsInProgress.getEntries().stream() + .noneMatch(entry -> entry.repository().equals(repoName) && entry.state() == SnapshotDeletionsInProgress.State.STARTED); for (IndexId index : indices) { final String indexName = index.getName(); final boolean isNewIndex = repositoryData.getIndices().containsKey(indexName) == false; @@ -1231,7 +1886,7 @@ private static ImmutableOpenMap busyShardsForRepo(String repoName, @Nullable SnapshotsInProgress snapshots) { + final List runningSnapshots = snapshots == null ? List.of() : snapshots.entries(); + final Set inProgressShards = new HashSet<>(); + for (SnapshotsInProgress.Entry runningSnapshot : runningSnapshots) { + if (runningSnapshot.repository().equals(repoName) == false) { + continue; + } + for (ObjectObjectCursor shard : runningSnapshot.shards()) { + if (shard.value.isActive()) { + inProgressShards.add(shard.key); + } + } + } + return inProgressShards; + } + /** * Returns the data streams that are currently being snapshotted (with partial == false) and that are contained in the * indices-to-check set. */ - public static Set snapshottingDataStreams(final ClusterState currentState, final Set dataStreamsToCheck) { + public static Set snapshottingDataStreams(final ClusterState currentState, + final Set dataStreamsToCheck) { + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); + if (snapshots == null) { + return emptySet(); + } + Map dataStreams = currentState.metadata().dataStreams(); - return currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries().stream() + return snapshots.entries().stream() .filter(e -> e.partial() == false) .flatMap(e -> e.dataStreams().stream()) .filter(ds -> dataStreams.containsKey(ds) && dataStreamsToCheck.contains(ds)) @@ -1285,8 +1974,13 @@ public static Set snapshottingDataStreams(final ClusterState currentStat * Returns the indices that are currently being snapshotted (with partial == false) and that are contained in the indices-to-check set. */ public static Set snapshottingIndices(final ClusterState currentState, final Set indicesToCheck) { + final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE); + if (snapshots == null) { + return emptySet(); + } + final Set indices = new HashSet<>(); - for (final SnapshotsInProgress.Entry entry : currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) { + for (final SnapshotsInProgress.Entry entry : snapshots.entries()) { if (entry.partial() == false) { for (IndexId index : entry.indices()) { IndexMetadata indexMetadata = currentState.metadata().index(index.getName()); @@ -1326,16 +2020,20 @@ protected void doClose() { } /** - * Assert that no in-memory state for any running snapshot operation exists in this instance. + * Assert that no in-memory state for any running snapshot-create or -delete operation exists in this instance. */ public boolean assertAllListenersResolved() { - synchronized (endingSnapshots) { - final DiscoveryNode localNode = clusterService.localNode(); - assert endingSnapshots.isEmpty() : "Found leaked ending snapshots " + endingSnapshots - + " on [" + localNode + "]"; - assert snapshotCompletionListeners.isEmpty() : "Found leaked snapshot completion listeners " + snapshotCompletionListeners - + " on [" + localNode + "]"; - } + final DiscoveryNode localNode = clusterService.localNode(); + assert endingSnapshots.isEmpty() : "Found leaked ending snapshots " + endingSnapshots + + " on [" + localNode + "]"; + assert snapshotCompletionListeners.isEmpty() : "Found leaked snapshot completion listeners " + snapshotCompletionListeners + + " on [" + localNode + "]"; + assert currentlyFinalizing.isEmpty() : "Found leaked finalizations " + currentlyFinalizing + + " on [" + localNode + "]"; + assert snapshotDeletionListeners.isEmpty() : "Found leaked snapshot delete listeners " + snapshotDeletionListeners + + " on [" + localNode + "]"; + assert repositoryOperations.isEmpty() : "Found leaked snapshots to finalize " + repositoryOperations + + " on [" + localNode + "]"; return true; } @@ -1344,23 +2042,44 @@ private static class SnapshotStateExecutor implements ClusterStateTaskExecutor execute(ClusterState currentState, List tasks) { - final SnapshotsInProgress snapshots = currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); int changedCount = 0; final List entries = new ArrayList<>(); - for (SnapshotsInProgress.Entry entry : snapshots.entries()) { + final Map> reusedShardIdsByRepo = new HashMap<>(); + for (SnapshotsInProgress.Entry entry : currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY).entries()) { ImmutableOpenMap.Builder shards = ImmutableOpenMap.builder(); boolean updated = false; for (UpdateIndexShardSnapshotStatusRequest updateSnapshotState : tasks) { + final ShardId finishedShardId = updateSnapshotState.shardId(); if (entry.snapshot().equals(updateSnapshotState.snapshot())) { logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshot(), - updateSnapshotState.shardId(), updateSnapshotState.status().state()); + finishedShardId, updateSnapshotState.status().state()); if (updated == false) { shards.putAll(entry.shards()); updated = true; } - shards.put(updateSnapshotState.shardId(), updateSnapshotState.status()); + shards.put(finishedShardId, updateSnapshotState.status()); changedCount++; + } else { + final String updatedRepository = updateSnapshotState.snapshot().getRepository(); + final Set reusedShardIds = reusedShardIdsByRepo.computeIfAbsent(updatedRepository, k -> new HashSet<>()); + if (entry.repository().equals(updatedRepository) && + entry.state().completed() == false && reusedShardIds.contains(finishedShardId) == false + && entry.shards().keys().contains(finishedShardId)) { + final ShardSnapshotStatus existingStatus = entry.shards().get(finishedShardId); + if (existingStatus.state() != ShardState.QUEUED) { + continue; + } + if (updated == false) { + shards.putAll(entry.shards()); + updated = true; + } + final ShardSnapshotStatus finishedStatus = updateSnapshotState.status(); + logger.trace("Starting [{}] on [{}] with generation [{}]", finishedShardId, + finishedStatus.nodeId(), finishedStatus.generation()); + shards.put(finishedShardId, new ShardSnapshotStatus(finishedStatus.nodeId(), finishedStatus.generation())); + reusedShardIds.add(finishedShardId); + } } } @@ -1380,8 +2099,8 @@ private static class SnapshotStateExecutor implements ClusterStateTaskExecutor 0) { logger.trace("changed cluster state triggered by {} snapshot state updates", changedCount); return ClusterTasksResult.builder().successes(tasks) - .build(ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, - SnapshotsInProgress.of(entries)).build()); + .build(ClusterState.builder(currentState).putCustom(SnapshotsInProgress.TYPE, + SnapshotsInProgress.of(unmodifiableList(entries))).build()); } return ClusterTasksResult.builder().successes(tasks).build(currentState); } @@ -1418,7 +2137,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS final SnapshotsInProgress.Entry updatedEntry = snapshotsInProgress.snapshot(request.snapshot()); // If the entry is still in the cluster state and is completed, try finalizing the snapshot in the repo if (updatedEntry != null && updatedEntry.state().completed()) { - endSnapshot(updatedEntry, newState.metadata()); + endSnapshot(updatedEntry, newState.metadata(), null); } } } @@ -1457,4 +2176,173 @@ protected ClusterBlockException checkBlock(UpdateIndexShardSnapshotStatusRequest return null; } } + + /** + * Cluster state update task that removes all {@link SnapshotsInProgress.Entry} and {@link SnapshotDeletionsInProgress.Entry} for a + * given repository from the cluster state and afterwards fails all relevant listeners in {@link #snapshotCompletionListeners} and + * {@link #snapshotDeletionListeners}. + */ + private final class FailPendingRepoTasksTask extends ClusterStateUpdateTask { + + // Snapshots to fail after the state update + private final List snapshotsToFail = new ArrayList<>(); + + // Delete uuids to fail because after the state update + private final List deletionsToFail = new ArrayList<>(); + + // Failure that caused the decision to fail all snapshots and deletes for a repo + private final Exception failure; + + private final String repository; + + FailPendingRepoTasksTask(String repository, Exception failure) { + this.repository = repository; + this.failure = failure; + } + + @Override + public ClusterState execute(ClusterState currentState) { + final SnapshotDeletionsInProgress deletionsInProgress = + currentState.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY); + boolean changed = false; + final List remainingEntries = deletionsInProgress.getEntries(); + List updatedEntries = new ArrayList<>(remainingEntries.size()); + for (SnapshotDeletionsInProgress.Entry entry : remainingEntries) { + if (entry.repository().equals(repository)) { + changed = true; + deletionsToFail.add(entry.uuid()); + } else { + updatedEntries.add(entry); + } + } + final SnapshotDeletionsInProgress updatedDeletions = changed ? SnapshotDeletionsInProgress.of(updatedEntries) : null; + final SnapshotsInProgress snapshotsInProgress = + currentState.custom(SnapshotsInProgress.TYPE, SnapshotsInProgress.EMPTY); + final List snapshotEntries = new ArrayList<>(); + boolean changedSnapshots = false; + for (SnapshotsInProgress.Entry entry : snapshotsInProgress.entries()) { + if (entry.repository().equals(repository)) { + // We failed to read repository data for this delete, it is not the job of SnapshotsService to + // retry these kinds of issues so we fail all the pending snapshots + snapshotsToFail.add(entry.snapshot()); + changedSnapshots = true; + } else { + // Entry is for another repository we just keep it as is + snapshotEntries.add(entry); + } + } + final SnapshotsInProgress updatedSnapshotsInProgress = changedSnapshots ? SnapshotsInProgress.of(snapshotEntries) : null; + return updateWithSnapshots(currentState, updatedSnapshotsInProgress, updatedDeletions); + } + + @Override + public void onFailure(String source, Exception e) { + logger.info( + () -> new ParameterizedMessage("Failed to remove all snapshot tasks for repo [{}] from cluster state", repository), e); + failAllListenersOnMasterFailOver(e); + } + + @Override + public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { + logger.warn(() -> + new ParameterizedMessage("Removed all snapshot tasks for repository [{}] from cluster state, now failing listeners", + repository), failure); + synchronized (currentlyFinalizing) { + Tuple finalization; + while ((finalization = repositoryOperations.pollFinalization(repository)) != null) { + assert snapshotsToFail.contains(finalization.v1().snapshot()) : + "[" + finalization.v1() + "] not found in snapshots to fail " + snapshotsToFail; + } + leaveRepoLoop(repository); + for (Snapshot snapshot : snapshotsToFail) { + failSnapshotCompletionListeners(snapshot, failure); + } + for (String delete : deletionsToFail) { + failListenersIgnoringException(snapshotDeletionListeners.remove(delete), failure); + repositoryOperations.finishDeletion(delete); + } + } + } + } + + private static final class OngoingRepositoryOperations { + + /** + * Map of repository name to a deque of {@link SnapshotsInProgress.Entry} that need to be finalized for the repository and the + * {@link Metadata to use when finalizing}. + */ + private final Map> snapshotsToFinalize = new HashMap<>(); + + /** + * Set of delete operations currently being executed against the repository. The values in this set are the delete UUIDs returned + * by {@link SnapshotDeletionsInProgress.Entry#uuid()}. + */ + private final Set runningDeletions = Collections.synchronizedSet(new HashSet<>()); + + @Nullable + private Metadata latestKnownMetaData; + + @Nullable + synchronized Tuple pollFinalization(String repository) { + assertConsistent(); + final SnapshotsInProgress.Entry nextEntry; + final Deque queued = snapshotsToFinalize.get(repository); + if (queued == null) { + return null; + } + nextEntry = queued.pollFirst(); + assert nextEntry != null; + final Tuple res = Tuple.tuple(nextEntry, latestKnownMetaData); + if (queued.isEmpty()) { + snapshotsToFinalize.remove(repository); + } + if (snapshotsToFinalize.isEmpty()) { + latestKnownMetaData = null; + } + assert assertConsistent(); + return res; + } + + boolean startDeletion(String deleteUUID) { + return runningDeletions.add(deleteUUID); + } + + void finishDeletion(String deleteUUID) { + runningDeletions.remove(deleteUUID); + } + + synchronized void addFinalization(SnapshotsInProgress.Entry entry, Metadata metadata) { + snapshotsToFinalize.computeIfAbsent(entry.repository(), k -> new LinkedList<>()).add(entry); + this.latestKnownMetaData = metadata; + assertConsistent(); + } + + /** + * Clear all state associated with running snapshots. To be used on master-failover if the current node stops + * being master. + */ + synchronized void clear() { + snapshotsToFinalize.clear(); + runningDeletions.clear(); + latestKnownMetaData = null; + } + + synchronized boolean isEmpty() { + return snapshotsToFinalize.isEmpty(); + } + + synchronized boolean assertNotQueued(Snapshot snapshot) { + assert snapshotsToFinalize.getOrDefault(snapshot.getRepository(), new LinkedList<>()).stream() + .noneMatch(entry -> entry.snapshot().equals(snapshot)) : "Snapshot [" + snapshot + "] is still in finalization queue"; + return true; + } + + synchronized boolean assertConsistent() { + assert (latestKnownMetaData == null && snapshotsToFinalize.isEmpty()) + || (latestKnownMetaData != null && snapshotsToFinalize.isEmpty() == false) : + "Should not hold on to metadata if there are no more queued snapshots"; + assert snapshotsToFinalize.values().stream().noneMatch(Collection::isEmpty) : "Found empty queue in " + snapshotsToFinalize; + return true; + } + } } diff --git a/server/src/main/java/org/elasticsearch/snapshots/package-info.java b/server/src/main/java/org/elasticsearch/snapshots/package-info.java index 55a0c93433b07..5c08aadece0b7 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/package-info.java +++ b/server/src/main/java/org/elasticsearch/snapshots/package-info.java @@ -98,5 +98,42 @@ *
  • After the deletion of the snapshot's data from the repository finishes, the {@code SnapshotsService} will submit a cluster state * update to remove the deletion's entry in {@code SnapshotDeletionsInProgress} which concludes the process of deleting a snapshot.
  • * + * + *

    Concurrent Snapshot Operations

    + * + * Snapshot create and delete operations may be started concurrently. Operations targeting different repositories run independently of + * each other. Multiple operations targeting the same repository are executed according to the following rules: + * + *

    Concurrent Snapshot Creation

    + * + * If multiple snapshot creation jobs are started at the same time, the data-node operations of multiple snapshots may run in parallel + * across different shards. If multiple snapshots want to snapshot a certain shard, then the shard snapshots for that shard will be + * executed one by one. This is enforced by the master node setting the shard's snapshot state to + * {@link org.elasticsearch.cluster.SnapshotsInProgress.ShardSnapshotStatus#UNASSIGNED_QUEUED} for all but one snapshot. The order of + * operations on a single shard is given by the order in which the snapshots were started. + * As soon as all shards for a given snapshot have finished, it will be finalized as explained above. Finalization will happen one snapshot + * at a time, working in the order in which snapshots had their shards completed. + * + *

    Concurrent Snapshot Deletes

    + * + * A snapshot delete will be executed as soon as there are no more shard snapshots or snapshot finalizations executing running for a given + * repository. Before a delete is executed on the repository it will be set to state + * {@link org.elasticsearch.cluster.SnapshotDeletionsInProgress.State#STARTED}. If it cannot be executed when it is received it will be + * set to state {@link org.elasticsearch.cluster.SnapshotDeletionsInProgress.State#WAITING} initially. + * If a delete is received for a given repository while there is already an ongoing delete for the same repository, there are two possible + * scenarios: + * 1. If the delete is in state {@code META_DATA} (i.e. already running on the repository) then the new delete will be added in state + * {@code WAITING} and will be executed after the current delete. The only exception here would be the case where the new delete covers + * the exact same snapshots as the already running delete. In this case no new delete operation is added and second delete request will + * simply wait for the existing delete to return. + * 2. If the existing delete is in state {@code WAITING} then the existing + * {@link org.elasticsearch.cluster.SnapshotDeletionsInProgress.Entry} in the cluster state will be updated to cover both the snapshots + * in the existing delete as well as additional snapshots that may be found in the second delete request. + * + * In either of the above scenarios, in-progress snapshots will be aborted in the same cluster state update that adds a delete to the + * cluster state, if a delete applies to them. + * + * If a snapshot request is received while there already is a delete in the cluster state for the same repository, that snapshot will not + * start doing any shard snapshots until the delete has been executed. */ package org.elasticsearch.snapshots; diff --git a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java index 060e34a1297e6..c3527d5b6432c 100644 --- a/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/serialization/ClusterSerializationTests.java @@ -117,10 +117,10 @@ public void testSnapshotDeletionsInProgressSerialization() throws Exception { ClusterState.Builder builder = ClusterState.builder(ClusterState.EMPTY_STATE) .putCustom(SnapshotDeletionsInProgress.TYPE, - SnapshotDeletionsInProgress.of(List.of( - new SnapshotDeletionsInProgress.Entry( - Collections.singletonList(new SnapshotId("snap1", UUIDs.randomBase64UUID())), "repo1", - randomNonNegativeLong(), randomNonNegativeLong())))); + SnapshotDeletionsInProgress.of(List.of( + new SnapshotDeletionsInProgress.Entry( + Collections.singletonList(new SnapshotId("snap1", UUIDs.randomBase64UUID())), "repo1", + randomNonNegativeLong(), randomNonNegativeLong(), SnapshotDeletionsInProgress.State.STARTED)))); if (includeRestore) { builder.putCustom(RestoreInProgress.TYPE, new RestoreInProgress.Builder().add( diff --git a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java index 02aaaf8b0202d..4782baf987300 100644 --- a/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java +++ b/server/src/test/java/org/elasticsearch/discovery/AbstractDisruptionTestCase.java @@ -121,7 +121,7 @@ List startCluster(int numberOfNodes) { return nodes; } - static final Settings DEFAULT_SETTINGS = Settings.builder() + public static final Settings DEFAULT_SETTINGS = Settings.builder() .put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "5s") // for hitting simulated network failures quickly .put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) // for hitting simulated network failures quickly .put(FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING.getKey(), "5s") // for hitting simulated network failures quickly diff --git a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java index 0ee651f219a35..6a7738c1c32e4 100644 --- a/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/RepositoriesServiceTests.java @@ -166,7 +166,7 @@ public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryS @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { listener.onResponse(null); } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index fdd8af46ddbd2..8650cd9917c33 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -210,6 +210,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.IntStream; import java.util.stream.Stream; import static java.util.Collections.emptyMap; @@ -224,7 +225,6 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.endsWith; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.iterableWithSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -552,31 +552,19 @@ public void testConcurrentSnapshotCreateAndDeleteOther() { createSnapshotResponse -> client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-2") .execute(createOtherSnapshotResponseStepListener)); - final StepListener deleteSnapshotStepListener = new StepListener<>(); + final StepListener deleteSnapshotStepListener = new StepListener<>(); continueOrDie(createOtherSnapshotResponseStepListener, - createSnapshotResponse -> client().admin().cluster().deleteSnapshot( - new DeleteSnapshotRequest(repoName, snapshotName), ActionListener.wrap( - resp -> deleteSnapshotStepListener.onResponse(true), - e -> { - final Throwable unwrapped = - ExceptionsHelper.unwrap(e, ConcurrentSnapshotExecutionException.class); - assertThat(unwrapped, instanceOf(ConcurrentSnapshotExecutionException.class)); - deleteSnapshotStepListener.onResponse(false); - }))); + createSnapshotResponse -> client().admin().cluster().prepareDeleteSnapshot( + repoName, snapshotName).execute(deleteSnapshotStepListener)); final StepListener createAnotherSnapshotResponseStepListener = new StepListener<>(); continueOrDie(deleteSnapshotStepListener, deleted -> { - if (deleted) { - // The delete worked out, creating a third snapshot - client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName).setWaitForCompletion(true) + client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName).setWaitForCompletion(true) .execute(createAnotherSnapshotResponseStepListener); - continueOrDie(createAnotherSnapshotResponseStepListener, createSnapshotResponse -> + continueOrDie(createAnotherSnapshotResponseStepListener, createSnapshotResponse -> assertEquals(createSnapshotResponse.getSnapshotInfo().state(), SnapshotState.SUCCESS)); - } else { - createAnotherSnapshotResponseStepListener.onResponse(null); - } }); deterministicTaskQueue.runAllRunnableTasks(); @@ -614,11 +602,16 @@ public void testBulkSnapshotDeleteWithAbort() { createIndexResponse -> client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) .setWaitForCompletion(true).execute(createSnapshotResponseStepListener)); - final StepListener createOtherSnapshotResponseStepListener = new StepListener<>(); + final int inProgressSnapshots = randomIntBetween(1, 5); + final StepListener> createOtherSnapshotResponseStepListener = new StepListener<>(); + final ActionListener createSnapshotListener = + new GroupedActionListener<>(createOtherSnapshotResponseStepListener, inProgressSnapshots); - continueOrDie(createSnapshotResponseStepListener, - createSnapshotResponse -> client().admin().cluster().prepareCreateSnapshot(repoName, "snapshot-2") - .execute(createOtherSnapshotResponseStepListener)); + continueOrDie(createSnapshotResponseStepListener, createSnapshotResponse -> { + for (int i = 0; i < inProgressSnapshots; i++) { + client().admin().cluster().prepareCreateSnapshot(repoName, "other-" + i).execute(createSnapshotListener); + } + }); final StepListener deleteSnapshotStepListener = new StepListener<>(); @@ -1009,6 +1002,66 @@ public void testSuccessfulSnapshotWithConcurrentDynamicMappingUpdates() { assertEquals(0, snapshotInfo.failedShards()); } + public void testRunConcurrentSnapshots() { + setupTestCluster(randomFrom(1, 3, 5), randomIntBetween(2, 10)); + + final String repoName = "repo"; + final List snapshotNames = IntStream.range(1, randomIntBetween(2, 4)) + .mapToObj(i -> "snapshot-" + i).collect(Collectors.toList()); + final String index = "test"; + final int shards = randomIntBetween(1, 10); + final int documents = randomIntBetween(1, 100); + + final TestClusterNodes.TestClusterNode masterNode = + testClusterNodes.currentMaster(testClusterNodes.nodes.values().iterator().next().clusterService.state()); + + final StepListener> allSnapshotsListener = new StepListener<>(); + final ActionListener snapshotListener = + new GroupedActionListener<>(allSnapshotsListener, snapshotNames.size()); + final AtomicBoolean doneIndexing = new AtomicBoolean(false); + continueOrDie(createRepoAndIndex(repoName, index, shards), createIndexResponse -> { + for (String snapshotName : snapshotNames) { + scheduleNow(() -> client().admin().cluster().prepareCreateSnapshot(repoName, snapshotName) + .setWaitForCompletion(true).execute(snapshotListener)); + } + final BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + for (int i = 0; i < documents; ++i) { + bulkRequest.add(new IndexRequest(index).source(Collections.singletonMap("foo", "bar" + i))); + } + final StepListener bulkResponseStepListener = new StepListener<>(); + client().bulk(bulkRequest, bulkResponseStepListener); + continueOrDie(bulkResponseStepListener, bulkResponse -> { + assertFalse("Failures in bulk response: " + bulkResponse.buildFailureMessage(), bulkResponse.hasFailures()); + assertEquals(documents, bulkResponse.getItems().length); + doneIndexing.set(true); + }); + }); + + final AtomicBoolean doneSnapshotting = new AtomicBoolean(false); + continueOrDie(allSnapshotsListener, createSnapshotResponses -> { + for (CreateSnapshotResponse createSnapshotResponse : createSnapshotResponses) { + final SnapshotInfo snapshotInfo = createSnapshotResponse.getSnapshotInfo(); + assertThat(snapshotInfo.state(), is(SnapshotState.SUCCESS)); + } + doneSnapshotting.set(true); + }); + + runUntil(() -> doneIndexing.get() && doneSnapshotting.get(), TimeUnit.MINUTES.toMillis(5L)); + SnapshotsInProgress finalSnapshotsInProgress = masterNode.clusterService.state().custom(SnapshotsInProgress.TYPE); + assertFalse(finalSnapshotsInProgress.entries().stream().anyMatch(entry -> entry.state().completed() == false)); + final Repository repository = masterNode.repositoriesService.repository(repoName); + Collection snapshotIds = getRepositoryData(repository).getSnapshotIds(); + assertThat(snapshotIds, hasSize(snapshotNames.size())); + + for (SnapshotId snapshotId : snapshotIds) { + final SnapshotInfo snapshotInfo = repository.getSnapshotInfo(snapshotId); + assertEquals(SnapshotState.SUCCESS, snapshotInfo.state()); + assertThat(snapshotInfo.indices(), containsInAnyOrder(index)); + assertEquals(shards, snapshotInfo.successfulShards()); + assertEquals(0, snapshotInfo.failedShards()); + } + } + private RepositoryData getRepositoryData(Repository repository) { final PlainActionFuture res = PlainActionFuture.newFuture(); repository.getRepositoryData(res); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java index c115996809327..6d1168f419b12 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/RestoreOnlyRepository.java @@ -105,7 +105,7 @@ public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryS @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { listener.onResponse(null); } diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java index 2c1834f5e60c0..8e07290bda18d 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -139,7 +139,7 @@ protected void disableRepoConsistencyCheck(String reason) { } protected RepositoryData getRepositoryData(String repository) { - return getRepositoryData(internalCluster().getMasterNodeInstance(RepositoriesService.class).repository(repository)); + return getRepositoryData(internalCluster().getCurrentMasterNodeInstance(RepositoriesService.class).repository(repository)); } protected RepositoryData getRepositoryData(Repository repository) { @@ -232,6 +232,12 @@ public static String blockMasterFromFinalizingSnapshotOnIndexFile(final String r return masterName; } + public static void blockMasterFromDeletingIndexNFile(String repositoryName) { + final String masterName = internalCluster().getMasterName(); + ((MockRepository)internalCluster().getInstance(RepositoriesService.class, masterName) + .repository(repositoryName)).setBlockOnDeleteIndexFile(); + } + public static String blockMasterFromFinalizingSnapshotOnSnapFile(final String repositoryName) { final String masterName = internalCluster().getMasterName(); ((MockRepository)internalCluster().getInstance(RepositoriesService.class, masterName) @@ -249,6 +255,11 @@ public static String blockNodeWithIndex(final String repositoryName, final Strin return null; } + public static void blockNodeOnAnyFiles(String repository, String nodeName) { + ((MockRepository) internalCluster().getInstance(RepositoriesService.class, nodeName) + .repository(repository)).setBlockOnAnyFiles(true); + } + public static void blockDataNode(String repository, String nodeName) { ((MockRepository) internalCluster().getInstance(RepositoriesService.class, nodeName) .repository(repository)).blockOnDataFiles(true); @@ -280,7 +291,8 @@ public static void waitForBlockOnAnyDataNode(String repository, TimeValue timeou assertTrue("No repository is blocked waiting on a data node", blocked); } - public static void unblockNode(final String repository, final String node) { + public void unblockNode(final String repository, final String node) { + logger.info("--> unblocking [{}] on node [{}]", repository, node); ((MockRepository)internalCluster().getInstance(RepositoriesService.class, node).repository(repository)).unblock(); } @@ -416,7 +428,7 @@ protected void awaitNoMoreRunningOperations(String viaNode) throws Exception { state.custom(SnapshotDeletionsInProgress.TYPE, SnapshotDeletionsInProgress.EMPTY).hasDeletionsInProgress() == false); } - private void awaitClusterState(String viaNode, Predicate statePredicate) throws Exception { + protected void awaitClusterState(String viaNode, Predicate statePredicate) throws Exception { final ClusterService clusterService = internalCluster().getInstance(ClusterService.class, viaNode); final ThreadPool threadPool = internalCluster().getInstance(ThreadPool.class, viaNode); final ClusterStateObserver observer = new ClusterStateObserver(clusterService, logger, threadPool.getThreadContext()); diff --git a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java index 14e6daef1c775..b8cff6d52d653 100644 --- a/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/elasticsearch/snapshots/mockstore/MockRepository.java @@ -42,6 +42,7 @@ import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.repositories.fs.FsRepository; import java.io.IOException; @@ -104,10 +105,12 @@ public long getFailureCount() { private final Environment env; - private volatile boolean blockOnControlFiles; + private volatile boolean blockOnAnyFiles; private volatile boolean blockOnDataFiles; + private volatile boolean blockOnDeleteIndexN; + /** Allows blocking on writing the index-N blob; this is a way to enforce blocking the * finalization of a snapshot, while permitting other IO operations to proceed unblocked. */ private volatile boolean blockOnWriteIndexFile; @@ -125,7 +128,7 @@ public MockRepository(RepositoryMetadata metadata, Environment environment, randomDataFileIOExceptionRate = metadata.settings().getAsDouble("random_data_file_io_exception_rate", 0.0); useLuceneCorruptionException = metadata.settings().getAsBoolean("use_lucene_corruption", false); maximumNumberOfFailures = metadata.settings().getAsLong("max_failure_number", 100L); - blockOnControlFiles = metadata.settings().getAsBoolean("block_on_control", false); + blockOnAnyFiles = metadata.settings().getAsBoolean("block_on_control", false); blockOnDataFiles = metadata.settings().getAsBoolean("block_on_data", false); blockAndFailOnWriteSnapFile = metadata.settings().getAsBoolean("block_on_snap", false); randomPrefix = metadata.settings().get("random", "default"); @@ -171,9 +174,10 @@ public synchronized void unblock() { blocked = false; // Clean blocking flags, so we wouldn't try to block again blockOnDataFiles = false; - blockOnControlFiles = false; + blockOnAnyFiles = false; blockOnWriteIndexFile = false; blockAndFailOnWriteSnapFile = false; + blockOnDeleteIndexN = false; this.notifyAll(); } @@ -181,6 +185,10 @@ public void blockOnDataFiles(boolean blocked) { blockOnDataFiles = blocked; } + public void setBlockOnAnyFiles(boolean blocked) { + blockOnAnyFiles = blocked; + } + public void setBlockAndFailOnWriteSnapFiles(boolean blocked) { blockAndFailOnWriteSnapFile = blocked; } @@ -189,6 +197,10 @@ public void setBlockOnWriteIndexFile(boolean blocked) { blockOnWriteIndexFile = blocked; } + public void setBlockOnDeleteIndexFile() { + blockOnDeleteIndexN = true; + } + public boolean blocked() { return blocked; } @@ -197,8 +209,8 @@ private synchronized boolean blockExecution() { logger.debug("[{}] Blocking execution", metadata.name()); boolean wasBlocked = false; try { - while (blockOnDataFiles || blockOnControlFiles || blockOnWriteIndexFile || - blockAndFailOnWriteSnapFile) { + while (blockOnDataFiles || blockOnAnyFiles || blockOnWriteIndexFile || + blockAndFailOnWriteSnapFile || blockOnDeleteIndexN) { blocked = true; this.wait(); wasBlocked = true; @@ -275,7 +287,7 @@ private void maybeIOExceptionOrBlock(String blobName) throws IOException { if (shouldFail(blobName, randomControlIOExceptionRate) && (incrementAndGetFailureCount() < maximumNumberOfFailures)) { logger.info("throwing random IOException for file [{}] at path [{}]", blobName, path()); throw new IOException("Random IOException"); - } else if (blockOnControlFiles) { + } else if (blockOnAnyFiles) { blockExecutionAndMaybeWait(blobName); } else if (blobName.startsWith("snap-") && blockAndFailOnWriteSnapFile) { blockExecutionAndFail(blobName); @@ -339,6 +351,15 @@ public DeleteResult delete() throws IOException { return deleteResult.add(deleteBlobCount, deleteByteCount); } + @Override + public void deleteBlobsIgnoringIfNotExists(List blobNames) throws IOException { + if (blockOnDeleteIndexN && blobNames.stream().anyMatch( + name -> name.startsWith(BlobStoreRepository.INDEX_FILE_PREFIX))) { + blockExecutionAndMaybeWait("index-{N}"); + } + super.deleteBlobsIgnoringIfNotExists(blobNames); + } + @Override public Map listBlobs() throws IOException { maybeIOExceptionOrBlock(""); diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java index e7d7d71091faf..611d11f1e2e08 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRepository.java @@ -266,7 +266,7 @@ public void finalizeSnapshot(ShardGenerations shardGenerations, long repositoryS @Override public void deleteSnapshots(Collection snapshotIds, long repositoryStateId, Version repositoryMetaVersion, - ActionListener listener) { + ActionListener listener) { throw new UnsupportedOperationException("Unsupported for repository of type: " + TYPE); } diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java index d52968bd3b0f2..3c5817646319d 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/slm/SnapshotRetentionTaskTests.java @@ -355,7 +355,8 @@ public void testOkToDeleteSnapshots() { SnapshotDeletionsInProgress delInProgress = SnapshotDeletionsInProgress.of( Collections.singletonList(new SnapshotDeletionsInProgress.Entry( - Collections.singletonList(snapshot.getSnapshotId()), snapshot.getRepository(), 0, 0))); + Collections.singletonList(snapshot.getSnapshotId()), snapshot.getRepository(), 0, 0, + SnapshotDeletionsInProgress.State.STARTED))); state = ClusterState.builder(new ClusterName("cluster")) .putCustom(SnapshotDeletionsInProgress.TYPE, delInProgress) .build();