diff --git a/docs/reference/modules/indices/recovery.asciidoc b/docs/reference/modules/indices/recovery.asciidoc index 5f67a63f437e6..1993d3da5db37 100644 --- a/docs/reference/modules/indices/recovery.asciidoc +++ b/docs/reference/modules/indices/recovery.asciidoc @@ -82,9 +82,21 @@ When `indices.recovery.use_snapshots` is `false` {es} will construct this new copy by transferring the index data from the current primary. When this setting is `true` {es} will attempt to copy the index data from a recent snapshot first, and will only copy data from the primary if it cannot identify a -suitable snapshot. +suitable snapshot. Defaults to `true`. + Setting this option to `true` reduces your operating costs if your cluster runs in an environment where the node-to-node data transfer costs are higher than the costs of recovering data from a snapshot. It also reduces the amount of work that the primary must do during a recovery. ++ +Additionally, repositories having the setting `use_for_peer_recovery=true` +will be consulted to find a good snapshot when recovering a shard. If none +of the registered repositories have this setting defined, index files will +be recovered from the source node. + +`indices.recovery.max_concurrent_snapshot_file_downloads`:: +(<>, Expert) Number of snapshot file downloads requests +sent in parallel to the target node for each recovery. Defaults to `5`. ++ +Do not increase this setting without carefully verifying that your cluster has +the resources available to handle the extra load that will result. diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SnapshotBasedRecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SnapshotBasedRecoveryIT.java new file mode 100644 index 0000000000000..3cf1339569d50 --- /dev/null +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/SnapshotBasedRecoveryIT.java @@ -0,0 +1,139 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.upgrades; + +import org.apache.http.client.methods.HttpPost; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.rest.action.document.RestBulkAction; +import org.elasticsearch.search.builder.SearchSourceBuilder; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; +import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; + +public class SnapshotBasedRecoveryIT extends AbstractRollingTestCase { + public void testSnapshotBasedRecovery() throws Exception { + final String indexName = "snapshot_based_recovery"; + final String repositoryName = "snapshot_based_recovery_repo"; + final int numDocs = 200; + switch (CLUSTER_TYPE) { + case OLD: + Settings.Builder settings = Settings.builder() + .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) + .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms") + .put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0"); // fail faster + createIndex(indexName, settings.build()); + ensureGreen(indexName); + indexDocs(indexName, numDocs); + flush(indexName, true); + + registerRepository( + repositoryName, + "fs", + true, + Settings.builder() + .put("location", "./snapshot_based_recovery") + .put(BlobStoreRepository.USE_FOR_PEER_RECOVERY_SETTING.getKey(), true) + .build() + ); + + createSnapshot(repositoryName, "snap", true); + + updateIndexSettings(indexName, Settings.builder().put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1)); + ensureGreen(indexName); + break; + case MIXED: + case UPGRADED: + // Drop replicas + updateIndexSettings(indexName, Settings.builder().put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0)); + ensureGreen(indexName); + + updateIndexSettings(indexName, Settings.builder().put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1)); + ensureGreen(indexName); + assertMatchAllReturnsAllDocuments(indexName, numDocs); + assertMatchQueryReturnsAllDocuments(indexName, numDocs); + break; + default: + throw new IllegalStateException("unknown type " + CLUSTER_TYPE); + } + } + + private void assertMatchAllReturnsAllDocuments(String indexName, int numDocs) throws IOException { + Map searchResults = search(indexName, QueryBuilders.matchAllQuery()); + List> hits = extractValue(searchResults, "hits.hits"); + assertThat(hits.size(), equalTo(numDocs)); + for (Map hit : hits) { + String docId = extractValue(hit, "_id"); + assertThat(Integer.parseInt(docId), allOf(greaterThanOrEqualTo(0), lessThan(numDocs))); + assertThat(extractValue(hit, "_source.field"), equalTo(Integer.parseInt(docId))); + assertThat(extractValue(hit, "_source.text"), equalTo("Some text " + docId)); + } + } + + private void assertMatchQueryReturnsAllDocuments(String indexName, int numDocs) throws IOException { + Map searchResults = search(indexName, QueryBuilders.matchQuery("text", "some")); + List> hits = extractValue(searchResults, "hits.hits"); + assertThat(hits.size(), equalTo(numDocs)); + for (Map hit : hits) { + String docId = extractValue(hit, "_id"); + assertThat(Integer.parseInt(docId), allOf(greaterThanOrEqualTo(0), lessThan(numDocs))); + assertThat(extractValue(hit, "_source.field"), equalTo(Integer.parseInt(docId))); + assertThat(extractValue(hit, "_source.text"), equalTo("Some text " + docId)); + } + } + + private static Map search(String index, QueryBuilder query) throws IOException { + final Request request = new Request(HttpPost.METHOD_NAME, '/' + index + "/_search"); + request.setJsonEntity(new SearchSourceBuilder().size(1000).query(query).toString()); + + final Response response = client().performRequest(request); + assertOK(response); + + final Map responseAsMap = responseAsMap(response); + assertThat( + extractValue(responseAsMap, "_shards.failed"), + equalTo(0) + ); + return responseAsMap; + } + + private void indexDocs(String indexName, int numDocs) throws IOException { + final StringBuilder bulkBody = new StringBuilder(); + for (int i = 0; i < numDocs; i++) { + bulkBody.append("{\"index\":{\"_id\":\"").append(i).append("\", \"_type\": \"_doc\"}}\n"); + bulkBody.append("{\"field\":").append(i).append(",\"text\":\"Some text ").append(i).append("\"}\n"); + } + + final Request documents = new Request(HttpPost.METHOD_NAME, '/' + indexName + "/_bulk"); + documents.addParameter("refresh", "true"); + documents.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); + documents.setJsonEntity(bulkBody.toString()); + assertOK(client().performRequest(documents)); + } + + @SuppressWarnings("unchecked") + private static T extractValue(Map map, String path) { + return (T) XContentMapValues.extractValue(path, map); + } +} diff --git a/qa/snapshot-based-recoveries/build.gradle b/qa/snapshot-based-recoveries/build.gradle new file mode 100644 index 0000000000000..bfceaf642d35e --- /dev/null +++ b/qa/snapshot-based-recoveries/build.gradle @@ -0,0 +1,2 @@ +apply plugin: 'elasticsearch.standalone-rest-test' +apply plugin: 'elasticsearch.internal-test-artifact' diff --git a/qa/snapshot-based-recoveries/fs/build.gradle b/qa/snapshot-based-recoveries/fs/build.gradle new file mode 100644 index 0000000000000..b5cdb145379d7 --- /dev/null +++ b/qa/snapshot-based-recoveries/fs/build.gradle @@ -0,0 +1,36 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +apply plugin: 'elasticsearch.java-rest-test' +apply plugin: 'elasticsearch.rest-resources' + +dependencies { + javaRestTestImplementation(testArtifact(project(':qa:snapshot-based-recoveries'))) +} + +final File repoDir = file("$buildDir/testclusters/snapshot-recoveries-repo") + +restResources { + restApi { + include 'indices', 'search', 'bulk', 'snapshot' + } +} + +tasks.withType(Test).configureEach { + doFirst { + delete(repoDir) + } + systemProperty 'tests.path.repo', repoDir +} + +testClusters.all { + testDistribution = 'DEFAULT' + numberOfNodes = 3 + setting 'path.repo', repoDir.absolutePath + setting 'xpack.security.enabled', 'false' +} diff --git a/qa/snapshot-based-recoveries/fs/src/javaRestTest/java/org/elasticsearch/recovery/FsSnapshotBasedRecoveryIT.java b/qa/snapshot-based-recoveries/fs/src/javaRestTest/java/org/elasticsearch/recovery/FsSnapshotBasedRecoveryIT.java new file mode 100644 index 0000000000000..a49f462c11466 --- /dev/null +++ b/qa/snapshot-based-recoveries/fs/src/javaRestTest/java/org/elasticsearch/recovery/FsSnapshotBasedRecoveryIT.java @@ -0,0 +1,26 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.recovery; + +import org.elasticsearch.common.settings.Settings; + +public class FsSnapshotBasedRecoveryIT extends AbstractSnapshotBasedRecoveryRestTestCase { + + @Override + protected String repositoryType() { + return "fs"; + } + + @Override + protected Settings repositorySettings() { + return Settings.builder() + .put("location", System.getProperty("tests.path.repo")) + .build(); + } +} diff --git a/qa/snapshot-based-recoveries/src/test/java/org/elasticsearch/recovery/AbstractSnapshotBasedRecoveryRestTestCase.java b/qa/snapshot-based-recoveries/src/test/java/org/elasticsearch/recovery/AbstractSnapshotBasedRecoveryRestTestCase.java new file mode 100644 index 0000000000000..dac2eba1ba02f --- /dev/null +++ b/qa/snapshot-based-recoveries/src/test/java/org/elasticsearch/recovery/AbstractSnapshotBasedRecoveryRestTestCase.java @@ -0,0 +1,134 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.recovery; + +import org.apache.http.client.methods.HttpPost; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.test.rest.ESRestTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import static org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; + +public abstract class AbstractSnapshotBasedRecoveryRestTestCase extends ESRestTestCase { + private static final String REPOSITORY_NAME = "repository"; + private static final String SNAPSHOT_NAME = "snapshot-for-recovery"; + + protected abstract String repositoryType(); + + protected abstract Settings repositorySettings(); + + public void testRecoveryUsingSnapshots() throws Exception { + final String repositoryType = repositoryType(); + Settings repositorySettings = Settings.builder().put(repositorySettings()) + .put(BlobStoreRepository.USE_FOR_PEER_RECOVERY_SETTING.getKey(), true) + .build(); + + registerRepository(REPOSITORY_NAME, repositoryType, true, repositorySettings); + + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + + createIndex( + indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build() + ); + ensureGreen(indexName); + + final int numDocs = randomIntBetween(1, 500); + indexDocs(indexName, numDocs); + + forceMerge(indexName, randomBoolean(), randomBoolean()); + + deleteSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME, true); + createSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME, true); + + // Add a new replica + updateIndexSettings(indexName, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)); + ensureGreen(indexName); + + for (int i = 0; i < 4; i++) { + assertSearchResultsAreCorrect(indexName, numDocs); + } + deleteSnapshot(REPOSITORY_NAME, SNAPSHOT_NAME, false); + } + + private void assertSearchResultsAreCorrect(String indexName, int numDocs) throws IOException { + if (randomBoolean()) { + Map searchResults = search(indexName, QueryBuilders.matchAllQuery()); + assertThat(extractValue(searchResults, "hits.total.value"), equalTo(numDocs)); + List> hits = extractValue(searchResults, "hits.hits"); + for (Map hit : hits) { + String docId = extractValue(hit, "_id"); + assertThat(Integer.parseInt(docId), allOf(greaterThanOrEqualTo(0), lessThan(numDocs))); + assertThat(extractValue(hit, "_source.field"), equalTo(Integer.parseInt(docId))); + assertThat(extractValue(hit, "_source.text"), equalTo("Some text " + docId)); + } + } else { + Map searchResults = search(indexName, QueryBuilders.matchQuery("text", "some")); + assertThat(extractValue(searchResults, "hits.total.value"), equalTo(numDocs)); + } + } + + private static void forceMerge(String index, boolean onlyExpungeDeletes, boolean flush) throws IOException { + final Request request = new Request(HttpPost.METHOD_NAME, '/' + index + "/_forcemerge"); + request.addParameter("only_expunge_deletes", Boolean.toString(onlyExpungeDeletes)); + request.addParameter("flush", Boolean.toString(flush)); + assertOK(client().performRequest(request)); + } + + private void indexDocs(String indexName, int numDocs) throws IOException { + final StringBuilder bulkBody = new StringBuilder(); + for (int i = 0; i < numDocs; i++) { + bulkBody.append("{\"index\":{\"_id\":\"").append(i).append("\"}}\n"); + bulkBody.append("{\"field\":").append(i).append(",\"text\":\"Some text ").append(i).append("\"}\n"); + } + + final Request documents = new Request(HttpPost.METHOD_NAME, '/' + indexName + "/_bulk"); + documents.addParameter("refresh", Boolean.TRUE.toString()); + documents.setJsonEntity(bulkBody.toString()); + assertOK(client().performRequest(documents)); + } + + private static Map search(String index, QueryBuilder query) throws IOException { + final Request request = new Request(HttpPost.METHOD_NAME, '/' + index + "/_search"); + request.setJsonEntity(new SearchSourceBuilder().trackTotalHits(true).query(query).toString()); + + final Response response = client().performRequest(request); + assertOK(response); + + final Map responseAsMap = responseAsMap(response); + assertThat( + extractValue(responseAsMap, "_shards.failed"), + equalTo(0) + ); + return responseAsMap; + } + + @SuppressWarnings("unchecked") + private static T extractValue(Map map, String path) { + return (T) XContentMapValues.extractValue(path, map); + } +} diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 4c162ffd01c7c..8a1cb7030caf9 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -18,6 +18,7 @@ import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryRequest; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; @@ -85,6 +86,7 @@ import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.repositories.Repository; import org.elasticsearch.repositories.RepositoryData; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; import org.elasticsearch.snapshots.Snapshot; import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.tasks.Task; @@ -117,6 +119,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; @@ -618,11 +621,7 @@ public void testSnapshotRecovery() throws Exception { String nodeA = internalCluster().startNode(); logger.info("--> create repository"); - assertAcked(client().admin().cluster().preparePutRepository(REPO_NAME) - .setType("fs").setSettings(Settings.builder() - .put("location", randomRepoPath()) - .put("compress", false) - ).get()); + createRepository(randomBoolean()); ensureGreen(); @@ -630,14 +629,7 @@ public void testSnapshotRecovery() throws Exception { createAndPopulateIndex(INDEX_NAME, 1, SHARD_COUNT, REPLICA_COUNT); logger.info("--> snapshot"); - CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(REPO_NAME, SNAP_NAME) - .setWaitForCompletion(true).setIndices(INDEX_NAME).get(); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); - assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), - equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); - - assertThat(client().admin().cluster().prepareGetSnapshots(REPO_NAME).setSnapshots(SNAP_NAME).get() - .getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS)); + CreateSnapshotResponse createSnapshotResponse = createSnapshot(INDEX_NAME); client().admin().indices().prepareClose(INDEX_NAME).execute().actionGet(); @@ -770,6 +762,7 @@ public void testTransientErrorsDuringRecoveryAreRetried() throws Exception { PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG, PeerRecoveryTargetService.Actions.TRANSLOG_OPS, PeerRecoveryTargetService.Actions.FILES_INFO, + PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT, PeerRecoveryTargetService.Actions.FILE_CHUNK, PeerRecoveryTargetService.Actions.CLEAN_FILES, PeerRecoveryTargetService.Actions.FINALIZE @@ -777,6 +770,10 @@ public void testTransientErrorsDuringRecoveryAreRetried() throws Exception { final String recoveryActionToBlock = randomFrom(recoveryActions); logger.info("--> will temporarily interrupt recovery action between blue & red on [{}]", recoveryActionToBlock); + if (recoveryActionToBlock.equals(PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT)) { + createSnapshotThatCanBeUsedDuringRecovery(indexName); + } + MockTransportService blueTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName); MockTransportService redTransportService = @@ -813,6 +810,9 @@ public void testTransientErrorsDuringRecoveryAreRetried() throws Exception { ).get(); ensureGreen(); + if (recoveryActionToBlock.equals(PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT)) { + assertThat(handlingBehavior.blocksRemaining.get(), is(equalTo(0))); + } searchResponse = client(redNodeName).prepareSearch(indexName).setPreference("_local").get(); assertHitCount(searchResponse, numDocs); } finally { @@ -940,6 +940,7 @@ public void testDisconnectsWhileRecovering() throws Exception { String[] recoveryActions = new String[]{ PeerRecoverySourceService.Actions.START_RECOVERY, PeerRecoveryTargetService.Actions.FILES_INFO, + PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT, PeerRecoveryTargetService.Actions.FILE_CHUNK, PeerRecoveryTargetService.Actions.CLEAN_FILES, //RecoveryTarget.Actions.TRANSLOG_OPS, <-- may not be sent if already flushed @@ -950,6 +951,11 @@ public void testDisconnectsWhileRecovering() throws Exception { final boolean dropRequests = randomBoolean(); logger.info("--> will {} between blue & red on [{}]", dropRequests ? "drop requests" : "break connection", recoveryActionToBlock); + // Generate a snapshot to recover from it if the action that we're blocking is sending the request snapshot files + if (recoveryActionToBlock.equals(PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT)) { + createSnapshotThatCanBeUsedDuringRecovery(indexName); + } + MockTransportService blueMockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName); MockTransportService redMockTransportService = @@ -1041,6 +1047,10 @@ public void testDisconnectsDuringRecovery() throws Exception { ensureSearchable(indexName); assertHitCount(client().prepareSearch(indexName).get(), numDocs); + if (randomBoolean()) { + createSnapshotThatCanBeUsedDuringRecovery(indexName); + } + MockTransportService masterTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, masterNodeName); MockTransportService blueMockTransportService = @@ -1876,4 +1886,48 @@ private void assertGlobalCheckpointIsStableAndSyncedInAllNodes(String indexName, assertBusy(() -> assertThat(nodeIndexShard.getLastSyncedGlobalCheckpoint(), equalTo(maxSeqNo))); } } + + private void createSnapshotThatCanBeUsedDuringRecovery(String indexName) throws Exception { + // Ensure that the safe commit == latest commit + assertBusy(() -> { + ShardStats stats = client().admin().indices().prepareStats(indexName).clear().get() + .asMap().entrySet().stream().filter(e -> e.getKey().shardId().getId() == 0) + .map(Map.Entry::getValue).findFirst().orElse(null); + assertThat(stats, is(notNullValue())); + assertThat(stats.getSeqNoStats(), is(notNullValue())); + + assertThat(Strings.toString(stats.getSeqNoStats()), + stats.getSeqNoStats().getMaxSeqNo(), equalTo(stats.getSeqNoStats().getGlobalCheckpoint())); + }, 60, TimeUnit.SECONDS); + + // Force merge to make sure that the resulting snapshot would contain the same index files as the safe commit + ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(indexName).setFlush(randomBoolean()).get(); + assertThat(forceMergeResponse.getTotalShards(), equalTo(forceMergeResponse.getSuccessfulShards())); + createRepository(true); + createSnapshot(indexName); + } + + private void createRepository(boolean enableSnapshotPeerRecoveries) { + assertAcked( + client().admin().cluster().preparePutRepository(REPO_NAME) + .setType("fs") + .setSettings(Settings.builder() + .put("location", randomRepoPath()) + .put(BlobStoreRepository.USE_FOR_PEER_RECOVERY_SETTING.getKey(), enableSnapshotPeerRecoveries) + .put("compress", false) + ).get() + ); + } + + private CreateSnapshotResponse createSnapshot(String indexName) { + CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(REPO_NAME, SNAP_NAME) + .setWaitForCompletion(true).setIndices(indexName).get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards())); + + assertThat(client().admin().cluster().prepareGetSnapshots(REPO_NAME).setSnapshots(SNAP_NAME).get() + .getSnapshots().get(0).state(), equalTo(SnapshotState.SUCCESS)); + return createSnapshotResponse; + } } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/SnapshotBasedIndexRecoveryIT.java b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/SnapshotBasedIndexRecoveryIT.java new file mode 100644 index 0000000000000..738e49f3098f1 --- /dev/null +++ b/server/src/internalClusterTest/java/org/elasticsearch/indices/recovery/SnapshotBasedIndexRecoveryIT.java @@ -0,0 +1,791 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.indices.recovery; + +import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; +import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsResponse; +import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; +import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; +import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.metadata.RepositoryMetadata; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.blobstore.BlobContainer; +import org.elasticsearch.common.blobstore.support.FilterBlobContainer; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.env.Environment; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.MergePolicyConfig; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.recovery.RecoveryStats; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.RepositoryPlugin; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.Repository; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; +import org.elasticsearch.repositories.fs.FsRepository; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.snapshots.AbstractSnapshotIntegTestCase; +import org.elasticsearch.snapshots.RestoreInfo; +import org.elasticsearch.snapshots.SnapshotInfo; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalSettingsPlugin; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING; +import static org.elasticsearch.indices.recovery.RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) +public class SnapshotBasedIndexRecoveryIT extends AbstractSnapshotIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return Arrays.asList( + TestRepositoryPlugin.class, + MockTransportService.TestPlugin.class, + InternalSettingsPlugin.class + ); + } + + public static class TestRepositoryPlugin extends Plugin implements RepositoryPlugin { + public static final String FAULTY_TYPE = "faultyrepo"; + public static final String INSTRUMENTED_TYPE = "instrumentedrepo"; + + @Override + public Map getRepositories( + Environment env, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings + ) { + return org.elasticsearch.core.Map.of( + FAULTY_TYPE, + metadata -> new FaultyRepository(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings), + INSTRUMENTED_TYPE, + metadata -> new InstrumentedRepo(metadata, env, namedXContentRegistry, clusterService, bigArrays, recoverySettings) + ); + } + } + + public static class InstrumentedRepo extends FsRepository { + AtomicLong totalBytesRead = new AtomicLong(); + + public InstrumentedRepo(RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings) { + super(metadata, environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings); + } + + @Override + public BlobContainer shardContainer(IndexId indexId, int shardId) { + return new FilterBlobContainer(super.shardContainer(indexId, shardId)) { + @Override + protected BlobContainer wrapChild(BlobContainer child) { + return child; + } + + @Override + public InputStream readBlob(String blobName) throws IOException { + // Take into account only index files + if (blobName.startsWith("__") == false) { + return super.readBlob(blobName); + } + + return new FilterInputStream(super.readBlob(blobName)) { + @Override + public int read(byte[] b, int off, int len) throws IOException { + int read = super.read(b, off, len); + if (read > 0) { + totalBytesRead.addAndGet(read); + } + return read; + } + }; + } + }; + } + } + + public static class FaultyRepository extends FsRepository { + public FaultyRepository(RepositoryMetadata metadata, + Environment environment, + NamedXContentRegistry namedXContentRegistry, + ClusterService clusterService, + BigArrays bigArrays, + RecoverySettings recoverySettings) { + super(metadata, environment, namedXContentRegistry, clusterService, bigArrays, recoverySettings); + } + + @Override + public BlobContainer shardContainer(IndexId indexId, int shardId) { + return new FilterBlobContainer(super.shardContainer(indexId, shardId)) { + @Override + protected BlobContainer wrapChild(BlobContainer child) { + return child; + } + + @Override + public InputStream readBlob(String blobName) throws IOException { + // Fail only in index files + if (blobName.startsWith("__") == false) { + return super.readBlob(blobName); + } + + return new FilterInputStream(super.readBlob(blobName)) { + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (randomBoolean()) { + // Return random data + for (int i = 0; i < len; i++) { + b[off + i] = randomByte(); + } + return len; + } else { + if (randomBoolean()) { + throw new IOException("Unable to read blob " + blobName); + } else { + // Skip some file chunks + int read = super.read(b, off, len); + return read / 2; + } + } + } + }; + } + }; + } + } + + public void testPeerRecoveryUsesSnapshots() throws Exception { + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + String snapshot = "snap"; + createSnapshot(repoName, snapshot, Collections.singletonList(indexName)); + + String targetNode = internalCluster().startDataOnlyNode(); + + MockTransportService sourceMockTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, sourceNode); + MockTransportService targetMockTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, targetNode); + + sourceMockTransportService.addSendBehavior(targetMockTransportService, (connection, requestId, action, request, options) -> { + assertNotEquals(PeerRecoveryTargetService.Actions.FILE_CHUNK, action); + connection.sendRequest(requestId, action, request, options); + }); + + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + ensureGreen(); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + RepositoriesService repositoriesService = internalCluster().getInstance(RepositoriesService.class, targetNode); + InstrumentedRepo repository = (InstrumentedRepo) repositoriesService.repository(repoName); + + // segments_N and .si files are recovered from the file metadata directly + long expectedRecoveredBytesFromRepo = 0; + for (RecoveryState.FileDetail fileDetail : recoveryState.getIndex().fileDetails()) { + if (fileDetail.name().startsWith("segments") || fileDetail.name().endsWith(".si")) { + continue; + } + expectedRecoveredBytesFromRepo += fileDetail.recovered(); + } + + assertThat(repository.totalBytesRead.get(), is(equalTo(expectedRecoveredBytesFromRepo))); + + long snapshotSizeForIndex = getSnapshotSizeForIndex(repoName, snapshot, indexName); + assertThat(repository.totalBytesRead.get(), is(greaterThan(0L))); + assertThat(repository.totalBytesRead.get(), is(lessThanOrEqualTo(snapshotSizeForIndex))); + + assertDocumentsAreEqual(indexName, numDocs); + } + + public void testFallbacksToSourceNodeWhenSnapshotDownloadFails() throws Exception { + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.FAULTY_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + String targetNode = internalCluster().startDataOnlyNode(); + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + ensureGreen(); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + assertDocumentsAreEqual(indexName, numDocs); + } + + public void testRateLimitingIsEnforced() throws Exception { + try { + updateSetting(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), "50k"); + + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + //we theoretically only need more than 256 bytes, since SimpleRateLimiter.MIN_PAUSE_CHECK_MSEC=5. + // We do need a bit more though to ensure we have enough time to handle if network and CI is generally slow, + // since if the experienced download rate is less than 50KB there will be no throttling. + // I would at least 4x that to be on a somewhat safe side against things like a single GC. + int numDocs = randomIntBetween(1000, 2000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, "fs"); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + String targetNode = internalCluster().startDataOnlyNode(); + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + ensureGreen(); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + assertDocumentsAreEqual(indexName, numDocs); + + NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear() + .setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get(); + for (NodeStats nodeStats : statsResponse.getNodes()) { + RecoveryStats recoveryStats = nodeStats.getIndices().getRecoveryStats(); + String nodeName = nodeStats.getNode().getName(); + if (nodeName.equals(sourceNode)) { + assertThat(recoveryStats.throttleTime().getMillis(), is(equalTo(0L))); + } + if (nodeName.equals(targetNode)) { + assertThat(recoveryStats.throttleTime().getMillis(), is(greaterThan(0L))); + } + } + } finally { + updateSetting(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING.getKey(), null); + } + } + + public void testPeerRecoveryTriesToUseMostOfTheDataFromAnAvailableSnapshot() throws Exception { + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + forceMerge(); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + int docsIndexedAfterSnapshot = randomIntBetween(1, 2000); + indexDocs(indexName, numDocs, docsIndexedAfterSnapshot); + + String targetNode = internalCluster().startDataOnlyNode(); + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + ensureGreen(); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + InstrumentedRepo repository = getRepositoryOnNode(repoName, targetNode); + + long snapshotSizeForIndex = getSnapshotSizeForIndex(repoName, "snap", indexName); + assertThat(repository.totalBytesRead.get(), is(greaterThan(0L))); + assertThat(repository.totalBytesRead.get(), is(lessThanOrEqualTo(snapshotSizeForIndex))); + + assertDocumentsAreEqual(indexName, numDocs + docsIndexedAfterSnapshot); + } + + public void testPeerRecoveryDoNotUseSnapshotsWhenSegmentsAreNotShared() throws Exception { + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + int docsIndexedAfterSnapshot = randomIntBetween(1, 2000); + indexDocs(indexName, numDocs, docsIndexedAfterSnapshot); + forceMerge(); + + String targetNode = internalCluster().startDataOnlyNode(); + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + ensureGreen(); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + InstrumentedRepo repository = getRepositoryOnNode(repoName, targetNode); + + assertThat(repository.totalBytesRead.get(), is(equalTo(0L))); + + assertDocumentsAreEqual(indexName, numDocs + docsIndexedAfterSnapshot); + } + + public void testRecoveryIsCancelledAfterDeletingTheIndex() throws Exception { + updateSetting(INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS.getKey(), "1"); + + try { + String sourceNode = internalCluster().startDataOnlyNode(); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.require._name", sourceNode) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, numDocs, numDocs); + + String repoName = "repo"; + createRepo(repoName, "fs"); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + String targetNode = internalCluster().startDataOnlyNode(); + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put("index.routing.allocation.require._name", targetNode)).get() + ); + + MockTransportService targetMockTransportService = + (MockTransportService) internalCluster().getInstance(TransportService.class, targetNode); + + CountDownLatch recoverSnapshotFileRequestReceived = new CountDownLatch(1); + CountDownLatch respondToRecoverSnapshotFile = new CountDownLatch(1); + AtomicInteger numberOfRecoverSnapshotFileRequestsReceived = new AtomicInteger(); + targetMockTransportService.addRequestHandlingBehavior(PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT, + (handler, request, channel, task) -> { + assertThat(numberOfRecoverSnapshotFileRequestsReceived.incrementAndGet(), is(equalTo(1))); + recoverSnapshotFileRequestReceived.countDown(); + respondToRecoverSnapshotFile.await(); + handler.messageReceived(request, channel, task); + } + ); + + recoverSnapshotFileRequestReceived.await(); + + assertAcked(client().admin().indices().prepareDelete(indexName).get()); + + respondToRecoverSnapshotFile.countDown(); + + assertThat(indexExists(indexName), is(equalTo(false))); + } finally { + updateSetting(INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS.getKey(), null); + } + } + + public void testRecoveryAfterRestoreUsesSnapshots() throws Exception { + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + assertAcked(client().admin().indices().prepareDelete(indexName).get()); + + List restoredIndexDataNodes = internalCluster().startDataOnlyNodes(2); + RestoreSnapshotResponse restoreSnapshotResponse = client().admin().cluster() + .prepareRestoreSnapshot(repoName, "snap") + .setIndices(indexName) + .setIndexSettings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1) + .put("index.routing.allocation.include._name", String.join(",", restoredIndexDataNodes)) + ).setWaitForCompletion(true) + .get(); + + RestoreInfo restoreInfo = restoreSnapshotResponse.getRestoreInfo(); + assertThat(restoreInfo.successfulShards(), is(equalTo(restoreInfo.totalShards()))); + + ensureGreen(indexName); + assertDocumentsAreEqual(indexName, numDocs); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + String sourceNode = recoveryState.getSourceNode().getName(); + String targetNode = recoveryState.getTargetNode().getName(); + + assertThat(restoredIndexDataNodes.contains(sourceNode), is(equalTo(true))); + assertThat(restoredIndexDataNodes.contains(targetNode), is(equalTo(true))); + assertPeerRecoveryWasSuccessful(recoveryState, sourceNode, targetNode); + + // Since we did a restore first, and the index is static the data retrieved by the target node + // via repository should be equal to the amount of data that the source node retrieved from the repo + InstrumentedRepo sourceRepo = getRepositoryOnNode(repoName, sourceNode); + InstrumentedRepo targetRepo = getRepositoryOnNode(repoName, targetNode); + assertThat(sourceRepo.totalBytesRead.get(), is(equalTo(targetRepo.totalBytesRead.get()))); + + long snapshotSizeForIndex = getSnapshotSizeForIndex(repoName, "snap", indexName); + + assertThat(sourceRepo.totalBytesRead.get(), is(greaterThan(0L))); + assertThat(sourceRepo.totalBytesRead.get(), is(lessThanOrEqualTo(snapshotSizeForIndex))); + } + + public void testReplicaRecoveryUsesSnapshots() throws Exception { + List dataNodes = internalCluster().startDataOnlyNodes(3); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .put("index.routing.allocation.include._name", String.join(",", dataNodes)) + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)) + ); + + ensureGreen(indexName); + assertDocumentsAreEqual(indexName, numDocs); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + String currentPrimary = recoveryState.getSourceNode().getName(); + String replica = recoveryState.getTargetNode().getName(); + assertPeerRecoveryWasSuccessful(recoveryState, currentPrimary, replica); + + long snapshotSizeForIndex = getSnapshotSizeForIndex(repoName, "snap", indexName); + + InstrumentedRepo replicaRepo = getRepositoryOnNode(repoName, replica); + assertThat(replicaRepo.totalBytesRead.get(), is(greaterThan(0L))); + assertThat(replicaRepo.totalBytesRead.get(), is(lessThanOrEqualTo(snapshotSizeForIndex))); + + // Stop the current replica + if (randomBoolean()) { + internalCluster().stopNode(replica); + + ensureGreen(indexName); + assertDocumentsAreEqual(indexName, numDocs); + + RecoveryState recoveryStateAfterReplicaFailure = getLatestPeerRecoveryStateForShard(indexName, 0); + final String name = recoveryStateAfterReplicaFailure.getSourceNode().getName(); + final String newReplica = recoveryStateAfterReplicaFailure.getTargetNode().getName(); + assertPeerRecoveryWasSuccessful(recoveryStateAfterReplicaFailure, name, newReplica); + + InstrumentedRepo newReplicaRepo = getRepositoryOnNode(repoName, newReplica); + assertThat(newReplicaRepo.totalBytesRead.get(), is(greaterThan(0L))); + assertThat(newReplicaRepo.totalBytesRead.get(), is(lessThanOrEqualTo(snapshotSizeForIndex))); + } + } + + public void testDisabledSnapshotBasedRecoveryUsesSourceFiles() throws Exception { + updateSetting(RecoverySettings.INDICES_RECOVERY_USE_SNAPSHOTS_SETTING.getKey(), "false"); + + try { + internalCluster().ensureAtLeastNumDataNodes(2); + String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(MergePolicyConfig.INDEX_MERGE_ENABLED, false) + .put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s") + .build() + ); + + int numDocs = randomIntBetween(300, 1000); + indexDocs(indexName, 0, numDocs); + + String repoName = "repo"; + createRepo(repoName, TestRepositoryPlugin.INSTRUMENTED_TYPE); + createSnapshot(repoName, "snap", Collections.singletonList(indexName)); + + assertAcked( + client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)) + ); + + ensureGreen(indexName); + assertDocumentsAreEqual(indexName, numDocs); + + RecoveryState recoveryState = getLatestPeerRecoveryStateForShard(indexName, 0); + String currentPrimary = recoveryState.getSourceNode().getName(); + String replica = recoveryState.getTargetNode().getName(); + assertPeerRecoveryWasSuccessful(recoveryState, currentPrimary, replica); + + InstrumentedRepo replicaRepo = getRepositoryOnNode(repoName, replica); + assertThat(replicaRepo.totalBytesRead.get(), is(equalTo(0L))); + } finally { + updateSetting(RecoverySettings.INDICES_RECOVERY_USE_SNAPSHOTS_SETTING.getKey(), null); + } + } + + private long getSnapshotSizeForIndex(String repository, String snapshot, String index) { + GetSnapshotsResponse getSnapshotsResponse = + client().admin().cluster().prepareGetSnapshots(repository).addSnapshots(snapshot).get(); + for (SnapshotInfo snapshotInfo : getSnapshotsResponse.getSnapshots()) { + SnapshotInfo.IndexSnapshotDetails indexSnapshotDetails = snapshotInfo.indexSnapshotDetails().get(index); + assertThat(indexSnapshotDetails, is(notNullValue())); + return indexSnapshotDetails.getSize().getBytes(); + } + + return -1; + } + + private void indexDocs(String indexName, int docIdOffset, int docCount) throws Exception { + IndexRequestBuilder[] builders = new IndexRequestBuilder[docCount]; + for (int i = 0; i < builders.length; i++) { + int docId = i + docIdOffset; + builders[i] = client().prepareIndex(indexName, "_doc") + .setId(Integer.toString(docId)) + .setSource("field", docId, "field2", "Some text " + docId); + } + indexRandom(true, builders); + + // Ensure that the safe commit == latest commit + assertBusy(() -> { + ShardStats stats = client().admin().indices().prepareStats(indexName).clear().get() + .asMap().entrySet().stream().filter(e -> e.getKey().shardId().getId() == 0) + .map(Map.Entry::getValue).findFirst().orElse(null); + assertThat(stats, is(notNullValue())); + assertThat(stats.getSeqNoStats(), is(notNullValue())); + + assertThat(Strings.toString(stats.getSeqNoStats()), + stats.getSeqNoStats().getMaxSeqNo(), equalTo(stats.getSeqNoStats().getGlobalCheckpoint())); + }, 60, TimeUnit.SECONDS); + } + + private void assertDocumentsAreEqual(String indexName, int docCount) { + assertDocCount(indexName, docCount); + for (int testCase = 0; testCase < 3; testCase++) { + final SearchRequestBuilder searchRequestBuilder = client().prepareSearch(indexName) + .addSort("field", SortOrder.ASC) + .setSize(10_000); + + SearchResponse searchResponse; + switch (testCase) { + case 0: + searchResponse = searchRequestBuilder. + setQuery(QueryBuilders.matchAllQuery()).get(); + assertSearchResponseContainsAllIndexedDocs(searchResponse, docCount); + break; + case 1: + int docIdToMatch = randomIntBetween(0, docCount - 1); + searchResponse = searchRequestBuilder.setQuery(QueryBuilders.termQuery("field", docIdToMatch)).get(); + assertThat(searchResponse.getSuccessfulShards(), equalTo(1)); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + SearchHit searchHit = searchResponse.getHits().getAt(0); + + Map source = searchHit.getSourceAsMap(); + + assertThat(source, is(notNullValue())); + assertThat(source.get("field"), is(equalTo(docIdToMatch))); + assertThat(source.get("field2"), is(equalTo("Some text " + docIdToMatch))); + break; + case 2: + searchResponse = searchRequestBuilder.setQuery(QueryBuilders.matchQuery("field2", "text")).get(); + assertSearchResponseContainsAllIndexedDocs(searchResponse, docCount); + break; + default: + throw new IllegalStateException("Unexpected value: " + testCase); + } + } + } + + private void assertSearchResponseContainsAllIndexedDocs(SearchResponse searchResponse, long docCount) { + assertThat(searchResponse.getSuccessfulShards(), equalTo(1)); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(docCount)); + for (int i = 0; i < searchResponse.getHits().getHits().length; i++) { + SearchHit searchHit = searchResponse.getHits().getAt(i); + Map source = searchHit.getSourceAsMap(); + + assertThat(source, is(notNullValue())); + assertThat(source.get("field"), is(equalTo(i))); + assertThat(source.get("field2"), is(equalTo("Some text " + i))); + } + } + + private void assertPeerRecoveryWasSuccessful(RecoveryState recoveryState, String sourceNode, String targetNode) throws Exception { + assertThat(recoveryState.getStage(), equalTo(RecoveryState.Stage.DONE)); + assertThat(recoveryState.getRecoverySource(), equalTo(RecoverySource.PeerRecoverySource.INSTANCE)); + + assertThat(recoveryState.getSourceNode(), notNullValue()); + assertThat(recoveryState.getSourceNode().getName(), equalTo(sourceNode)); + assertThat(recoveryState.getTargetNode(), notNullValue()); + assertThat(recoveryState.getTargetNode().getName(), equalTo(targetNode)); + + RecoveryState.Index indexState = recoveryState.getIndex(); + assertThat(indexState.recoveredBytesPercent(), greaterThanOrEqualTo(0.0f)); + assertThat(indexState.recoveredBytesPercent(), lessThanOrEqualTo(100.0f)); + } + + private RecoveryState getLatestPeerRecoveryStateForShard(String indexName, int shardId) { + RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries(indexName).get(); + assertThat(recoveryResponse.hasRecoveries(), equalTo(true)); + List indexRecoveries = recoveryResponse.shardRecoveryStates().get(indexName); + assertThat(indexRecoveries, notNullValue()); + + List peerRecoveries = indexRecoveries.stream() + .filter(recoveryState -> recoveryState.getRecoverySource().equals(RecoverySource.PeerRecoverySource.INSTANCE)) + .filter(recoveryState -> recoveryState.getShardId().getId() == shardId) + .collect(Collectors.toList()); + + assertThat(peerRecoveries, is(not(empty()))); + return peerRecoveries.get(peerRecoveries.size() - 1); + } + + private void updateSetting(String key, String value) { + ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest(); + settingsRequest.persistentSettings(Settings.builder().put(key, value)); + assertAcked(client().admin().cluster().updateSettings(settingsRequest).actionGet()); + } + + private void createRepo(String repoName, String type) { + final Settings.Builder settings = Settings.builder() + .put(BlobStoreRepository.USE_FOR_PEER_RECOVERY_SETTING.getKey(), true) + .put("location", randomRepoPath()); + createRepository(logger, repoName, type, settings, true); + } +} 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 79206640c95c7..8e77a112e0c01 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -231,6 +231,7 @@ public void apply(Settings value, Settings current, Settings previous) { RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_OPERATIONS_SETTING, RecoverySettings.INDICES_RECOVERY_USE_SNAPSHOTS_SETTING, + RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index 82fbb2a288a27..337e494433c82 100644 --- a/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/elasticsearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -11,6 +11,9 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.Strings; +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.ByteSizeValue; import org.elasticsearch.common.xcontent.ParseField; import org.elasticsearch.common.xcontent.ToXContent; @@ -22,6 +25,7 @@ import java.io.IOException; import java.util.List; +import java.util.Objects; import java.util.stream.IntStream; /** @@ -32,7 +36,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment { /** * Information about snapshotted file */ - public static class FileInfo { + public static class FileInfo implements Writeable { private final String name; private final ByteSizeValue partSize; @@ -48,7 +52,7 @@ public static class FileInfo { * @param partSize size of the single chunk */ public FileInfo(String name, StoreFileMetadata metadata, ByteSizeValue partSize) { - this.name = name; + this.name = Objects.requireNonNull(name); this.metadata = metadata; long partBytes = Long.MAX_VALUE; @@ -71,6 +75,17 @@ public FileInfo(String name, StoreFileMetadata metadata, ByteSizeValue partSize) assert IntStream.range(0, numberOfParts).mapToLong(this::partBytes).sum() == metadata.length(); } + public FileInfo(StreamInput in) throws IOException { + this(in.readString(), new StoreFileMetadata(in), in.readOptionalWriteable(ByteSizeValue::new)); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(name); + metadata.writeTo(out); + out.writeOptionalWriteable(partSize); + } + /** * Returns the base file name * diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java index 90eab3619fc8f..c401aa6514bcb 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/MultiFileWriter.java @@ -16,15 +16,17 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.AbstractRefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; -import org.elasticsearch.core.AbstractRefCounted; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.transport.Transports; +import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; @@ -55,7 +57,6 @@ public MultiFileWriter(Store store, RecoveryState.Index indexState, String tempF private final ConcurrentMap openIndexOutputs = ConcurrentCollections.newConcurrentMap(); private final ConcurrentMap fileChunkWriters = ConcurrentCollections.newConcurrentMap(); - final Map tempFileNames = ConcurrentCollections.newConcurrentMap(); public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk) @@ -70,6 +71,48 @@ public void writeFileChunk(StoreFileMetadata fileMetadata, long position, Releas } } + public void writeFile(StoreFileMetadata fileMetadata, long readSnapshotFileBufferSize, InputStream stream) throws Exception { + ensureOpen.run(); + assert Transports.assertNotTransportThread("multi_file_writer"); + + String fileName = fileMetadata.name(); + String tempFileName = getTempNameForFile(fileName); + if (tempFileNames.containsKey(tempFileName)) { + throw new IllegalStateException("output for file [" + fileName + "] has already been created"); + } + tempFileNames.put(tempFileName, fileName); + + incRef(); + try(IndexOutput indexOutput = store.createVerifyingOutput(tempFileName, fileMetadata, IOContext.DEFAULT)) { + int bufferSize = Math.toIntExact(Math.min(readSnapshotFileBufferSize, fileMetadata.length())); + byte[] buffer = new byte[bufferSize]; + int length; + long bytesWritten = 0; + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, length); + indexState.addRecoveredBytesToFile(fileName, length); + bytesWritten += length; + } + + if (bytesWritten < fileMetadata.length()) { + throw new EOFException("Expected to write a file of length [" + fileMetadata.length() + "] " + + "but only [" + bytesWritten + "] bytes were written"); + } + + Store.verify(indexOutput); + assert Arrays.asList(store.directory().listAll()).contains(tempFileName) : + "expected: [" + tempFileName + "] in " + Arrays.toString(store.directory().listAll()); + store.directory().sync(Collections.singleton(tempFileName)); + } catch (Exception e) { + tempFileNames.remove(tempFileName); + store.deleteQuiet(tempFileName); + indexState.resetRecoveredBytesOfFile(fileName); + throw e; + } finally { + decRef(); + } + } + /** Get a temporary name for the provided file name. */ String getTempNameForFile(String origFile) { return tempFilePrefix + origFile; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index 6e8c3ff95ffbd..8ce4c829c6d9a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -325,6 +325,8 @@ private Tuple createRecovery Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks(), recoverySettings.getMaxConcurrentOperations(), + recoverySettings.getMaxConcurrentSnapshotFileDownloads(), + recoverySettings.getUseSnapshotsDuringRecovery(), recoveryPlannerService); return Tuple.tuple(handler, recoveryTarget); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index c28a85559f4e3..da020586d62f4 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -25,15 +25,15 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.core.CheckedFunction; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.core.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.core.CheckedFunction; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; @@ -41,8 +41,8 @@ import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.ShardLongFieldRange; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardLongFieldRange; import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -78,6 +78,7 @@ public class PeerRecoveryTargetService implements IndexEventListener { public static class Actions { public static final String FILES_INFO = "internal:index/shard/recovery/filesInfo"; + public static final String RESTORE_FILE_FROM_SNAPSHOT = "internal:index/shard/recovery/restore_file_from_snapshot"; public static final String FILE_CHUNK = "internal:index/shard/recovery/file_chunk"; public static final String CLEAN_FILES = "internal:index/shard/recovery/clean_files"; public static final String TRANSLOG_OPS = "internal:index/shard/recovery/translog_ops"; @@ -92,19 +93,26 @@ public static class Actions { private final RecoverySettings recoverySettings; private final ClusterService clusterService; + private final SnapshotFilesProvider snapshotFilesProvider; private final RecoveriesCollection onGoingRecoveries; - public PeerRecoveryTargetService(ThreadPool threadPool, TransportService transportService, - RecoverySettings recoverySettings, ClusterService clusterService) { + public PeerRecoveryTargetService(ThreadPool threadPool, + TransportService transportService, + RecoverySettings recoverySettings, + ClusterService clusterService, + SnapshotFilesProvider snapshotFilesProvider) { this.threadPool = threadPool; this.transportService = transportService; this.recoverySettings = recoverySettings; this.clusterService = clusterService; + this.snapshotFilesProvider = snapshotFilesProvider; this.onGoingRecoveries = new RecoveriesCollection(logger, threadPool); transportService.registerRequestHandler(Actions.FILES_INFO, ThreadPool.Names.GENERIC, RecoveryFilesInfoRequest::new, new FilesInfoRequestHandler()); + transportService.registerRequestHandler(Actions.RESTORE_FILE_FROM_SNAPSHOT, ThreadPool.Names.GENERIC, + RecoverySnapshotFileRequest::new, new RestoreFileFromSnapshotTransportRequestHandler()); transportService.registerRequestHandler(Actions.FILE_CHUNK, ThreadPool.Names.GENERIC, RecoveryFileChunkRequest::new, new FileChunkTransportRequestHandler()); transportService.registerRequestHandler(Actions.CLEAN_FILES, ThreadPool.Names.GENERIC, @@ -131,7 +139,8 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh public void startRecovery(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) { // create a new recovery status, and process... - final long recoveryId = onGoingRecoveries.startRecovery(indexShard, sourceNode, listener, recoverySettings.activityTimeout()); + final long recoveryId = + onGoingRecoveries.startRecovery(indexShard, sourceNode, snapshotFilesProvider, listener, recoverySettings.activityTimeout()); // we fork off quickly here and go async but this is called from the cluster state applier thread too and that can cause // assertions to trip if we executed it on the same thread hence we fork off to the generic threadpool. threadPool.generic().execute(new RecoveryRunner(recoveryId)); @@ -470,6 +479,22 @@ public void messageReceived(final RecoveryFileChunkRequest request, TransportCha } } + class RestoreFileFromSnapshotTransportRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(final RecoverySnapshotFileRequest request, TransportChannel channel, Task task) throws Exception { + try (RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.getRecoveryId(), request.getShardId())) { + final RecoveryTarget recoveryTarget = recoveryRef.target(); + final ActionListener listener = + createOrFinishListener(recoveryRef, channel, Actions.RESTORE_FILE_FROM_SNAPSHOT, request); + if (listener == null) { + return; + } + + recoveryTarget.restoreFileFromSnapshot(request.getRepository(), request.getIndexId(), request.getFileInfo(), listener); + } + } + } + private ActionListener createOrFinishListener(final RecoveryRef recoveryRef, final TransportChannel channel, final String action, final RecoveryTransportRequest request) { return createOrFinishListener(recoveryRef, channel, action, request, nullVal -> TransportResponse.Empty.INSTANCE); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index b7ac3a69df25e..0646949c08c0f 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -12,9 +12,9 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardClosedException; import org.elasticsearch.index.shard.ShardId; @@ -50,9 +50,12 @@ public RecoveriesCollection(Logger logger, ThreadPool threadPool) { * * @return the id of the new recovery. */ - public long startRecovery(IndexShard indexShard, DiscoveryNode sourceNode, - PeerRecoveryTargetService.RecoveryListener listener, TimeValue activityTimeout) { - RecoveryTarget recoveryTarget = new RecoveryTarget(indexShard, sourceNode, listener); + public long startRecovery(IndexShard indexShard, + DiscoveryNode sourceNode, + SnapshotFilesProvider snapshotFilesProvider, + PeerRecoveryTargetService.RecoveryListener listener, + TimeValue activityTimeout) { + RecoveryTarget recoveryTarget = new RecoveryTarget(indexShard, sourceNode, snapshotFilesProvider, listener); startRecoveryInternal(recoveryTarget, activityTimeout); return recoveryTarget.recoveryId(); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java index 1e2c98a99f9ed..c3c719e370edb 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java @@ -138,6 +138,15 @@ public class RecoverySettings { public static final Setting INDICES_RECOVERY_USE_SNAPSHOTS_SETTING = Setting.boolSetting("indices.recovery.use_snapshots", true, Property.Dynamic, Property.NodeScope); + public static final Setting INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS = + Setting.intSetting("indices.recovery.max_concurrent_snapshot_file_downloads", + 5, + 1, + 20, + Property.Dynamic, + Property.NodeScope + ); + public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue(512, ByteSizeUnit.KB); private volatile ByteSizeValue maxBytesPerSec; @@ -151,6 +160,7 @@ public class RecoverySettings { private volatile TimeValue internalActionRetryTimeout; private volatile TimeValue internalActionLongTimeout; private volatile boolean useSnapshotsDuringRecovery; + private volatile int maxConcurrentSnapshotFileDownloads; private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE; @@ -174,6 +184,7 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac()); } this.useSnapshotsDuringRecovery = INDICES_RECOVERY_USE_SNAPSHOTS_SETTING.get(settings); + this.maxConcurrentSnapshotFileDownloads = INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS.get(settings); logger.debug("using max_bytes_per_sec[{}]", maxBytesPerSec); @@ -188,6 +199,8 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { this::setInternalActionLongTimeout); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, this::setActivityTimeout); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_USE_SNAPSHOTS_SETTING, this::setUseSnapshotsDuringRecovery); + clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS, + this::setMaxConcurrentSnapshotFileDownloads); } public RateLimiter rateLimiter() { @@ -281,4 +294,12 @@ public boolean getUseSnapshotsDuringRecovery() { private void setUseSnapshotsDuringRecovery(boolean useSnapshotsDuringRecovery) { this.useSnapshotsDuringRecovery = useSnapshotsDuringRecovery; } + + public int getMaxConcurrentSnapshotFileDownloads() { + return maxConcurrentSnapshotFileDownloads; + } + + public void setMaxConcurrentSnapshotFileDownloads(int maxConcurrentSnapshotFileDownloads) { + this.maxConcurrentSnapshotFileDownloads = maxConcurrentSnapshotFileDownloads; + } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySnapshotFileRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySnapshotFileRequest.java new file mode 100644 index 0000000000000..13e647857980f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySnapshotFileRequest.java @@ -0,0 +1,80 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.indices.recovery; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.repositories.IndexId; + +import java.io.IOException; + +public class RecoverySnapshotFileRequest extends RecoveryTransportRequest { + private final long recoveryId; + private final ShardId shardId; + private final String repository; + private final IndexId indexId; + private final BlobStoreIndexShardSnapshot.FileInfo fileInfo; + + public RecoverySnapshotFileRequest(long recoveryId, + long requestSeqNo, + ShardId shardId, + String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + super(requestSeqNo); + this.recoveryId = recoveryId; + this.shardId = shardId; + this.repository = repository; + this.indexId = indexId; + this.fileInfo = fileInfo; + } + + public RecoverySnapshotFileRequest(StreamInput in) throws IOException { + super(in); + this.recoveryId = in.readLong(); + this.shardId = new ShardId(in); + this.repository = in.readString(); + this.indexId = new IndexId(in); + this.fileInfo = new BlobStoreIndexShardSnapshot.FileInfo(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + assert out.getVersion().onOrAfter(RecoverySettings.SNAPSHOT_RECOVERIES_SUPPORTED_VERSION) : + "Unexpected serialization version " + out.getVersion(); + super.writeTo(out); + out.writeLong(recoveryId); + shardId.writeTo(out); + out.writeString(repository); + indexId.writeTo(out); + fileInfo.writeTo(out); + } + + public long getRecoveryId() { + return recoveryId; + } + + public ShardId getShardId() { + return shardId; + } + + public String getRepository() { + return repository; + } + + public IndexId getIndexId() { + return indexId; + } + + public BlobStoreIndexShardSnapshot.FileInfo getFileInfo() { + return fileInfo; + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 29b50d059e2ef..1494fb7785f7c 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -36,9 +36,11 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CancellableThreads; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ListenableFuture; import org.elasticsearch.core.CheckedRunnable; +import org.elasticsearch.core.Nullable; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; import org.elasticsearch.core.TimeValue; @@ -71,17 +73,24 @@ import java.util.Collections; import java.util.Comparator; import java.util.Deque; +import java.util.HashSet; import java.util.List; import java.util.Locale; +import java.util.Set; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.stream.StreamSupport; +import static org.elasticsearch.common.util.CollectionUtils.concatLists; + /** * RecoverySourceHandler handles the three phases of shard recovery, which is * everything relating to copying the segment files as well as sending translog @@ -105,6 +114,8 @@ public class RecoverySourceHandler { private final RecoveryTargetHandler recoveryTarget; private final int maxConcurrentFileChunks; private final int maxConcurrentOperations; + private final int maxConcurrentSnapshotFileDownloads; + private final boolean useSnapshots; private final ThreadPool threadPool; private final RecoveryPlannerService recoveryPlannerService; private final CancellableThreads cancellableThreads = new CancellableThreads(); @@ -113,7 +124,8 @@ public class RecoverySourceHandler { public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, ThreadPool threadPool, StartRecoveryRequest request, int fileChunkSizeInBytes, int maxConcurrentFileChunks, - int maxConcurrentOperations, RecoveryPlannerService recoveryPlannerService) { + int maxConcurrentOperations, int maxConcurrentSnapshotFileDownloads, boolean useSnapshots, + RecoveryPlannerService recoveryPlannerService) { this.shard = shard; this.recoveryTarget = recoveryTarget; this.threadPool = threadPool; @@ -125,6 +137,8 @@ public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTar // if the target is on an old version, it won't be able to handle out-of-order file chunks. this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_6_7_0) ? maxConcurrentFileChunks : 1; this.maxConcurrentOperations = maxConcurrentOperations; + this.maxConcurrentSnapshotFileDownloads = maxConcurrentSnapshotFileDownloads; + this.useSnapshots = useSnapshots; } public StartRecoveryRequest getRequest() { @@ -481,7 +495,7 @@ void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, A startingSeqNo, translogOps.getAsInt(), getRequest().targetNode().getVersion(), - false, + useSnapshots, ActionListener.wrap(plan -> recoverFilesFromSourceAndSnapshot(plan, store, stopWatch, listener), listener::onFailure) ); @@ -548,6 +562,7 @@ void recoverFilesFromSourceAndSnapshot(ShardRecoveryPlan shardRecoveryPlan, } final StepListener sendFileInfoStep = new StepListener<>(); + final StepListener> recoverSnapshotFilesStep = new StepListener<>(); final StepListener sendFilesStep = new StepListener<>(); final StepListener createRetentionLeaseStep = new StepListener<>(); final StepListener cleanFilesStep = new StepListener<>(); @@ -561,11 +576,19 @@ void recoverFilesFromSourceAndSnapshot(ShardRecoveryPlan shardRecoveryPlan, sendFileInfoStep ); - final List sourceFiles = shardRecoveryPlan.getSourceFilesToRecover(); + sendFileInfoStep.whenComplete(r -> recoverSnapshotFiles(shardRecoveryPlan, recoverSnapshotFilesStep), listener::onFailure); + + recoverSnapshotFilesStep.whenComplete(filesFailedToRecoverFromSnapshot -> { + final List filesToRecoverFromSource; + if (filesFailedToRecoverFromSnapshot.isEmpty()) { + filesToRecoverFromSource = shardRecoveryPlan.getSourceFilesToRecover(); + } else { + filesToRecoverFromSource = concatLists(shardRecoveryPlan.getSourceFilesToRecover(), filesFailedToRecoverFromSnapshot); + } - sendFileInfoStep.whenComplete(r -> sendFiles(store, - sourceFiles.toArray(new StoreFileMetadata[0]), shardRecoveryPlan::getTranslogOps, sendFilesStep), listener::onFailure); + filesToRecoverFromSource.toArray(new StoreFileMetadata[0]), shardRecoveryPlan::getTranslogOps, sendFilesStep); + }, listener::onFailure); final long startingSeqNo = shardRecoveryPlan.getStartingSeqNo(); sendFilesStep.whenComplete(r -> createRetentionLease(startingSeqNo, createRetentionLeaseStep), listener::onFailure); @@ -594,6 +617,168 @@ void recoverFilesFromSourceAndSnapshot(ShardRecoveryPlan shardRecoveryPlan, }, listener::onFailure); } + /** + * Send requests to the target node to recover files from a given snapshot. In case of failure, the listener + * value contains the list of files that failed to be recovered from a snapshot. + */ + void recoverSnapshotFiles(ShardRecoveryPlan shardRecoveryPlan, + ActionListener> listener) { + ShardRecoveryPlan.SnapshotFilesToRecover snapshotFilesToRecover = shardRecoveryPlan.getSnapshotFilesToRecover(); + + if (snapshotFilesToRecover.isEmpty()) { + listener.onResponse(Collections.emptyList()); + return; + } + + new SnapshotRecoverFileRequestsSender(shardRecoveryPlan, listener).start(); + } + + private class SnapshotRecoverFileRequestsSender { + private final ShardRecoveryPlan.SnapshotFilesToRecover snapshotFilesToRecover; + private final ActionListener> listener; + private final CountDown countDown; + private final BlockingQueue pendingSnapshotFilesToRecover; + private final AtomicBoolean cancelled = new AtomicBoolean(); + private final Set> outstandingRequests = new HashSet<>(maxConcurrentSnapshotFileDownloads); + private List filesFailedToDownloadFromSnapshot; + + SnapshotRecoverFileRequestsSender(ShardRecoveryPlan shardRecoveryPlan, ActionListener> listener) { + this.snapshotFilesToRecover = shardRecoveryPlan.getSnapshotFilesToRecover(); + this.listener = listener; + this.countDown = new CountDown(shardRecoveryPlan.getSnapshotFilesToRecover().size()); + this.pendingSnapshotFilesToRecover = + new LinkedBlockingQueue<>(shardRecoveryPlan.getSnapshotFilesToRecover().getSnapshotFiles()); + } + + void start() { + for (int i = 0; i < maxConcurrentSnapshotFileDownloads; i++) { + sendRequest(); + } + } + + void sendRequest() { + BlobStoreIndexShardSnapshot.FileInfo snapshotFileToRecover = pendingSnapshotFilesToRecover.poll(); + if (snapshotFileToRecover == null) { + return; + } + + final ListenableFuture requestFuture = new ListenableFuture<>(); + try { + cancellableThreads.checkForCancel(); + + ActionListener sendRequestListener = new ActionListener() { + @Override + public void onResponse(Void unused) { + onRequestCompletion(snapshotFileToRecover.metadata(), null); + } + + @Override + public void onFailure(Exception e) { + logger.warn(new ParameterizedMessage("failed to recover file [{}] from snapshot, " + + "will recover from primary instead", snapshotFileToRecover.metadata()), e); + onRequestCompletion(snapshotFileToRecover.metadata(), e); + } + }; + requestFuture.addListener(sendRequestListener); + + trackOutstandingRequest(requestFuture); + recoveryTarget.restoreFileFromSnapshot( + snapshotFilesToRecover.getRepository(), + snapshotFilesToRecover.getIndexId(), + snapshotFileToRecover, + ActionListener.runBefore(requestFuture, () -> unTrackOutstandingRequest(requestFuture)) + ); + } catch (CancellableThreads.ExecutionCancelledException e) { + onCancellation(e); + } catch (Exception e) { + unTrackOutstandingRequest(requestFuture); + onRequestCompletion(snapshotFileToRecover.metadata(), e); + } + } + + void onCancellation(Exception e) { + if (cancelled.compareAndSet(false, true)) { + pendingSnapshotFilesToRecover.clear(); + notifyFailureOnceAllOutstandingRequestAreDone(e); + } + } + + void onRequestCompletion(StoreFileMetadata storeFileMetadata, @Nullable Exception exception) { + if (cancelled.get()) { + return; + } + + if (exception != null) { + addFileFailedToRecoverFromSnapshot(storeFileMetadata); + } + + if (countDown.countDown()) { + final List failedToRecoverFromSnapshotFiles = getFilesFailedToRecoverFromSnapshot(); + listener.onResponse(failedToRecoverFromSnapshotFiles); + } else { + sendRequest(); + } + } + + synchronized void addFileFailedToRecoverFromSnapshot(StoreFileMetadata storeFileMetadata) { + if (filesFailedToDownloadFromSnapshot == null) { + filesFailedToDownloadFromSnapshot = new ArrayList<>(); + } + filesFailedToDownloadFromSnapshot.add(storeFileMetadata); + } + + synchronized List getFilesFailedToRecoverFromSnapshot() { + if (filesFailedToDownloadFromSnapshot == null) { + return Collections.emptyList(); + } + return filesFailedToDownloadFromSnapshot; + } + + private void trackOutstandingRequest(ListenableFuture future) { + boolean cancelled; + synchronized (outstandingRequests) { + cancelled = cancellableThreads.isCancelled(); + if (cancelled == false) { + outstandingRequests.add(future); + } + } + if (cancelled) { + cancellableThreads.checkForCancel(); + } + } + + private void unTrackOutstandingRequest(ListenableFuture future) { + synchronized (outstandingRequests) { + outstandingRequests.remove(future); + } + } + + private void notifyFailureOnceAllOutstandingRequestAreDone(Exception e) { + assert cancelled.get(); + + final Set> pendingRequests; + synchronized (outstandingRequests) { + pendingRequests = new HashSet<>(outstandingRequests); + } + + if (pendingRequests.isEmpty()) { + listener.onFailure(e); + return; + } + // The recovery was cancelled so from this point onwards outstandingRequests won't track + // new requests and therefore we can safely use to wait until all the pending requests complete + // to notify the listener about the cancellation + final CountDown pendingRequestsCountDown = new CountDown(pendingRequests.size()); + for (ListenableFuture outstandingFuture : pendingRequests) { + outstandingFuture.addListener(ActionListener.wrap(() -> { + if (pendingRequestsCountDown.countDown()) { + listener.onFailure(e); + } + })); + } + } + } + void createRetentionLease(final long startingSeqNo, ActionListener listener) { runUnderPrimaryPermit(() -> { // Clone the peer recovery retention lease belonging to the source shard. We are retaining history between the the local diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 45071a829fa85..965f58c883411 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -640,6 +640,12 @@ void addRecoveredBytes(long bytes) { recovered += bytes; } + void resetRecoveredBytes() { + assert reused == false : "file is marked as reused, can't update recovered bytes"; + // TODO: change this once we keep track of recovered data broke down by snapshot/primary + recovered = 0; + } + /** * file name * */ @@ -768,6 +774,12 @@ public void addRecoveredBytesToFile(String name, long bytes) { file.addRecoveredBytes(bytes); } + public void resetRecoveredBytesOfFile(String name) { + FileDetail file = fileDetails.get(name); + assert file != null : "file [" + name + "] hasn't been reported"; + file.resetRecoveredBytes(); + } + public FileDetail get(String name) { return fileDetails.get(name); } @@ -852,6 +864,10 @@ public synchronized void addRecoveredBytesToFile(String name, long bytes) { fileDetails.addRecoveredBytesToFile(name, bytes); } + public synchronized void resetRecoveredBytesOfFile(String name) { + fileDetails.resetRecoveredBytesOfFile(name); + } + public synchronized void addSourceThrottling(long timeInNanos) { if (sourceThrottlingInNanos == UNKNOWN) { sourceThrottlingInNanos = timeInNanos; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 720d8f4e9e2b3..f54cbbf627a37 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -9,6 +9,7 @@ package org.elasticsearch.indices.recovery; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; @@ -19,14 +20,14 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.core.Nullable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.ReleasableBytesReference; -import org.elasticsearch.core.Releasable; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Releasable; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.MapperException; import org.elasticsearch.index.seqno.ReplicationTracker; @@ -36,11 +37,14 @@ import org.elasticsearch.index.shard.IndexShardNotRecoveringException; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.repositories.IndexId; import java.io.IOException; +import java.io.InputStream; import java.nio.file.Path; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -63,6 +67,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget private final long recoveryId; private final IndexShard indexShard; private final DiscoveryNode sourceNode; + private final SnapshotFilesProvider snapshotFilesProvider; private final MultiFileWriter multiFileWriter; private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker(); private final Store store; @@ -87,7 +92,10 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget * @param sourceNode source node of the recovery where we recover from * @param listener called when recovery is completed/failed */ - public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener) { + public RecoveryTarget(IndexShard indexShard, + DiscoveryNode sourceNode, + SnapshotFilesProvider snapshotFilesProvider, + PeerRecoveryTargetService.RecoveryListener listener) { super("recovery_status"); this.cancellableThreads = new CancellableThreads(); this.recoveryId = idGenerator.incrementAndGet(); @@ -95,6 +103,7 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecov this.logger = Loggers.getLogger(getClass(), indexShard.shardId()); this.indexShard = indexShard; this.sourceNode = sourceNode; + this.snapshotFilesProvider = snapshotFilesProvider; this.shardId = indexShard.shardId(); final String tempFilePrefix = RECOVERY_PREFIX + UUIDs.randomBase64UUID() + "."; this.multiFileWriter = new MultiFileWriter(indexShard.store(), indexShard.recoveryState().getIndex(), tempFilePrefix, logger, @@ -111,7 +120,7 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecov * @return a copy of this recovery target */ public RecoveryTarget retryCopy() { - return new RecoveryTarget(indexShard, sourceNode, listener); + return new RecoveryTarget(indexShard, sourceNode, snapshotFilesProvider, listener); } @Nullable @@ -498,6 +507,30 @@ public void writeFileChunk(StoreFileMetadata fileMetadata, long position, Releas } } + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo fileInfo, + ActionListener listener) { + try (InputStream inputStream = + snapshotFilesProvider.getInputStreamForSnapshotFile(repository, indexId, shardId, fileInfo, this::registerThrottleTime)) { + StoreFileMetadata metadata = fileInfo.metadata(); + int readSnapshotFileBufferSize = snapshotFilesProvider.getReadSnapshotFileBufferSizeForRepo(repository); + multiFileWriter.writeFile(metadata, readSnapshotFileBufferSize, inputStream); + listener.onResponse(null); + } catch (Exception e) { + logger.debug( + new ParameterizedMessage("Unable to recover snapshot file {} from repository {}", fileInfo, repository), e + ); + listener.onFailure(e); + } + } + + private void registerThrottleTime(long throttleTimeInNanos) { + state().getIndex().addTargetThrottling(throttleTimeInNanos); + indexShard.recoveryStats().addThrottleTime(throttleTimeInNanos); + } + /** Get a temporary name for the provided file name. */ public String getTempNameForFile(String origFile) { return multiFileWriter.getTempNameForFile(origFile); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index 37380cd76cf62..a25aafe06ad69 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -11,9 +11,11 @@ import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeases; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.repositories.IndexId; import java.util.List; @@ -92,6 +94,17 @@ void receiveFileInfo(List phase1FileNames, */ void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetadata, ActionListener listener); + /** + * Restores a snapshot file in the target store + * @param repository the repository to fetch the snapshot file + * @param indexId the repository index id that identifies the shard index + * @param snapshotFile the actual snapshot file to download + */ + void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener); + /** writes a partial file chunk to the target store */ void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 2f325581dbd1a..fb6f2a38a0786 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -30,9 +30,11 @@ import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.repositories.IndexId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.RemoteTransportException; @@ -170,6 +172,20 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada executeRetryableAction(action, request, TransportRequestOptions.EMPTY, responseListener, reader); } + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + final String action = PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT; + final long requestSeqNo = requestSeqNoGenerator.getAndIncrement(); + final RecoverySnapshotFileRequest request = + new RecoverySnapshotFileRequest(recoveryId, requestSeqNo, shardId, repository, indexId, snapshotFile); + final Writeable.Reader reader = in -> TransportResponse.Empty.INSTANCE; + final ActionListener responseListener = listener.map(r -> null); + executeRetryableAction(action, request, TransportRequestOptions.EMPTY, responseListener, reader); + } + @Override public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/SnapshotFilesProvider.java b/server/src/main/java/org/elasticsearch/indices/recovery/SnapshotFilesProvider.java new file mode 100644 index 0000000000000..610db58fe7872 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/indices/recovery/SnapshotFilesProvider.java @@ -0,0 +1,61 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0 and the Server Side Public License, v 1; you may not use this file except + * in compliance with, at your election, the Elastic License 2.0 or the Server + * Side Public License, v 1. + */ + +package org.elasticsearch.indices.recovery; + +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.blobstore.BlobContainer; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.snapshots.blobstore.SlicedInputStream; +import org.elasticsearch.index.store.StoreFileMetadata; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoriesService; +import org.elasticsearch.repositories.blobstore.BlobStoreRepository; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Objects; +import java.util.function.Consumer; + +public class SnapshotFilesProvider { + private final RepositoriesService repositoriesService; + + public SnapshotFilesProvider(RepositoriesService repositoriesService) { + this.repositoriesService = Objects.requireNonNull(repositoriesService); + } + + public InputStream getInputStreamForSnapshotFile(String repositoryName, + IndexId indexId, + ShardId shardId, + BlobStoreIndexShardSnapshot.FileInfo fileInfo, + Consumer rateLimiterListener) { + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repositoriesService.repository(repositoryName); + StoreFileMetadata storeFileMetadata = fileInfo.metadata(); + final InputStream inputStream; + if (storeFileMetadata.hashEqualsContents()) { + BytesRef content = storeFileMetadata.hash(); + inputStream = new ByteArrayInputStream(content.bytes, content.offset, content.length); + } else { + BlobContainer container = blobStoreRepository.shardContainer(indexId, shardId.id()); + inputStream = new SlicedInputStream(fileInfo.numberOfParts()) { + @Override + protected InputStream openSlice(int slice) throws IOException { + return container.readBlob(fileInfo.partName(slice)); + } + }; + } + return blobStoreRepository.maybeRateLimitRestores(inputStream, rateLimiterListener::accept); + } + + public int getReadSnapshotFileBufferSizeForRepo(String repository) { + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repositoriesService.repository(repository); + return blobStoreRepository.getReadBufferSizeInBytes(); + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardRecoveryPlan.java b/server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardRecoveryPlan.java index 2bcd8e0dda102..108b6866139a7 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardRecoveryPlan.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/plan/ShardRecoveryPlan.java @@ -130,6 +130,10 @@ public boolean isEmpty() { return snapshotFiles.isEmpty(); } + public List getSnapshotFiles() { + return snapshotFiles; + } + @Override public Iterator iterator() { return snapshotFiles.iterator(); diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index deec3eefe5562..5fe91e029449c 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -116,7 +116,10 @@ import org.elasticsearch.indices.recovery.PeerRecoverySourceService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoverySettings; -import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService; +import org.elasticsearch.indices.recovery.SnapshotFilesProvider; +import org.elasticsearch.indices.recovery.plan.RecoveryPlannerService; +import org.elasticsearch.indices.recovery.plan.ShardSnapshotsService; +import org.elasticsearch.indices.recovery.plan.SnapshotsRecoveryPlannerService; import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.ingest.IngestService; import org.elasticsearch.monitor.MonitorService; @@ -750,10 +753,18 @@ protected Node(final Environment initialEnvironment, b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery()); { processRecoverySettings(settingsModule.getClusterSettings(), recoverySettings); + final ShardSnapshotsService shardSnapshotsService = new ShardSnapshotsService(client, + repositoryService, + threadPool, + clusterService + ); + final RecoveryPlannerService recoveryPlannerService = new SnapshotsRecoveryPlannerService(shardSnapshotsService); + final SnapshotFilesProvider snapshotFilesProvider = + new SnapshotFilesProvider(repositoryService); b.bind(PeerRecoverySourceService.class).toInstance(new PeerRecoverySourceService(transportService, - indicesService, recoverySettings, SourceOnlyRecoveryPlannerService.INSTANCE)); + indicesService, recoverySettings, recoveryPlannerService)); b.bind(PeerRecoveryTargetService.class).toInstance(new PeerRecoveryTargetService(threadPool, - transportService, recoverySettings, clusterService)); + transportService, recoverySettings, clusterService, snapshotFilesProvider)); } b.bind(HttpServerTransport.class).toInstance(httpServerTransport); pluginComponents.forEach(p -> { 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 f4cf074f3126c..ba15e98bed84f 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -3477,6 +3477,10 @@ public boolean hasAtomicOverwrites() { return true; } + public int getReadBufferSizeInBytes() { + return bufferSize; + } + /** * The result of removing a snapshot from a shard folder in the repository. */ diff --git a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java index ef123cbd971ac..f3af3f44520e5 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/IndexLevelReplicationTests.java @@ -22,9 +22,9 @@ import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.TimeValue; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; @@ -117,7 +117,7 @@ public void run() { thread.start(); IndexShard replica = shards.addReplica(); Future future = shards.asyncRecoverReplica(replica, - (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) { + (indexShard, node) -> new RecoveryTarget(indexShard, node, null, recoveryListener) { @Override public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetadata, ActionListener listener) { @@ -194,7 +194,7 @@ public IndexResult index(Index op) throws IOException { thread.start(); IndexShard replica = shards.addReplica(); Future fut = shards.asyncRecoverReplica(replica, - (shard, node) -> new RecoveryTarget(shard, node, recoveryListener) { + (shard, node) -> new RecoveryTarget(shard, node, null, recoveryListener) { @Override public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { try { diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 58513d5667540..cd95e3355ac9c 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -23,11 +23,11 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.core.Releasable; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; +import org.elasticsearch.core.Releasable; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.engine.DocIdSeqNoAndSource; @@ -492,7 +492,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { AtomicBoolean recoveryDone = new AtomicBoolean(false); final Future recoveryFuture = shards.asyncRecoverReplica(newReplica, (indexShard, node) -> { recoveryStart.countDown(); - return new RecoveryTarget(indexShard, node, recoveryListener) { + return new RecoveryTarget(indexShard, node, null, recoveryListener) { @Override public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionListener listener) { recoveryDone.set(true); @@ -547,7 +547,7 @@ protected EngineFactory getEngineFactory(final ShardRouting routing) { final IndexShard replica = shards.addReplica(); final Future recoveryFuture = shards.asyncRecoverReplica( replica, - (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) { + (indexShard, node) -> new RecoveryTarget(indexShard, node, null, recoveryListener) { @Override public void indexTranslogOperations( final List operations, @@ -812,7 +812,7 @@ public static class BlockingTarget extends RecoveryTarget { public BlockingTarget(RecoveryState.Stage stageToBlock, CountDownLatch recoveryBlocked, CountDownLatch releaseRecovery, IndexShard shard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener, Logger logger) { - super(shard, sourceNode, listener); + super(shard, sourceNode, null, listener); this.recoveryBlocked = recoveryBlocked; this.releaseRecovery = releaseRecovery; this.stageToBlock = stageToBlock; diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 23e18e3a84961..e0bfd7b974022 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2583,7 +2583,7 @@ public void testTranslogRecoverySyncsTranslog() throws IOException { indexDoc(primary, "_doc", "0", "{\"foo\" : \"bar\"}"); IndexShard replica = newShard(primary.shardId(), false, "n2", metadata, null); recoverReplica(replica, primary, (shard, discoveryNode) -> - new RecoveryTarget(shard, discoveryNode, recoveryListener) { + new RecoveryTarget(shard, discoveryNode, null, recoveryListener) { @Override public void indexTranslogOperations( final List operations, @@ -2690,7 +2690,7 @@ public void testShardActiveDuringPeerRecovery() throws IOException { // Shard is still inactive since we haven't started recovering yet assertFalse(replica.isActive()); recoverReplica(replica, primary, (shard, discoveryNode) -> - new RecoveryTarget(shard, discoveryNode, recoveryListener) { + new RecoveryTarget(shard, discoveryNode, null, recoveryListener) { @Override public void indexTranslogOperations( final List operations, @@ -2746,7 +2746,7 @@ public void testRefreshListenersDuringPeerRecovery() throws IOException { replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode)); assertListenerCalled.accept(replica); recoverReplica(replica, primary, (shard, discoveryNode) -> - new RecoveryTarget(shard, discoveryNode, recoveryListener) { + new RecoveryTarget(shard, discoveryNode, null, recoveryListener) { // we're only checking that listeners are called when the engine is open, before there is no point @Override public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index d94b456cda28a..be40b98d7fc1d 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.shard.PrimaryReplicaSyncer; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; +import org.elasticsearch.indices.recovery.SnapshotFilesProvider; import org.elasticsearch.repositories.RepositoriesService; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -486,7 +487,7 @@ private IndicesClusterStateService createIndicesClusterStateService(DiscoveryNod final RepositoriesService repositoriesService = new RepositoriesService(settings, clusterService, transportService, Collections.emptyMap(), Collections.emptyMap(), threadPool); final PeerRecoveryTargetService recoveryTargetService = new PeerRecoveryTargetService(threadPool, - transportService, null, clusterService); + transportService, null, clusterService, mock(SnapshotFilesProvider.class)); final ShardStateAction shardStateAction = mock(ShardStateAction.class); final PrimaryReplicaSyncer primaryReplicaSyncer = mock(PrimaryReplicaSyncer.class); return new IndicesClusterStateService( diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 5a5d2edf8c780..ee3371498a98b 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -8,8 +8,12 @@ package org.elasticsearch.indices.recovery; +import org.apache.lucene.codecs.CodecUtil; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.OutputStreamIndexOutput; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -23,8 +27,12 @@ import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.ReleasableBytesReference; +import org.elasticsearch.common.lucene.store.ByteArrayIndexInput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.core.Tuple; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.NoOpEngine; import org.elasticsearch.index.mapper.SourceToParse; @@ -32,27 +40,42 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.repositories.RepositoriesService; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.ExecutionException; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.LongStream; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Mockito.mock; public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { + private static final ByteSizeValue SNAPSHOT_FILE_PART_SIZE = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); public void testWriteFileChunksConcurrently() throws Exception { IndexShard sourceShard = newStartedShard(true); @@ -70,7 +93,7 @@ public void testWriteFileChunksConcurrently() throws Exception { final DiscoveryNode pNode = getFakeDiscoNode(sourceShard.routingEntry().currentNodeId()); final DiscoveryNode rNode = getFakeDiscoNode(targetShard.routingEntry().currentNodeId()); targetShard.markAsRecovering("test-peer-recovery", new RecoveryState(targetShard.routingEntry(), rNode, pNode)); - final RecoveryTarget recoveryTarget = new RecoveryTarget(targetShard, null, null); + final RecoveryTarget recoveryTarget = new RecoveryTarget(targetShard, null, null, null); final PlainActionFuture receiveFileInfoFuture = new PlainActionFuture<>(); recoveryTarget.receiveFileInfo( mdFiles.stream().map(StoreFileMetadata::name).collect(Collectors.toList()), @@ -271,7 +294,7 @@ public void testResetStartingSeqNoIfLastCommitCorrupted() throws Exception { shard.prepareForIndexRecovery(); long startingSeqNo = shard.recoverLocallyUpToGlobalCheckpoint(); shard.store().markStoreCorrupted(new IOException("simulated")); - RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null); + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null, null); StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(logger, rNode, recoveryTarget, startingSeqNo); assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO)); assertThat(request.metadataSnapshot().size(), equalTo(0)); @@ -298,7 +321,7 @@ public void testResetStartRequestIfTranslogIsCorrupted() throws Exception { shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE)); shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode)); shard.prepareForIndexRecovery(); - RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null); + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null, null); StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest( logger, rNode, recoveryTarget, randomNonNegativeLong()); assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO)); @@ -306,4 +329,286 @@ public void testResetStartRequestIfTranslogIsCorrupted() throws Exception { recoveryTarget.decRef(); closeShards(shard); } + + public void testSnapshotFileWrite() throws Exception { + DiscoveryNode pNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + DiscoveryNode rNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + + IndexShard shard = newShard(false); + shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE)); + shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode)); + shard.prepareForIndexRecovery(); + + RecoveryState.Index recoveryStateIndex = shard.recoveryState().getIndex(); + + Directory directory = shard.store().directory(); + + String fileName = randomAlphaOfLength(10); + Tuple storeFileMetadataAndData = createStoreFileMetadataWithRandomContent(fileName); + StoreFileMetadata storeFileMetadata = storeFileMetadataAndData.v1(); + byte[] fileData = storeFileMetadataAndData.v2(); + + String repositoryName = "repo"; + IndexId indexId = new IndexId("index", "uuid"); + ShardId shardId = shard.shardId(); + BlobStoreIndexShardSnapshot.FileInfo fileInfo = + new BlobStoreIndexShardSnapshot.FileInfo("name", storeFileMetadata, SNAPSHOT_FILE_PART_SIZE); + + SnapshotFilesProvider snapshotFilesProvider = new SnapshotFilesProvider(mock(RepositoriesService.class)) { + @Override + public InputStream getInputStreamForSnapshotFile(String requestedRepositoryName, + IndexId requestedIndexId, + ShardId requestedShardId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo, + Consumer rateLimiterListener) { + assertThat(requestedRepositoryName, equalTo(repositoryName)); + assertThat(requestedIndexId, equalTo(indexId)); + assertThat(requestedShardId, equalTo(shardId)); + + assertThat(snapshotFileInfo.name(), equalTo(fileInfo.name())); + assertThat(snapshotFileInfo.metadata().isSame(storeFileMetadata), equalTo(true)); + + return new ByteArrayInputStream(fileData); + } + + @Override + public int getReadSnapshotFileBufferSizeForRepo(String repository) { + return (int) new ByteSizeValue(128, ByteSizeUnit.KB).getBytes(); + } + }; + + recoveryStateIndex.addFileDetail(storeFileMetadata.name(), storeFileMetadata.length(), false); + recoveryStateIndex.setFileDetailsComplete(); + + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, snapshotFilesProvider, null); + + PlainActionFuture writeSnapshotFileFuture = PlainActionFuture.newFuture(); + recoveryTarget.restoreFileFromSnapshot(repositoryName, indexId, fileInfo, writeSnapshotFileFuture); + writeSnapshotFileFuture.get(); + + Optional tmpFileName = Arrays.stream(directory.listAll()) + .filter(directoryFile -> directoryFile.endsWith(fileName)) + .findFirst(); + + assertThat(tmpFileName.isPresent(), is(equalTo(true))); + try (IndexInput indexInput = directory.openInput(tmpFileName.get(), IOContext.READONCE)) { + byte[] writtenData = new byte[(int) storeFileMetadata.length()]; + indexInput.readBytes(writtenData, 0, (int) storeFileMetadata.length()); + assertThat(writtenData, is(equalTo(fileData))); + } + + RecoveryState.FileDetail fileDetails = recoveryStateIndex.getFileDetails(storeFileMetadata.name()); + assertThat(fileDetails.recovered(), equalTo(storeFileMetadata.length())); + + recoveryTarget.decRef(); + closeShards(shard); + } + + enum DownloadFileErrorType { + CORRUPTED_FILE, + TRUNCATED_FILE, + LARGER_THAN_EXPECTED_FILE, + FETCH_ERROR + } + + public void testSnapshotFileIsDeletedAfterFailure() throws Exception { + DiscoveryNode pNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + DiscoveryNode rNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + + IndexShard shard = newShard(false); + shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode)); + shard.prepareForIndexRecovery(); + + RecoveryState.Index recoveryStateIndex = shard.recoveryState().getIndex(); + + Directory directory = shard.store().directory(); + String[] filesBeforeRestoringSnapshotFile = directory.listAll(); + + String fileName = randomAlphaOfLength(10); + Tuple storeFileMetadataAndData = createStoreFileMetadataWithRandomContent(fileName); + StoreFileMetadata storeFileMetadata = storeFileMetadataAndData.v1(); + byte[] fileData = storeFileMetadataAndData.v2(); + final DownloadFileErrorType downloadFileErrorType = randomFrom(DownloadFileErrorType.values()); + + SnapshotFilesProvider snapshotFilesProvider = new SnapshotFilesProvider(mock(RepositoriesService.class)) { + @Override + public InputStream getInputStreamForSnapshotFile(String requestedRepositoryName, + IndexId requestedIndexId, + ShardId requestedShardId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo, + Consumer rateLimiterListener) { + switch (downloadFileErrorType) { + case CORRUPTED_FILE: + byte[] fileDataCopy = new byte[fileData.length]; + System.arraycopy(fileData, 0, fileDataCopy, 0, fileData.length); + // Corrupt the file + for (int i = 0; i < randomIntBetween(1, fileDataCopy.length); i++) { + fileDataCopy[i] ^= 0xFF; + } + return new ByteArrayInputStream(fileDataCopy); + case TRUNCATED_FILE: + final int truncatedFileLength = fileData.length / 2; + byte[] truncatedCopy = new byte[truncatedFileLength]; + System.arraycopy(fileData, 0, truncatedCopy, 0, truncatedFileLength); + return new ByteArrayInputStream(truncatedCopy); + case LARGER_THAN_EXPECTED_FILE: + byte[] largerData = new byte[fileData.length + randomIntBetween(1, 250)]; + System.arraycopy(fileData, 0, largerData, 0, fileData.length); + for (int i = fileData.length; i < largerData.length; i++) { + largerData[i] = randomByte(); + } + return new ByteArrayInputStream(largerData); + case FETCH_ERROR: + throw new RuntimeException("Unexpected error"); + default: + throw new IllegalStateException("Unexpected value: " + downloadFileErrorType); + } + } + + @Override + public int getReadSnapshotFileBufferSizeForRepo(String repository) { + return (int) new ByteSizeValue(128, ByteSizeUnit.KB).getBytes(); + } + }; + + recoveryStateIndex.addFileDetail(storeFileMetadata.name(), storeFileMetadata.length(), false); + recoveryStateIndex.setFileDetailsComplete(); + + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, snapshotFilesProvider, null); + + String repositoryName = "repo"; + IndexId indexId = new IndexId("index", "uuid"); + BlobStoreIndexShardSnapshot.FileInfo fileInfo = + new BlobStoreIndexShardSnapshot.FileInfo("name", storeFileMetadata, SNAPSHOT_FILE_PART_SIZE); + + PlainActionFuture writeSnapshotFileFuture = PlainActionFuture.newFuture(); + recoveryTarget.restoreFileFromSnapshot(repositoryName, indexId, fileInfo, writeSnapshotFileFuture); + ExecutionException executionException = expectThrows(ExecutionException.class, writeSnapshotFileFuture::get); + + Throwable downloadFileError = executionException.getCause(); + switch (downloadFileErrorType) { + case CORRUPTED_FILE: + case LARGER_THAN_EXPECTED_FILE: + // Files larger than expected are caught by VerifyingIndexInput too + assertThat(downloadFileError, is(instanceOf(CorruptIndexException.class))); + break; + case TRUNCATED_FILE: + assertThat(downloadFileError, is(instanceOf(EOFException.class))); + break; + case FETCH_ERROR: + assertThat(downloadFileError, is(instanceOf(RuntimeException.class))); + break; + default: + throw new IllegalStateException("Unexpected value: " + downloadFileErrorType); + } + + assertThat(filesBeforeRestoringSnapshotFile, equalTo(directory.listAll())); + + RecoveryState.FileDetail fileDetails = recoveryStateIndex.getFileDetails(storeFileMetadata.name()); + assertThat(fileDetails.recovered(), equalTo(0L)); + + // Subsequent writes on the same file can proceed without issues + PlainActionFuture writeChunkFuture = PlainActionFuture.newFuture(); + ReleasableBytesReference bytesRef = ReleasableBytesReference.wrap(new BytesArray(fileData)); + recoveryTarget.writeFileChunk(storeFileMetadata, 0, bytesRef, true, 0, writeChunkFuture); + writeChunkFuture.get(); + + recoveryTarget.decRef(); + closeShards(shard); + } + + public void testSnapshotFileAreDeletedAfterCancel() throws Exception { + DiscoveryNode pNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + DiscoveryNode rNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), + Collections.emptyMap(), Collections.emptySet(), Version.CURRENT); + + IndexShard shard = newShard(false); + shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode)); + shard.prepareForIndexRecovery(); + + RecoveryState.Index recoveryStateIndex = shard.recoveryState().getIndex(); + + Directory directory = shard.store().directory(); + String[] filesBeforeRestoringSnapshotFile = directory.listAll(); + + String fileName = randomAlphaOfLength(10); + Tuple storeFileMetadataAndData = createStoreFileMetadataWithRandomContent(fileName); + StoreFileMetadata storeFileMetadata = storeFileMetadataAndData.v1(); + byte[] fileData = storeFileMetadataAndData.v2(); + + SnapshotFilesProvider snapshotFilesProvider = new SnapshotFilesProvider(mock(RepositoriesService.class)) { + @Override + public InputStream getInputStreamForSnapshotFile(String requestedRepositoryName, + IndexId requestedIndexId, + ShardId requestedShardId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo, + Consumer rateLimiterListener) { + return new ByteArrayInputStream(fileData); + } + + @Override + public int getReadSnapshotFileBufferSizeForRepo(String repository) { + return (int) new ByteSizeValue(128, ByteSizeUnit.KB).getBytes(); + } + }; + + recoveryStateIndex.addFileDetail(storeFileMetadata.name(), storeFileMetadata.length(), false); + recoveryStateIndex.setFileDetailsComplete(); + + RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, snapshotFilesProvider, null); + + String repository = "repo"; + IndexId indexId = new IndexId("index", "uuid"); + BlobStoreIndexShardSnapshot.FileInfo fileInfo = + new BlobStoreIndexShardSnapshot.FileInfo("name", storeFileMetadata, new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)); + + recoveryTarget.incRef(); + + PlainActionFuture writeSnapshotFileFuture = PlainActionFuture.newFuture(); + recoveryTarget.restoreFileFromSnapshot(repository, indexId, fileInfo, writeSnapshotFileFuture); + writeSnapshotFileFuture.get(); + + RecoveryState.FileDetail fileDetails = recoveryStateIndex.getFileDetails(storeFileMetadata.name()); + assertThat(fileDetails.recovered(), equalTo(storeFileMetadata.length())); + + final String[] filesBeforeCancellingRecovery = directory.listAll(); + + recoveryTarget.cancel("This is a test"); + + final String[] filesAfterCancellingRecoveryWithOneOutstandingReference = directory.listAll(); + + // Since there's still one outstanding reference the snapshot file is kept around + assertThat(filesBeforeCancellingRecovery, equalTo(filesAfterCancellingRecoveryWithOneOutstandingReference)); + + recoveryTarget.decRef(); + + // Once the reference is released, the tmp file should be deleted + assertThat(filesBeforeRestoringSnapshotFile, equalTo(directory.listAll())); + + closeShards(shard); + } + + private Tuple createStoreFileMetadataWithRandomContent(String fileName) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + try (OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("test", "file", out, 1024)) { + byte[] buffer = randomByteArrayOfLength(1024); + indexOutput.writeBytes(buffer, buffer.length); + CodecUtil.writeFooter(indexOutput); + } + + byte[] luceneEncodedFileBytes = out.toByteArray(); + long checksum = CodecUtil.retrieveChecksum(new ByteArrayIndexInput("test", luceneEncodedFileBytes)); + + String encodedChecksum = Store.digestToString(checksum); + String writtenBy = org.apache.lucene.util.Version.LATEST.toString(); + return Tuple.tuple( + new StoreFileMetadata(fileName, luceneEncodedFileBytes.length, encodedChecksum, writtenBy), + luceneEncodedFileBytes + ); + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 818c7e7db2be8..3d0cad23d2706 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -44,6 +44,8 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.XContentType; @@ -66,12 +68,14 @@ import org.elasticsearch.index.shard.IndexShardRelocatedException; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.plan.RecoveryPlannerService; import org.elasticsearch.indices.recovery.plan.ShardRecoveryPlan; import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService; +import org.elasticsearch.repositories.IndexId; import org.elasticsearch.test.CorruptionUtils; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; @@ -105,10 +109,12 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.function.IntSupplier; import java.util.stream.Collectors; import java.util.zip.CRC32; +import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.hamcrest.Matchers.containsString; @@ -193,6 +199,8 @@ public void writeFileChunk(StoreFileMetadata md, long position, ReleasableBytesR Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5), between(1, 5), + between(1, 5), + false, recoveryPlannerService); PlainActionFuture sendFilesFuture = new PlainActionFuture<>(); handler.sendFiles(store, metas.toArray(new StoreFileMetadata[0]), () -> 0, sendFilesFuture); @@ -256,9 +264,9 @@ public void indexTranslogOperations(List operations, int tot } }; RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - threadPool, request, fileChunkSizeInBytes, between(1, 10), between(1, 10), recoveryPlannerService); + threadPool, request, fileChunkSizeInBytes, between(1, 10), between(1, 10), between(1, 10), false, recoveryPlannerService); PlainActionFuture future = new PlainActionFuture<>(); - handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), + handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, emptyList()), randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1); RecoverySourceHandler.SendSnapshotResult result = future.actionGet(); @@ -298,11 +306,11 @@ public void indexTranslogOperations(List operations, int tot } }; RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), - threadPool, request, fileChunkSizeInBytes, between(1, 10), between(1, 10), recoveryPlannerService); + threadPool, request, fileChunkSizeInBytes, between(1, 10), between(1, 10), between(1, 10), false, recoveryPlannerService); PlainActionFuture future = new PlainActionFuture<>(); final long startingSeqNo = randomLongBetween(0, ops.size() - 1L); final long endingSeqNo = randomLongBetween(startingSeqNo, ops.size() - 1L); - handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(ops, Collections.emptyList()), + handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(ops, emptyList()), randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future); if (wasFailed.get()) { final RecoveryEngineException error = expectThrows(RecoveryEngineException.class, future::actionGet); @@ -352,7 +360,8 @@ public void indexTranslogOperations(List operations, int rec List skipOperations = randomSubsetOf(operations); Translog.Snapshot snapshot = newTranslogSnapshot(operations, skipOperations); RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(target, recoveryExecutor), - threadPool, getStartRecoveryRequest(), between(1, 10 * 1024), between(1, 5), between(1, 5), recoveryPlannerService); + threadPool, getStartRecoveryRequest(), between(1, 10 * 1024), between(1, 5), between(1, 5), between(1, 5), false, + recoveryPlannerService); handler.phase2(startingSeqNo, endingSeqNo, snapshot, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases, mappingVersion, sendFuture); RecoverySourceHandler.SendSnapshotResult sendSnapshotResult = sendFuture.actionGet(); @@ -428,7 +437,8 @@ public void writeFileChunk(StoreFileMetadata md, long position, ReleasableBytesR } }; RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, - request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8), between(1, 8), recoveryPlannerService) { + request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8), between(1, 8), between(1, 8), false, + recoveryPlannerService) { @Override protected void failEngine(IOException cause) { assertFalse(failedEngine.get()); @@ -485,7 +495,8 @@ public void writeFileChunk(StoreFileMetadata md, long position, ReleasableBytesR } }; RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool, - request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10), between(1, 4), recoveryPlannerService) { + request, Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10), between(1, 4), between(1, 4), false, + recoveryPlannerService) { @Override protected void failEngine(IOException cause) { assertFalse(failedEngine.get()); @@ -536,13 +547,16 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean(); final AtomicBoolean phase2Called = new AtomicBoolean(); final RecoverySourceHandler handler = new RecoverySourceHandler( - shard, - mock(RecoveryTargetHandler.class), - threadPool, - request, - Math.toIntExact(recoverySettings.getChunkSize().getBytes()), - between(1, 8), between(1, 8), - recoveryPlannerService) { + shard, + mock(RecoveryTargetHandler.class), + threadPool, + request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + between(1, 8), + between(1, 8), + between(1, 5), + false, + recoveryPlannerService) { @Override void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, ActionListener listener) { @@ -620,7 +634,7 @@ public void writeFileChunk(StoreFileMetadata md, long position, ReleasableBytesR final int maxConcurrentChunks = between(1, 8); final int chunkSize = between(1, 32); final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, threadPool, getStartRecoveryRequest(), - chunkSize, maxConcurrentChunks, between(1, 10), recoveryPlannerService); + chunkSize, maxConcurrentChunks, between(1, 10), between(1, 5), false, recoveryPlannerService); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); @@ -678,7 +692,8 @@ public void writeFileChunk(StoreFileMetadata md, long position, ReleasableBytesR final int maxConcurrentChunks = between(1, 4); final int chunkSize = between(1, 16); final RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(recoveryTarget, recoveryExecutor), - threadPool, getStartRecoveryRequest(), chunkSize, maxConcurrentChunks, between(1, 5), recoveryPlannerService); + threadPool, getStartRecoveryRequest(), chunkSize, maxConcurrentChunks, between(1, 5), between(1, 5), + false, recoveryPlannerService); Store store = newStore(createTempDir(), false); List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); @@ -758,8 +773,16 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada } }; final StartRecoveryRequest startRecoveryRequest = getStartRecoveryRequest(); - final RecoverySourceHandler handler = new RecoverySourceHandler( - shard, recoveryTarget, threadPool, startRecoveryRequest, between(1, 16), between(1, 4), between(1, 4), recoveryPlannerService) { + final RecoverySourceHandler handler = new RecoverySourceHandler(shard, + recoveryTarget, + threadPool, + startRecoveryRequest, + between(1, 16), + between(1, 4), + between(1, 4), + between(1, 4), + false, + recoveryPlannerService) { @Override void createRetentionLease(long startingSeqNo, ActionListener listener) { final String leaseId = ReplicationTracker.getPeerRecoveryRetentionLeaseId(startRecoveryRequest.targetNode().getId()); @@ -795,6 +818,8 @@ public void testVerifySeqNoStatsWhenRecoverWithSyncId() throws Exception { between(1, 16), between(1, 4), between(1, 4), + between(1, 4), + false, recoveryPlannerService); String syncId = UUIDs.randomBase64UUID(); @@ -833,35 +858,7 @@ public void testRecoveryPlannerServiceIsUsed() throws Exception { writer.close(); when(shard.state()).thenReturn(IndexShardState.STARTED); - TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { - @Override - public void receiveFileInfo(List phase1FileNames, - List phase1FileSizes, - List phase1ExistingFileNames, - List phase1ExistingFileSizes, - int totalTranslogOps, - ActionListener listener) { - listener.onResponse(null); - } - - @Override - public void cleanFiles(int totalTranslogOps, - long globalCheckpoint, - Store.MetadataSnapshot sourceMetadata, - ActionListener listener) { - listener.onResponse(null); - } - - @Override - public void writeFileChunk(StoreFileMetadata fileMetadata, - long position, - ReleasableBytesReference content, - boolean lastChunk, - int totalTranslogOps, - ActionListener listener) { - listener.onResponse(null); - } - }; + TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler(); AtomicReference computedRecoveryPlanRef = new AtomicReference<>(); RecoverySourceHandler handler = new RecoverySourceHandler( shard, @@ -871,6 +868,8 @@ public void writeFileChunk(StoreFileMetadata fileMetadata, between(1, 16), between(1, 4), between(1, 4), + between(1, 4), + true, recoveryPlannerService ) { @Override @@ -889,10 +888,7 @@ void recoverFilesFromSourceAndSnapshot(ShardRecoveryPlan shardRecoveryPlan, }; PlainActionFuture phase1Listener = PlainActionFuture.newFuture(); IndexCommit indexCommit = DirectoryReader.listCommits(dir).get(0); - handler.phase1(indexCommit, - 0, - () -> 0, - phase1Listener); + handler.phase1(indexCommit, 0, () -> 0, phase1Listener); phase1Listener.get(); ShardRecoveryPlan computedRecoveryPlan = computedRecoveryPlanRef.get(); @@ -906,6 +902,361 @@ void recoverFilesFromSourceAndSnapshot(ShardRecoveryPlan shardRecoveryPlan, } } + public void testSnapshotFilesThatFailToDownloadAreSentFromSource() throws Exception { + try (Store store = newStore(createTempDir("source"), false)) { + IndexShard shard = mock(IndexShard.class); + when(shard.store()).thenReturn(store); + when(shard.state()).thenReturn(IndexShardState.STARTED); + + final ShardRecoveryPlan shardRecoveryPlan = createShardRecoveryPlan(store, randomIntBetween(10, 20), randomIntBetween(10, 20)); + + final ShardRecoveryPlan.SnapshotFilesToRecover snapshotFilesToRecover = shardRecoveryPlan.getSnapshotFilesToRecover(); + final List fileNamesToBeRecoveredFromSnapshot = snapshotFilesToRecover.getSnapshotFiles() + .stream() + .map(fileInfo -> fileInfo.metadata().name()) + .collect(Collectors.toList()); + + final List sourceFilesToRecover = + shardRecoveryPlan.getSourceFilesToRecover().stream().map(StoreFileMetadata::name).collect(Collectors.toList()); + + Set filesFailedToDownload = Collections.synchronizedSet(new HashSet<>()); + Set filesRecoveredFromSource = Collections.synchronizedSet(new HashSet<>()); + Set filesRecoveredFromSnapshot = Collections.synchronizedSet(new HashSet<>()); + TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() { + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + assertThat(repository, is(equalTo(snapshotFilesToRecover.getRepository()))); + assertThat(indexId, is(equalTo(snapshotFilesToRecover.getIndexId()))); + assertThat(containsSnapshotFile(snapshotFilesToRecover, snapshotFile), is(equalTo(true))); + String fileName = snapshotFile.metadata().name(); + + if (randomBoolean()) { + filesFailedToDownload.add(fileName); + if (randomBoolean()) { + listener.onFailure(randomFrom(new IOException("Failure"), new CorruptIndexException("idx", ""))); + } else { + throw new RuntimeException("Unexpected exception"); + } + } else { + filesRecoveredFromSnapshot.add(fileName); + listener.onResponse(null); + } + } + + @Override + public void writeFileChunk(StoreFileMetadata fileMetadata, + long position, + ReleasableBytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener) { + filesRecoveredFromSource.add(fileMetadata.name()); + listener.onResponse(null); + } + }; + + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + recoveryTarget, + threadPool, + getStartRecoveryRequest(), + between(1, 16), + between(1, 4), + between(1, 4), + between(1, 4), + true, + recoveryPlannerService) { + @Override + void createRetentionLease(long startingSeqNo, ActionListener listener) { + listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test")); + } + }; + + PlainActionFuture future = PlainActionFuture.newFuture(); + handler.recoverFilesFromSourceAndSnapshot(shardRecoveryPlan, + store, + mock(StopWatch.class), + future + ); + future.actionGet(); + + + Set expectedFilesRecoveredFromSource = new HashSet<>(); + expectedFilesRecoveredFromSource.addAll(sourceFilesToRecover); + expectedFilesRecoveredFromSource.addAll(filesFailedToDownload); + assertThat(filesRecoveredFromSource, is(equalTo(expectedFilesRecoveredFromSource))); + + assertThat(fileNamesToBeRecoveredFromSnapshot.containsAll(filesRecoveredFromSnapshot), is(equalTo(true))); + } + } + + public void testSnapshotFilesRequestAreSentConcurrently() throws Exception { + try (Store store = newStore(createTempDir("source"), false)) { + IndexShard shard = mock(IndexShard.class); + when(shard.store()).thenReturn(store); + when(shard.state()).thenReturn(IndexShardState.STARTED); + + ShardRecoveryPlan shardRecoveryPlan = createShardRecoveryPlan(store, 0, randomIntBetween(10, 20)); + final int snapshotFileToRecoverCount = shardRecoveryPlan.getSnapshotFilesToRecover().size(); + + AtomicInteger recoverSnapshotFileRequests = new AtomicInteger(); + List unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>(); + TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() { + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + unrespondedRecoverSnapshotFiles.add(new RecoverSnapshotFileResponse(snapshotFile, listener)); + recoverSnapshotFileRequests.incrementAndGet(); + } + + @Override + public void writeFileChunk(StoreFileMetadata fileMetadata, + long position, + ReleasableBytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener) { + assert false : "Unexpected call"; + } + }; + + int maxConcurrentSnapshotFileDownloads = between(1, 4); + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + recoveryTarget, + threadPool, + getStartRecoveryRequest(), + between(1, 16), + between(1, 4), + between(1, 4), + maxConcurrentSnapshotFileDownloads, + true, + null) { + @Override + void createRetentionLease(long startingSeqNo, ActionListener listener) { + listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test")); + } + }; + + PlainActionFuture future = PlainActionFuture.newFuture(); + handler.recoverFilesFromSourceAndSnapshot(shardRecoveryPlan, store, mock(StopWatch.class), future); + + assertBusy(() -> { + assertThat(recoverSnapshotFileRequests.get(), + equalTo(Math.min(snapshotFileToRecoverCount, maxConcurrentSnapshotFileDownloads))); + assertThat(unrespondedRecoverSnapshotFiles, hasSize(recoverSnapshotFileRequests.get())); + }); + + while (recoverSnapshotFileRequests.get() < snapshotFileToRecoverCount || unrespondedRecoverSnapshotFiles.isEmpty() == false) { + List recoverSnapshotFilesToRespond = + randomSubsetOf(between(1, unrespondedRecoverSnapshotFiles.size()), unrespondedRecoverSnapshotFiles); + unrespondedRecoverSnapshotFiles.removeAll(recoverSnapshotFilesToRespond); + + int newRecoverSnapshotFileRequestCount = Math.min( + Math.min(recoverSnapshotFilesToRespond.size(), maxConcurrentSnapshotFileDownloads), + snapshotFileToRecoverCount - recoverSnapshotFileRequests.get() + ); + + int expectedSentRecoverSnapshotFiles = recoverSnapshotFileRequests.get() + newRecoverSnapshotFileRequestCount; + int expectedUnAckedRecoverSnapshotFiles = unrespondedRecoverSnapshotFiles.size() + newRecoverSnapshotFileRequestCount; + recoverSnapshotFilesToRespond.forEach(c -> c.listener.onResponse(null)); + assertBusy(() -> { + assertThat(recoverSnapshotFileRequests.get(), equalTo(expectedSentRecoverSnapshotFiles)); + assertThat(unrespondedRecoverSnapshotFiles, hasSize(expectedUnAckedRecoverSnapshotFiles)); + }); + } + + future.actionGet(); + } + } + + public void testDownloadSnapshotFilesRequestStopAfterCancelling() throws Exception { + try (Store store = newStore(createTempDir("source"), false)) { + IndexShard shard = mock(IndexShard.class); + when(shard.store()).thenReturn(store); + when(shard.state()).thenReturn(IndexShardState.STARTED); + + ShardRecoveryPlan shardRecoveryPlan = createShardRecoveryPlan(store, 0, randomIntBetween(10, 20)); + + CountDownLatch downloadSnapshotFileReceived = new CountDownLatch(1); + List unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>(); + TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() { + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + assert unrespondedRecoverSnapshotFiles.isEmpty(): "Unexpected call"; + + unrespondedRecoverSnapshotFiles.add(new RecoverSnapshotFileResponse(snapshotFile, listener)); + downloadSnapshotFileReceived.countDown(); + } + + @Override + public void writeFileChunk(StoreFileMetadata fileMetadata, + long position, + ReleasableBytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener) { + assert false : "Unexpected call"; + } + }; + + int maxConcurrentSnapshotFileDownloads = 1; + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + recoveryTarget, + threadPool, + getStartRecoveryRequest(), + between(1, 16), + between(1, 4), + between(1, 4), + maxConcurrentSnapshotFileDownloads, + true, + null) { + @Override + void createRetentionLease(long startingSeqNo, ActionListener listener) { + listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test")); + } + }; + + PlainActionFuture future = PlainActionFuture.newFuture(); + handler.recoverFilesFromSourceAndSnapshot(shardRecoveryPlan, store, mock(StopWatch.class), future); + + downloadSnapshotFileReceived.await(); + assertThat(unrespondedRecoverSnapshotFiles.size(), is(equalTo(1))); + + handler.cancel("test"); + + RecoverSnapshotFileResponse recoverSnapshotFileResponse = unrespondedRecoverSnapshotFiles.get(0); + recoverSnapshotFileResponse.listener.onResponse(null); + + expectThrows(Exception.class, future::get); + + assertThat(unrespondedRecoverSnapshotFiles.size(), is(equalTo(1))); + } + } + + public void testWaitsForOutstandingRestoreFileFromSnapshotRequestsToNotifyCancellation() throws Exception { + try (Store store = newStore(createTempDir("source"), false)) { + IndexShard shard = mock(IndexShard.class); + when(shard.store()).thenReturn(store); + when(shard.state()).thenReturn(IndexShardState.STARTED); + + ShardRecoveryPlan shardRecoveryPlan = createShardRecoveryPlan(store, 0, randomIntBetween(10, 20)); + + int maxConcurrentSnapshotFileDownloads = randomIntBetween(2, 4); + CountDownLatch downloadSnapshotFileReceived = new CountDownLatch(maxConcurrentSnapshotFileDownloads); + List unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>(); + TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() { + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + unrespondedRecoverSnapshotFiles.add(new RecoverSnapshotFileResponse(snapshotFile, listener)); + downloadSnapshotFileReceived.countDown(); + } + + @Override + public void writeFileChunk(StoreFileMetadata fileMetadata, + long position, + ReleasableBytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener) { + assert false : "Unexpected call"; + } + }; + + RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + recoveryTarget, + threadPool, + getStartRecoveryRequest(), + between(1, 16), + between(1, 4), + between(1, 4), + maxConcurrentSnapshotFileDownloads, + true, + null) { + @Override + void createRetentionLease(long startingSeqNo, ActionListener listener) { + listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test")); + } + }; + + PlainActionFuture future = PlainActionFuture.newFuture(); + handler.recoverFilesFromSourceAndSnapshot(shardRecoveryPlan, store, mock(StopWatch.class), future); + + downloadSnapshotFileReceived.await(); + assertThat(unrespondedRecoverSnapshotFiles.size(), is(equalTo(maxConcurrentSnapshotFileDownloads))); + + handler.cancel("test"); + + assertThat(future.isDone(), is(equalTo(false))); + for (int i = 0; i < unrespondedRecoverSnapshotFiles.size(); i++) { + RecoverSnapshotFileResponse snapshotFileResponse = unrespondedRecoverSnapshotFiles.get(i); + if (randomBoolean()) { + snapshotFileResponse.listener.onResponse(null); + } else { + snapshotFileResponse.listener.onFailure(new RuntimeException("boom")); + } + + if (i < unrespondedRecoverSnapshotFiles.size() - 1) { + assertThat(future.isDone(), is(equalTo(false))); + } + } + + expectThrows(Exception.class, future::get); + } + } + + private boolean containsSnapshotFile(ShardRecoveryPlan.SnapshotFilesToRecover snapshotFilesToRecover, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile) { + return snapshotFilesToRecover.getSnapshotFiles().stream().anyMatch(f -> f.metadata().isSame(snapshotFile.metadata())); + } + + private ShardRecoveryPlan createShardRecoveryPlan(Store store, int sourceFileCount, int snapshotFileCount) throws Exception { + List sourceFiles = generateFiles(store, snapshotFileCount + sourceFileCount, () -> randomIntBetween(1, 100)); + Store.MetadataSnapshot metadata = new Store.MetadataSnapshot( + sourceFiles.stream().collect(Collectors.toMap(StoreFileMetadata::name, Function.identity())), + emptyMap(), + 0 + ); + + ByteSizeValue partSize = new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES); + + List filesToRecoverFromSource = sourceFiles.subList(0, sourceFileCount); + List filesToRecoverFromSnapshot = sourceFiles.subList(sourceFileCount, sourceFiles.size()); + + List snapshotFiles = new ArrayList<>(snapshotFileCount); + for (StoreFileMetadata storeFileMetadata : filesToRecoverFromSnapshot) { + snapshotFiles.add(new BlobStoreIndexShardSnapshot.FileInfo(storeFileMetadata.name(), storeFileMetadata, partSize)); + } + + IndexId indexId = new IndexId("index", "id"); + String repository = "repo"; + ShardRecoveryPlan.SnapshotFilesToRecover snapshotFilesToRecover = new ShardRecoveryPlan.SnapshotFilesToRecover(indexId, + repository, + snapshotFiles + ); + + return new ShardRecoveryPlan(snapshotFilesToRecover, + filesToRecoverFromSource, + emptyList(), + 0, + 0, + metadata + ); + } + private Store.MetadataSnapshot newMetadataSnapshot(String syncId, String localCheckpoint, String maxSeqNo, int numDocs) { Map userData = new HashMap<>(); userData.put(Engine.SYNC_COMMIT_ID, syncId); @@ -993,12 +1344,47 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada ActionListener listener) { } + @Override + public void restoreFileFromSnapshot(String repository, IndexId indexId, BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + } + @Override public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk, int totalTranslogOps, ActionListener listener) { } } + class Phase1RecoveryTargetHandler extends TestRecoveryTargetHandler { + @Override + public void receiveFileInfo(List phase1FileNames, + List phase1FileSizes, + List phase1ExistingFileNames, + List phase1ExistingFileSizes, + int totalTranslogOps, + ActionListener listener) { + listener.onResponse(null); + } + + @Override + public void writeFileChunk(StoreFileMetadata fileMetadata, + long position, + ReleasableBytesReference content, + boolean lastChunk, + int totalTranslogOps, + ActionListener listener) { + listener.onResponse(null); + } + + @Override + public void cleanFiles(int totalTranslogOps, + long globalCheckpoint, + Store.MetadataSnapshot sourceMetadata, + ActionListener listener) { + listener.onResponse(null); + } + } + private Translog.Snapshot newTranslogSnapshot(List operations, List operationsToSkip) { Iterator iterator = operations.iterator(); return new Translog.Snapshot() { @@ -1053,4 +1439,14 @@ private static List generateOperations(int numOps) { } return operations; } + + static class RecoverSnapshotFileResponse { + final BlobStoreIndexShardSnapshot.FileInfo fileInfo; + final ActionListener listener; + + RecoverSnapshotFileResponse(BlobStoreIndexShardSnapshot.FileInfo fileInfo, ActionListener listener) { + this.fileInfo = fileInfo; + this.listener = listener; + } + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 76c8269b670a0..49b2423fe9cc7 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -9,6 +9,7 @@ package org.elasticsearch.indices.recovery; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; + import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexWriter; @@ -328,7 +329,8 @@ public void testPeerRecoverySendSafeCommitInFileBased() throws Exception { } IndexShard replicaShard = newShard(primaryShard.shardId(), false); updateMappings(replicaShard, primaryShard.indexSettings().getIndexMetadata()); - recoverReplica(replicaShard, primaryShard, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener) { + recoverReplica(replicaShard, primaryShard, + (r, sourceNode) -> new RecoveryTarget(r, sourceNode, null, recoveryListener) { @Override public void prepareForTranslogOperations(int totalTranslogOps, ActionListener listener) { super.prepareForTranslogOperations(totalTranslogOps, listener); @@ -439,17 +441,19 @@ public long addDocument(Iterable doc) throws IOExcepti allowShardFailures(); IndexShard replica = group.addReplica(); expectThrows(Exception.class, () -> group.recoverReplica(replica, - (shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, new PeerRecoveryTargetService.RecoveryListener() { - @Override - public void onRecoveryDone(RecoveryState state, ShardLongFieldRange timestampMillisFieldRange) { - throw new AssertionError("recovery must fail"); - } - - @Override - public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) { - assertThat(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), equalTo("simulated")); - } - }))); + (shard, sourceNode) -> { + return new RecoveryTarget(shard, sourceNode, null, new PeerRecoveryTargetService.RecoveryListener() { + @Override + public void onRecoveryDone(RecoveryState state, ShardLongFieldRange timestampMillisFieldRange) { + throw new AssertionError("recovery must fail"); + } + + @Override + public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) { + assertThat(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), equalTo("simulated")); + } + }); + })); expectThrows(AlreadyClosedException.class, () -> replica.refresh("test")); group.removeReplica(replica); replica.store().close(); diff --git a/server/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java b/server/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java index 22741fec5c464..dd6f13c8f2796 100644 --- a/server/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java +++ b/server/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java @@ -12,13 +12,13 @@ import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.replication.ESIndexLevelReplicationTestCase; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.ShardLongFieldRange; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardLongFieldRange; import org.elasticsearch.index.store.Store; +import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveriesCollection; import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; -import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveryTarget; import java.util.concurrent.CountDownLatch; @@ -150,6 +150,6 @@ long startRecovery(RecoveriesCollection collection, DiscoveryNode sourceNode, In final DiscoveryNode rNode = getDiscoveryNode(indexShard.routingEntry().currentNodeId()); indexShard.markAsRecovering("remote", new RecoveryState(indexShard.routingEntry(), sourceNode, rNode)); indexShard.prepareForIndexRecovery(); - return collection.startRecovery(indexShard, sourceNode, listener, timeValue); + return collection.startRecovery(indexShard, sourceNode, null, listener, timeValue); } } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index 9608d75f335e5..3f6040c4722f9 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -156,6 +156,7 @@ import org.elasticsearch.indices.recovery.PeerRecoverySourceService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoverySettings; +import org.elasticsearch.indices.recovery.SnapshotFilesProvider; import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService; import org.elasticsearch.ingest.IngestService; import org.elasticsearch.monitor.StatusInfo; @@ -1806,12 +1807,13 @@ protected void assertSnapshotOrGenericThread() { SourceOnlyRecoveryPlannerService.INSTANCE ); + final SnapshotFilesProvider snapshotFilesProvider = new SnapshotFilesProvider(repositoriesService); indicesClusterStateService = new IndicesClusterStateService( settings, indicesService, clusterService, threadPool, - new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService), + new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService, snapshotFilesProvider), shardStateAction, repositoriesService, mock(SearchService.class), diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 91922081919d9..7d916e704db98 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -50,11 +50,11 @@ import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.TimeValue; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.DocIdSeqNoAndSource; @@ -399,7 +399,8 @@ public synchronized boolean removeReplica(IndexShard replica) throws IOException } public void recoverReplica(IndexShard replica) throws IOException { - recoverReplica(replica, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener)); + recoverReplica(replica, + (r, sourceNode) -> new RecoveryTarget(r, sourceNode, null, recoveryListener)); } public void recoverReplica(IndexShard replica, BiFunction targetSupplier) diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index e94701bbc0cd0..0ba8c3e6e3d75 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -582,7 +582,7 @@ protected DiscoveryNode getFakeDiscoNode(String id) { /** recovers a replica from the given primary **/ protected void recoverReplica(IndexShard replica, IndexShard primary, boolean startReplica) throws IOException { recoverReplica(replica, primary, - (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener), + (r, sourceNode) -> new RecoveryTarget(r, sourceNode, null, recoveryListener), true, startReplica); } @@ -638,7 +638,7 @@ protected final void recoverUnstartedReplica(final IndexShard replica, final RecoveryPlannerService recoveryPlannerService = SourceOnlyRecoveryPlannerService.INSTANCE; final RecoverySourceHandler recovery = new RecoverySourceHandler(primary, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool, - request, fileChunkSizeInBytes, between(1, 8), between(1, 8), recoveryPlannerService); + request, fileChunkSizeInBytes, between(1, 8), between(1, 8), between(1, 8), false, recoveryPlannerService); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable); try { diff --git a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java index 06eaffebf6a9d..5b34fe3b569eb 100644 --- a/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/elasticsearch/indices/recovery/AsyncRecoveryTarget.java @@ -12,9 +12,11 @@ import org.elasticsearch.common.bytes.ReleasableBytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeases; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetadata; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.repositories.IndexId; import java.util.List; import java.util.concurrent.Executor; @@ -82,4 +84,12 @@ public void writeFileChunk(StoreFileMetadata fileMetadata, long position, Releas } } } + + @Override + public void restoreFileFromSnapshot(String repository, + IndexId indexId, + BlobStoreIndexShardSnapshot.FileInfo snapshotFile, + ActionListener listener) { + executor.execute(() -> target.restoreFileFromSnapshot(repository, indexId, snapshotFile, listener)); + } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java index 4bc01b6db672b..96dbb74b2498f 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardFollowTaskReplicationTests.java @@ -8,6 +8,7 @@ import com.carrotsearch.hppc.LongHashSet; import com.carrotsearch.hppc.LongSet; + import org.apache.lucene.store.IOContext; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; @@ -26,13 +27,13 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.UUIDs; -import org.elasticsearch.core.Tuple; -import org.elasticsearch.core.Releasable; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.core.Releasable; import org.elasticsearch.core.TimeValue; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineFactory; @@ -372,7 +373,7 @@ protected EngineFactory getEngineFactory(ShardRouting routing) { // We need to recover the replica async to release the main thread for the following task to fill missing // operations between the local checkpoint and max_seq_no which the recovering replica is waiting for. recoveryFuture = group.asyncRecoverReplica(newReplica, - (shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, recoveryListener) {}); + (shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, null, recoveryListener) {}); } } if (recoveryFuture != null) {