mirror of
https://github.com/elastic/elasticsearch.git
synced 2025-04-25 07:37:19 -04:00
Add peer recoveries using snapshot files when possible (#76237)
This commit adds peer recoveries from snapshots. It allows establishing a replica by downloading file data from a snapshot rather than transferring the data from the primary. Enabling this feature is done on the repository definition. Repositories having the setting `use_for_peer_recovery=true` will be consulted to find a good snapshot when recovering a shard. Relates #73496
This commit is contained in:
parent
ff5df64a42
commit
2ebe5cd075
38 changed files with 2566 additions and 131 deletions
|
@ -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
|
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
|
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
|
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
|
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
|
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
|
the costs of recovering data from a snapshot. It also reduces the amount of
|
||||||
work that the primary must do during a recovery.
|
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`::
|
||||||
|
(<<cluster-update-settings,Dynamic>>, 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.
|
||||||
|
|
|
@ -0,0 +1,130 @@
|
||||||
|
/*
|
||||||
|
* 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.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<String, Object> searchResults = search(indexName, QueryBuilders.matchAllQuery());
|
||||||
|
assertThat(extractValue(searchResults, "hits.total.value"), equalTo(numDocs));
|
||||||
|
List<Map<String, Object>> hits = extractValue(searchResults, "hits.hits");
|
||||||
|
for (Map<String, Object> 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<String, Object> searchResults = search(indexName, QueryBuilders.matchQuery("text", "some"));
|
||||||
|
assertThat(extractValue(searchResults, "hits.total.value"), equalTo(numDocs));
|
||||||
|
}
|
||||||
|
|
||||||
|
private static Map<String, Object> 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<String, Object> 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("\"}}\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.setJsonEntity(bulkBody.toString());
|
||||||
|
assertOK(client().performRequest(documents));
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private static <T> T extractValue(Map<String, Object> map, String path) {
|
||||||
|
return (T) XContentMapValues.extractValue(path, map);
|
||||||
|
}
|
||||||
|
}
|
2
qa/snapshot-based-recoveries/build.gradle
Normal file
2
qa/snapshot-based-recoveries/build.gradle
Normal file
|
@ -0,0 +1,2 @@
|
||||||
|
apply plugin: 'elasticsearch.standalone-rest-test'
|
||||||
|
apply plugin: 'elasticsearch.internal-test-artifact'
|
36
qa/snapshot-based-recoveries/fs/build.gradle
Normal file
36
qa/snapshot-based-recoveries/fs/build.gradle
Normal file
|
@ -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'
|
||||||
|
}
|
|
@ -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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -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<String, Object> searchResults = search(indexName, QueryBuilders.matchAllQuery());
|
||||||
|
assertThat(extractValue(searchResults, "hits.total.value"), equalTo(numDocs));
|
||||||
|
List<Map<String, Object>> hits = extractValue(searchResults, "hits.hits");
|
||||||
|
for (Map<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> responseAsMap = responseAsMap(response);
|
||||||
|
assertThat(
|
||||||
|
extractValue(responseAsMap, "_shards.failed"),
|
||||||
|
equalTo(0)
|
||||||
|
);
|
||||||
|
return responseAsMap;
|
||||||
|
}
|
||||||
|
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
private static <T> T extractValue(Map<String, Object> map, String path) {
|
||||||
|
return (T) XContentMapValues.extractValue(path, map);
|
||||||
|
}
|
||||||
|
}
|
|
@ -18,6 +18,7 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
|
||||||
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
|
import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse;
|
||||||
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
|
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
|
||||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
|
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.RecoveryRequest;
|
||||||
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
|
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
|
||||||
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
|
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
|
||||||
|
@ -85,6 +86,7 @@ import org.elasticsearch.plugins.PluginsService;
|
||||||
import org.elasticsearch.repositories.RepositoriesService;
|
import org.elasticsearch.repositories.RepositoriesService;
|
||||||
import org.elasticsearch.repositories.Repository;
|
import org.elasticsearch.repositories.Repository;
|
||||||
import org.elasticsearch.repositories.RepositoryData;
|
import org.elasticsearch.repositories.RepositoryData;
|
||||||
|
import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
|
||||||
import org.elasticsearch.snapshots.Snapshot;
|
import org.elasticsearch.snapshots.Snapshot;
|
||||||
import org.elasticsearch.snapshots.SnapshotState;
|
import org.elasticsearch.snapshots.SnapshotState;
|
||||||
import org.elasticsearch.tasks.Task;
|
import org.elasticsearch.tasks.Task;
|
||||||
|
@ -115,6 +117,7 @@ import java.util.Map;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.Semaphore;
|
import java.util.concurrent.Semaphore;
|
||||||
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
|
@ -624,11 +627,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
String nodeA = internalCluster().startNode();
|
String nodeA = internalCluster().startNode();
|
||||||
|
|
||||||
logger.info("--> create repository");
|
logger.info("--> create repository");
|
||||||
assertAcked(client().admin().cluster().preparePutRepository(REPO_NAME)
|
createRepository(randomBoolean());
|
||||||
.setType("fs").setSettings(Settings.builder()
|
|
||||||
.put("location", randomRepoPath())
|
|
||||||
.put("compress", false)
|
|
||||||
).get());
|
|
||||||
|
|
||||||
ensureGreen();
|
ensureGreen();
|
||||||
|
|
||||||
|
@ -636,14 +635,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
createAndPopulateIndex(INDEX_NAME, 1, SHARD_COUNT, REPLICA_COUNT);
|
createAndPopulateIndex(INDEX_NAME, 1, SHARD_COUNT, REPLICA_COUNT);
|
||||||
|
|
||||||
logger.info("--> snapshot");
|
logger.info("--> snapshot");
|
||||||
CreateSnapshotResponse createSnapshotResponse = client().admin().cluster().prepareCreateSnapshot(REPO_NAME, SNAP_NAME)
|
CreateSnapshotResponse createSnapshotResponse = createSnapshot(INDEX_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));
|
|
||||||
|
|
||||||
client().admin().indices().prepareClose(INDEX_NAME).execute().actionGet();
|
client().admin().indices().prepareClose(INDEX_NAME).execute().actionGet();
|
||||||
|
|
||||||
|
@ -776,6 +768,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG,
|
PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG,
|
||||||
PeerRecoveryTargetService.Actions.TRANSLOG_OPS,
|
PeerRecoveryTargetService.Actions.TRANSLOG_OPS,
|
||||||
PeerRecoveryTargetService.Actions.FILES_INFO,
|
PeerRecoveryTargetService.Actions.FILES_INFO,
|
||||||
|
PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT,
|
||||||
PeerRecoveryTargetService.Actions.FILE_CHUNK,
|
PeerRecoveryTargetService.Actions.FILE_CHUNK,
|
||||||
PeerRecoveryTargetService.Actions.CLEAN_FILES,
|
PeerRecoveryTargetService.Actions.CLEAN_FILES,
|
||||||
PeerRecoveryTargetService.Actions.FINALIZE
|
PeerRecoveryTargetService.Actions.FINALIZE
|
||||||
|
@ -783,6 +776,10 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
final String recoveryActionToBlock = randomFrom(recoveryActions);
|
final String recoveryActionToBlock = randomFrom(recoveryActions);
|
||||||
logger.info("--> will temporarily interrupt recovery action between blue & red on [{}]", recoveryActionToBlock);
|
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 blueTransportService =
|
||||||
(MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName);
|
(MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName);
|
||||||
MockTransportService redTransportService =
|
MockTransportService redTransportService =
|
||||||
|
@ -819,6 +816,9 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
).get();
|
).get();
|
||||||
|
|
||||||
ensureGreen();
|
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();
|
searchResponse = client(redNodeName).prepareSearch(indexName).setPreference("_local").get();
|
||||||
assertHitCount(searchResponse, numDocs);
|
assertHitCount(searchResponse, numDocs);
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -946,6 +946,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
String[] recoveryActions = new String[]{
|
String[] recoveryActions = new String[]{
|
||||||
PeerRecoverySourceService.Actions.START_RECOVERY,
|
PeerRecoverySourceService.Actions.START_RECOVERY,
|
||||||
PeerRecoveryTargetService.Actions.FILES_INFO,
|
PeerRecoveryTargetService.Actions.FILES_INFO,
|
||||||
|
PeerRecoveryTargetService.Actions.RESTORE_FILE_FROM_SNAPSHOT,
|
||||||
PeerRecoveryTargetService.Actions.FILE_CHUNK,
|
PeerRecoveryTargetService.Actions.FILE_CHUNK,
|
||||||
PeerRecoveryTargetService.Actions.CLEAN_FILES,
|
PeerRecoveryTargetService.Actions.CLEAN_FILES,
|
||||||
//RecoveryTarget.Actions.TRANSLOG_OPS, <-- may not be sent if already flushed
|
//RecoveryTarget.Actions.TRANSLOG_OPS, <-- may not be sent if already flushed
|
||||||
|
@ -956,6 +957,11 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
final boolean dropRequests = randomBoolean();
|
final boolean dropRequests = randomBoolean();
|
||||||
logger.info("--> will {} between blue & red on [{}]", dropRequests ? "drop requests" : "break connection", recoveryActionToBlock);
|
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 blueMockTransportService =
|
||||||
(MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName);
|
(MockTransportService) internalCluster().getInstance(TransportService.class, blueNodeName);
|
||||||
MockTransportService redMockTransportService =
|
MockTransportService redMockTransportService =
|
||||||
|
@ -1047,6 +1053,10 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
ensureSearchable(indexName);
|
ensureSearchable(indexName);
|
||||||
assertHitCount(client().prepareSearch(indexName).get(), numDocs);
|
assertHitCount(client().prepareSearch(indexName).get(), numDocs);
|
||||||
|
|
||||||
|
if (randomBoolean()) {
|
||||||
|
createSnapshotThatCanBeUsedDuringRecovery(indexName);
|
||||||
|
}
|
||||||
|
|
||||||
MockTransportService masterTransportService =
|
MockTransportService masterTransportService =
|
||||||
(MockTransportService) internalCluster().getInstance(TransportService.class, masterNodeName);
|
(MockTransportService) internalCluster().getInstance(TransportService.class, masterNodeName);
|
||||||
MockTransportService blueMockTransportService =
|
MockTransportService blueMockTransportService =
|
||||||
|
@ -1815,4 +1825,48 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
assertBusy(() -> assertThat(nodeIndexShard.getLastSyncedGlobalCheckpoint(), equalTo(maxSeqNo)));
|
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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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<Class<? extends Plugin>> 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<String, Repository.Factory> getRepositories(
|
||||||
|
Environment env,
|
||||||
|
NamedXContentRegistry namedXContentRegistry,
|
||||||
|
ClusterService clusterService,
|
||||||
|
BigArrays bigArrays,
|
||||||
|
RecoverySettings recoverySettings
|
||||||
|
) {
|
||||||
|
return 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<String> 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<String> 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)
|
||||||
|
.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<String, Object> 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<String, Object> 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<RecoveryState> indexRecoveries = recoveryResponse.shardRecoveryStates().get(indexName);
|
||||||
|
assertThat(indexRecoveries, notNullValue());
|
||||||
|
|
||||||
|
List<RecoveryState> 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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -214,6 +214,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
|
||||||
RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING,
|
RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING,
|
||||||
RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_OPERATIONS_SETTING,
|
RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_OPERATIONS_SETTING,
|
||||||
RecoverySettings.INDICES_RECOVERY_USE_SNAPSHOTS_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_INITIAL_PRIMARIES_RECOVERIES_SETTING,
|
||||||
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING,
|
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING,
|
||||||
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING,
|
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING,
|
||||||
|
|
|
@ -11,6 +11,9 @@ package org.elasticsearch.index.snapshots.blobstore;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.elasticsearch.ElasticsearchParseException;
|
import org.elasticsearch.ElasticsearchParseException;
|
||||||
import org.elasticsearch.common.Strings;
|
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.unit.ByteSizeValue;
|
||||||
import org.elasticsearch.common.xcontent.ParseField;
|
import org.elasticsearch.common.xcontent.ParseField;
|
||||||
import org.elasticsearch.common.xcontent.ToXContentFragment;
|
import org.elasticsearch.common.xcontent.ToXContentFragment;
|
||||||
|
@ -21,6 +24,7 @@ import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
import java.util.Objects;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -31,7 +35,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
|
||||||
/**
|
/**
|
||||||
* Information about snapshotted file
|
* Information about snapshotted file
|
||||||
*/
|
*/
|
||||||
public static class FileInfo {
|
public static class FileInfo implements Writeable {
|
||||||
|
|
||||||
private final String name;
|
private final String name;
|
||||||
private final ByteSizeValue partSize;
|
private final ByteSizeValue partSize;
|
||||||
|
@ -47,7 +51,7 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
|
||||||
* @param partSize size of the single chunk
|
* @param partSize size of the single chunk
|
||||||
*/
|
*/
|
||||||
public FileInfo(String name, StoreFileMetadata metadata, ByteSizeValue partSize) {
|
public FileInfo(String name, StoreFileMetadata metadata, ByteSizeValue partSize) {
|
||||||
this.name = name;
|
this.name = Objects.requireNonNull(name);
|
||||||
this.metadata = metadata;
|
this.metadata = metadata;
|
||||||
|
|
||||||
long partBytes = Long.MAX_VALUE;
|
long partBytes = Long.MAX_VALUE;
|
||||||
|
@ -70,6 +74,17 @@ public class BlobStoreIndexShardSnapshot implements ToXContentFragment {
|
||||||
assert IntStream.range(0, numberOfParts).mapToLong(this::partBytes).sum() == metadata.length();
|
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
|
* Returns the base file name
|
||||||
*
|
*
|
||||||
|
|
|
@ -16,15 +16,17 @@ import org.apache.lucene.util.BytesRefIterator;
|
||||||
import org.elasticsearch.common.Strings;
|
import org.elasticsearch.common.Strings;
|
||||||
import org.elasticsearch.common.bytes.BytesReference;
|
import org.elasticsearch.common.bytes.BytesReference;
|
||||||
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
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.Releasable;
|
||||||
import org.elasticsearch.core.Releasables;
|
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.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.transport.Transports;
|
import org.elasticsearch.transport.Transports;
|
||||||
|
|
||||||
|
import java.io.EOFException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
|
@ -55,7 +57,6 @@ public class MultiFileWriter extends AbstractRefCounted implements Releasable {
|
||||||
private final ConcurrentMap<String, IndexOutput> openIndexOutputs = ConcurrentCollections.newConcurrentMap();
|
private final ConcurrentMap<String, IndexOutput> openIndexOutputs = ConcurrentCollections.newConcurrentMap();
|
||||||
private final ConcurrentMap<String, FileChunkWriter> fileChunkWriters = ConcurrentCollections.newConcurrentMap();
|
private final ConcurrentMap<String, FileChunkWriter> fileChunkWriters = ConcurrentCollections.newConcurrentMap();
|
||||||
|
|
||||||
|
|
||||||
final Map<String, String> tempFileNames = ConcurrentCollections.newConcurrentMap();
|
final Map<String, String> tempFileNames = ConcurrentCollections.newConcurrentMap();
|
||||||
|
|
||||||
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk)
|
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk)
|
||||||
|
@ -70,6 +71,48 @@ public class MultiFileWriter extends AbstractRefCounted implements Releasable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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. */
|
/** Get a temporary name for the provided file name. */
|
||||||
String getTempNameForFile(String origFile) {
|
String getTempNameForFile(String origFile) {
|
||||||
return tempFilePrefix + origFile;
|
return tempFilePrefix + origFile;
|
||||||
|
|
|
@ -325,6 +325,8 @@ public class PeerRecoverySourceService extends AbstractLifecycleComponent implem
|
||||||
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
||||||
recoverySettings.getMaxConcurrentFileChunks(),
|
recoverySettings.getMaxConcurrentFileChunks(),
|
||||||
recoverySettings.getMaxConcurrentOperations(),
|
recoverySettings.getMaxConcurrentOperations(),
|
||||||
|
recoverySettings.getMaxConcurrentSnapshotFileDownloads(),
|
||||||
|
recoverySettings.getUseSnapshotsDuringRecovery(),
|
||||||
recoveryPlannerService);
|
recoveryPlannerService);
|
||||||
return Tuple.tuple(handler, recoveryTarget);
|
return Tuple.tuple(handler, recoveryTarget);
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,15 +24,15 @@ import org.elasticsearch.cluster.ClusterStateObserver;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
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.common.io.stream.StreamInput;
|
||||||
import org.elasticsearch.core.Releasable;
|
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||||
import org.elasticsearch.core.TimeValue;
|
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
import org.elasticsearch.common.util.CancellableThreads;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
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.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.engine.RecoveryEngineException;
|
import org.elasticsearch.index.engine.RecoveryEngineException;
|
||||||
import org.elasticsearch.index.mapper.MapperException;
|
import org.elasticsearch.index.mapper.MapperException;
|
||||||
|
@ -40,8 +40,8 @@ import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
|
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
|
||||||
import org.elasticsearch.index.shard.IndexEventListener;
|
import org.elasticsearch.index.shard.IndexEventListener;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardLongFieldRange;
|
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.index.shard.ShardLongFieldRange;
|
||||||
import org.elasticsearch.index.shard.ShardNotFoundException;
|
import org.elasticsearch.index.shard.ShardNotFoundException;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
|
@ -77,6 +77,7 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
|
|
||||||
public static class Actions {
|
public static class Actions {
|
||||||
public static final String FILES_INFO = "internal:index/shard/recovery/filesInfo";
|
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 FILE_CHUNK = "internal:index/shard/recovery/file_chunk";
|
||||||
public static final String CLEAN_FILES = "internal:index/shard/recovery/clean_files";
|
public static final String CLEAN_FILES = "internal:index/shard/recovery/clean_files";
|
||||||
public static final String TRANSLOG_OPS = "internal:index/shard/recovery/translog_ops";
|
public static final String TRANSLOG_OPS = "internal:index/shard/recovery/translog_ops";
|
||||||
|
@ -91,19 +92,26 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
|
|
||||||
private final RecoverySettings recoverySettings;
|
private final RecoverySettings recoverySettings;
|
||||||
private final ClusterService clusterService;
|
private final ClusterService clusterService;
|
||||||
|
private final SnapshotFilesProvider snapshotFilesProvider;
|
||||||
|
|
||||||
private final RecoveriesCollection onGoingRecoveries;
|
private final RecoveriesCollection onGoingRecoveries;
|
||||||
|
|
||||||
public PeerRecoveryTargetService(ThreadPool threadPool, TransportService transportService,
|
public PeerRecoveryTargetService(ThreadPool threadPool,
|
||||||
RecoverySettings recoverySettings, ClusterService clusterService) {
|
TransportService transportService,
|
||||||
|
RecoverySettings recoverySettings,
|
||||||
|
ClusterService clusterService,
|
||||||
|
SnapshotFilesProvider snapshotFilesProvider) {
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
this.transportService = transportService;
|
this.transportService = transportService;
|
||||||
this.recoverySettings = recoverySettings;
|
this.recoverySettings = recoverySettings;
|
||||||
this.clusterService = clusterService;
|
this.clusterService = clusterService;
|
||||||
|
this.snapshotFilesProvider = snapshotFilesProvider;
|
||||||
this.onGoingRecoveries = new RecoveriesCollection(logger, threadPool);
|
this.onGoingRecoveries = new RecoveriesCollection(logger, threadPool);
|
||||||
|
|
||||||
transportService.registerRequestHandler(Actions.FILES_INFO, ThreadPool.Names.GENERIC, RecoveryFilesInfoRequest::new,
|
transportService.registerRequestHandler(Actions.FILES_INFO, ThreadPool.Names.GENERIC, RecoveryFilesInfoRequest::new,
|
||||||
new FilesInfoRequestHandler());
|
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,
|
transportService.registerRequestHandler(Actions.FILE_CHUNK, ThreadPool.Names.GENERIC, RecoveryFileChunkRequest::new,
|
||||||
new FileChunkTransportRequestHandler());
|
new FileChunkTransportRequestHandler());
|
||||||
transportService.registerRequestHandler(Actions.CLEAN_FILES, ThreadPool.Names.GENERIC,
|
transportService.registerRequestHandler(Actions.CLEAN_FILES, ThreadPool.Names.GENERIC,
|
||||||
|
@ -130,7 +138,8 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
|
|
||||||
public void startRecovery(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) {
|
public void startRecovery(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) {
|
||||||
// create a new recovery status, and process...
|
// 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
|
// 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.
|
// 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));
|
threadPool.generic().execute(new RecoveryRunner(recoveryId));
|
||||||
|
@ -469,6 +478,22 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class RestoreFileFromSnapshotTransportRequestHandler implements TransportRequestHandler<RecoverySnapshotFileRequest> {
|
||||||
|
@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<Void> 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<Void> createOrFinishListener(final RecoveryRef recoveryRef, final TransportChannel channel,
|
private ActionListener<Void> createOrFinishListener(final RecoveryRef recoveryRef, final TransportChannel channel,
|
||||||
final String action, final RecoveryTransportRequest request) {
|
final String action, final RecoveryTransportRequest request) {
|
||||||
return createOrFinishListener(recoveryRef, channel, action, request, nullVal -> TransportResponse.Empty.INSTANCE);
|
return createOrFinishListener(recoveryRef, channel, action, request, nullVal -> TransportResponse.Empty.INSTANCE);
|
||||||
|
|
|
@ -12,9 +12,9 @@ import org.apache.logging.log4j.Logger;
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.elasticsearch.ElasticsearchTimeoutException;
|
import org.elasticsearch.ElasticsearchTimeoutException;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.core.TimeValue;
|
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||||
|
import org.elasticsearch.core.TimeValue;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardClosedException;
|
import org.elasticsearch.index.shard.IndexShardClosedException;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
@ -50,9 +50,12 @@ public class RecoveriesCollection {
|
||||||
*
|
*
|
||||||
* @return the id of the new recovery.
|
* @return the id of the new recovery.
|
||||||
*/
|
*/
|
||||||
public long startRecovery(IndexShard indexShard, DiscoveryNode sourceNode,
|
public long startRecovery(IndexShard indexShard,
|
||||||
PeerRecoveryTargetService.RecoveryListener listener, TimeValue activityTimeout) {
|
DiscoveryNode sourceNode,
|
||||||
RecoveryTarget recoveryTarget = new RecoveryTarget(indexShard, sourceNode, listener);
|
SnapshotFilesProvider snapshotFilesProvider,
|
||||||
|
PeerRecoveryTargetService.RecoveryListener listener,
|
||||||
|
TimeValue activityTimeout) {
|
||||||
|
RecoveryTarget recoveryTarget = new RecoveryTarget(indexShard, sourceNode, snapshotFilesProvider, listener);
|
||||||
startRecoveryInternal(recoveryTarget, activityTimeout);
|
startRecoveryInternal(recoveryTarget, activityTimeout);
|
||||||
return recoveryTarget.recoveryId();
|
return recoveryTarget.recoveryId();
|
||||||
}
|
}
|
||||||
|
|
|
@ -138,6 +138,15 @@ public class RecoverySettings {
|
||||||
public static final Setting<Boolean> INDICES_RECOVERY_USE_SNAPSHOTS_SETTING =
|
public static final Setting<Boolean> INDICES_RECOVERY_USE_SNAPSHOTS_SETTING =
|
||||||
Setting.boolSetting("indices.recovery.use_snapshots", true, Property.Dynamic, Property.NodeScope);
|
Setting.boolSetting("indices.recovery.use_snapshots", true, Property.Dynamic, Property.NodeScope);
|
||||||
|
|
||||||
|
public static final Setting<Integer> 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);
|
public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue(512, ByteSizeUnit.KB);
|
||||||
|
|
||||||
private volatile ByteSizeValue maxBytesPerSec;
|
private volatile ByteSizeValue maxBytesPerSec;
|
||||||
|
@ -151,6 +160,7 @@ public class RecoverySettings {
|
||||||
private volatile TimeValue internalActionRetryTimeout;
|
private volatile TimeValue internalActionRetryTimeout;
|
||||||
private volatile TimeValue internalActionLongTimeout;
|
private volatile TimeValue internalActionLongTimeout;
|
||||||
private volatile boolean useSnapshotsDuringRecovery;
|
private volatile boolean useSnapshotsDuringRecovery;
|
||||||
|
private volatile int maxConcurrentSnapshotFileDownloads;
|
||||||
|
|
||||||
private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE;
|
private volatile ByteSizeValue chunkSize = DEFAULT_CHUNK_SIZE;
|
||||||
|
|
||||||
|
@ -174,6 +184,7 @@ public class RecoverySettings {
|
||||||
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac());
|
rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac());
|
||||||
}
|
}
|
||||||
this.useSnapshotsDuringRecovery = INDICES_RECOVERY_USE_SNAPSHOTS_SETTING.get(settings);
|
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);
|
logger.debug("using max_bytes_per_sec[{}]", maxBytesPerSec);
|
||||||
|
|
||||||
|
@ -188,6 +199,8 @@ public class RecoverySettings {
|
||||||
this::setInternalActionLongTimeout);
|
this::setInternalActionLongTimeout);
|
||||||
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, this::setActivityTimeout);
|
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, this::setActivityTimeout);
|
||||||
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_USE_SNAPSHOTS_SETTING, this::setUseSnapshotsDuringRecovery);
|
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_USE_SNAPSHOTS_SETTING, this::setUseSnapshotsDuringRecovery);
|
||||||
|
clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_CONCURRENT_SNAPSHOT_FILE_DOWNLOADS,
|
||||||
|
this::setMaxConcurrentSnapshotFileDownloads);
|
||||||
}
|
}
|
||||||
|
|
||||||
public RateLimiter rateLimiter() {
|
public RateLimiter rateLimiter() {
|
||||||
|
@ -281,4 +294,12 @@ public class RecoverySettings {
|
||||||
private void setUseSnapshotsDuringRecovery(boolean useSnapshotsDuringRecovery) {
|
private void setUseSnapshotsDuringRecovery(boolean useSnapshotsDuringRecovery) {
|
||||||
this.useSnapshotsDuringRecovery = useSnapshotsDuringRecovery;
|
this.useSnapshotsDuringRecovery = useSnapshotsDuringRecovery;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getMaxConcurrentSnapshotFileDownloads() {
|
||||||
|
return maxConcurrentSnapshotFileDownloads;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setMaxConcurrentSnapshotFileDownloads(int maxConcurrentSnapshotFileDownloads) {
|
||||||
|
this.maxConcurrentSnapshotFileDownloads = maxConcurrentSnapshotFileDownloads;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
|
@ -36,9 +36,11 @@ import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
||||||
import org.elasticsearch.common.logging.Loggers;
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
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.FutureUtils;
|
||||||
import org.elasticsearch.common.util.concurrent.ListenableFuture;
|
import org.elasticsearch.common.util.concurrent.ListenableFuture;
|
||||||
import org.elasticsearch.core.CheckedRunnable;
|
import org.elasticsearch.core.CheckedRunnable;
|
||||||
|
import org.elasticsearch.core.Nullable;
|
||||||
import org.elasticsearch.core.Releasable;
|
import org.elasticsearch.core.Releasable;
|
||||||
import org.elasticsearch.core.Releasables;
|
import org.elasticsearch.core.Releasables;
|
||||||
import org.elasticsearch.core.TimeValue;
|
import org.elasticsearch.core.TimeValue;
|
||||||
|
@ -71,18 +73,25 @@ import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.Deque;
|
import java.util.Deque;
|
||||||
|
import java.util.HashSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ConcurrentLinkedDeque;
|
import java.util.concurrent.ConcurrentLinkedDeque;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
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.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.function.IntSupplier;
|
import java.util.function.IntSupplier;
|
||||||
import java.util.stream.StreamSupport;
|
import java.util.stream.StreamSupport;
|
||||||
|
|
||||||
|
import static org.elasticsearch.common.util.CollectionUtils.concatLists;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* RecoverySourceHandler handles the three phases of shard recovery, which is
|
* RecoverySourceHandler handles the three phases of shard recovery, which is
|
||||||
* everything relating to copying the segment files as well as sending translog
|
* everything relating to copying the segment files as well as sending translog
|
||||||
|
@ -106,6 +115,8 @@ public class RecoverySourceHandler {
|
||||||
private final RecoveryTargetHandler recoveryTarget;
|
private final RecoveryTargetHandler recoveryTarget;
|
||||||
private final int maxConcurrentFileChunks;
|
private final int maxConcurrentFileChunks;
|
||||||
private final int maxConcurrentOperations;
|
private final int maxConcurrentOperations;
|
||||||
|
private final int maxConcurrentSnapshotFileDownloads;
|
||||||
|
private final boolean useSnapshots;
|
||||||
private final ThreadPool threadPool;
|
private final ThreadPool threadPool;
|
||||||
private final RecoveryPlannerService recoveryPlannerService;
|
private final RecoveryPlannerService recoveryPlannerService;
|
||||||
private final CancellableThreads cancellableThreads = new CancellableThreads();
|
private final CancellableThreads cancellableThreads = new CancellableThreads();
|
||||||
|
@ -114,7 +125,8 @@ public class RecoverySourceHandler {
|
||||||
|
|
||||||
public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, ThreadPool threadPool,
|
public RecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, ThreadPool threadPool,
|
||||||
StartRecoveryRequest request, int fileChunkSizeInBytes, int maxConcurrentFileChunks,
|
StartRecoveryRequest request, int fileChunkSizeInBytes, int maxConcurrentFileChunks,
|
||||||
int maxConcurrentOperations, RecoveryPlannerService recoveryPlannerService) {
|
int maxConcurrentOperations, int maxConcurrentSnapshotFileDownloads, boolean useSnapshots,
|
||||||
|
RecoveryPlannerService recoveryPlannerService) {
|
||||||
this.shard = shard;
|
this.shard = shard;
|
||||||
this.recoveryTarget = recoveryTarget;
|
this.recoveryTarget = recoveryTarget;
|
||||||
this.threadPool = threadPool;
|
this.threadPool = threadPool;
|
||||||
|
@ -125,6 +137,8 @@ public class RecoverySourceHandler {
|
||||||
this.chunkSizeInBytes = fileChunkSizeInBytes;
|
this.chunkSizeInBytes = fileChunkSizeInBytes;
|
||||||
this.maxConcurrentFileChunks = maxConcurrentFileChunks;
|
this.maxConcurrentFileChunks = maxConcurrentFileChunks;
|
||||||
this.maxConcurrentOperations = maxConcurrentOperations;
|
this.maxConcurrentOperations = maxConcurrentOperations;
|
||||||
|
this.maxConcurrentSnapshotFileDownloads = maxConcurrentSnapshotFileDownloads;
|
||||||
|
this.useSnapshots = useSnapshots;
|
||||||
}
|
}
|
||||||
|
|
||||||
public StartRecoveryRequest getRequest() {
|
public StartRecoveryRequest getRequest() {
|
||||||
|
@ -476,7 +490,7 @@ public class RecoverySourceHandler {
|
||||||
startingSeqNo,
|
startingSeqNo,
|
||||||
translogOps.getAsInt(),
|
translogOps.getAsInt(),
|
||||||
getRequest().targetNode().getVersion(),
|
getRequest().targetNode().getVersion(),
|
||||||
false,
|
useSnapshots,
|
||||||
ActionListener.wrap(plan ->
|
ActionListener.wrap(plan ->
|
||||||
recoverFilesFromSourceAndSnapshot(plan, store, stopWatch, listener), listener::onFailure)
|
recoverFilesFromSourceAndSnapshot(plan, store, stopWatch, listener), listener::onFailure)
|
||||||
);
|
);
|
||||||
|
@ -543,6 +557,7 @@ public class RecoverySourceHandler {
|
||||||
}
|
}
|
||||||
|
|
||||||
final StepListener<Void> sendFileInfoStep = new StepListener<>();
|
final StepListener<Void> sendFileInfoStep = new StepListener<>();
|
||||||
|
final StepListener<List<StoreFileMetadata>> recoverSnapshotFilesStep = new StepListener<>();
|
||||||
final StepListener<Void> sendFilesStep = new StepListener<>();
|
final StepListener<Void> sendFilesStep = new StepListener<>();
|
||||||
final StepListener<RetentionLease> createRetentionLeaseStep = new StepListener<>();
|
final StepListener<RetentionLease> createRetentionLeaseStep = new StepListener<>();
|
||||||
final StepListener<Void> cleanFilesStep = new StepListener<>();
|
final StepListener<Void> cleanFilesStep = new StepListener<>();
|
||||||
|
@ -556,11 +571,19 @@ public class RecoverySourceHandler {
|
||||||
sendFileInfoStep
|
sendFileInfoStep
|
||||||
);
|
);
|
||||||
|
|
||||||
final List<StoreFileMetadata> sourceFiles = shardRecoveryPlan.getSourceFilesToRecover();
|
sendFileInfoStep.whenComplete(r -> recoverSnapshotFiles(shardRecoveryPlan, recoverSnapshotFilesStep), listener::onFailure);
|
||||||
|
|
||||||
|
recoverSnapshotFilesStep.whenComplete(filesFailedToRecoverFromSnapshot -> {
|
||||||
|
final List<StoreFileMetadata> filesToRecoverFromSource;
|
||||||
|
if (filesFailedToRecoverFromSnapshot.isEmpty()) {
|
||||||
|
filesToRecoverFromSource = shardRecoveryPlan.getSourceFilesToRecover();
|
||||||
|
} else {
|
||||||
|
filesToRecoverFromSource = concatLists(shardRecoveryPlan.getSourceFilesToRecover(), filesFailedToRecoverFromSnapshot);
|
||||||
|
}
|
||||||
|
|
||||||
sendFileInfoStep.whenComplete(r ->
|
|
||||||
sendFiles(store,
|
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();
|
final long startingSeqNo = shardRecoveryPlan.getStartingSeqNo();
|
||||||
sendFilesStep.whenComplete(r -> createRetentionLease(startingSeqNo, createRetentionLeaseStep), listener::onFailure);
|
sendFilesStep.whenComplete(r -> createRetentionLease(startingSeqNo, createRetentionLeaseStep), listener::onFailure);
|
||||||
|
@ -589,6 +612,165 @@ public class RecoverySourceHandler {
|
||||||
}, listener::onFailure);
|
}, 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<List<StoreFileMetadata>> 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<List<StoreFileMetadata>> listener;
|
||||||
|
private final CountDown countDown;
|
||||||
|
private final BlockingQueue<BlobStoreIndexShardSnapshot.FileInfo> pendingSnapshotFilesToRecover;
|
||||||
|
private final AtomicBoolean cancelled = new AtomicBoolean();
|
||||||
|
private final Set<ListenableFuture<Void>> outstandingRequests = new HashSet<>(maxConcurrentSnapshotFileDownloads);
|
||||||
|
private List<StoreFileMetadata> filesFailedToDownloadFromSnapshot;
|
||||||
|
|
||||||
|
SnapshotRecoverFileRequestsSender(ShardRecoveryPlan shardRecoveryPlan, ActionListener<List<StoreFileMetadata>> 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<Void> requestFuture = new ListenableFuture<>();
|
||||||
|
try {
|
||||||
|
cancellableThreads.checkForCancel();
|
||||||
|
|
||||||
|
ActionListener<Void> 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<StoreFileMetadata> failedToRecoverFromSnapshotFiles = getFilesFailedToRecoverFromSnapshot();
|
||||||
|
listener.onResponse(failedToRecoverFromSnapshotFiles);
|
||||||
|
} else {
|
||||||
|
sendRequest();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronized void addFileFailedToRecoverFromSnapshot(StoreFileMetadata storeFileMetadata) {
|
||||||
|
if (filesFailedToDownloadFromSnapshot == null) {
|
||||||
|
filesFailedToDownloadFromSnapshot = new ArrayList<>();
|
||||||
|
}
|
||||||
|
filesFailedToDownloadFromSnapshot.add(storeFileMetadata);
|
||||||
|
}
|
||||||
|
|
||||||
|
synchronized List<StoreFileMetadata> getFilesFailedToRecoverFromSnapshot() {
|
||||||
|
return Objects.requireNonNullElse(filesFailedToDownloadFromSnapshot, Collections.emptyList());
|
||||||
|
}
|
||||||
|
|
||||||
|
private void trackOutstandingRequest(ListenableFuture<Void> future) {
|
||||||
|
boolean cancelled;
|
||||||
|
synchronized (outstandingRequests) {
|
||||||
|
cancelled = cancellableThreads.isCancelled();
|
||||||
|
if (cancelled == false) {
|
||||||
|
outstandingRequests.add(future);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (cancelled) {
|
||||||
|
cancellableThreads.checkForCancel();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void unTrackOutstandingRequest(ListenableFuture<Void> future) {
|
||||||
|
synchronized (outstandingRequests) {
|
||||||
|
outstandingRequests.remove(future);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void notifyFailureOnceAllOutstandingRequestAreDone(Exception e) {
|
||||||
|
assert cancelled.get();
|
||||||
|
|
||||||
|
final Set<ListenableFuture<Void>> 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<Void> outstandingFuture : pendingRequests) {
|
||||||
|
outstandingFuture.addListener(ActionListener.wrap(() -> {
|
||||||
|
if (pendingRequestsCountDown.countDown()) {
|
||||||
|
listener.onFailure(e);
|
||||||
|
}
|
||||||
|
}));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void createRetentionLease(final long startingSeqNo, ActionListener<RetentionLease> listener) {
|
void createRetentionLease(final long startingSeqNo, ActionListener<RetentionLease> listener) {
|
||||||
runUnderPrimaryPermit(() -> {
|
runUnderPrimaryPermit(() -> {
|
||||||
// Clone the peer recovery retention lease belonging to the source shard. We are retaining history between the the local
|
// Clone the peer recovery retention lease belonging to the source shard. We are retaining history between the the local
|
||||||
|
|
|
@ -633,6 +633,12 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
recovered += 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 *
|
* file name *
|
||||||
*/
|
*/
|
||||||
|
@ -761,6 +767,12 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
file.addRecoveredBytes(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) {
|
public FileDetail get(String name) {
|
||||||
return fileDetails.get(name);
|
return fileDetails.get(name);
|
||||||
}
|
}
|
||||||
|
@ -845,6 +857,10 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
fileDetails.addRecoveredBytesToFile(name, bytes);
|
fileDetails.addRecoveredBytesToFile(name, bytes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public synchronized void resetRecoveredBytesOfFile(String name) {
|
||||||
|
fileDetails.resetRecoveredBytesOfFile(name);
|
||||||
|
}
|
||||||
|
|
||||||
public synchronized void addSourceThrottling(long timeInNanos) {
|
public synchronized void addSourceThrottling(long timeInNanos) {
|
||||||
if (sourceThrottlingInNanos == UNKNOWN) {
|
if (sourceThrottlingInNanos == UNKNOWN) {
|
||||||
sourceThrottlingInNanos = timeInNanos;
|
sourceThrottlingInNanos = timeInNanos;
|
||||||
|
|
|
@ -9,6 +9,7 @@
|
||||||
package org.elasticsearch.indices.recovery;
|
package org.elasticsearch.indices.recovery;
|
||||||
|
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.apache.lucene.index.CorruptIndexException;
|
import org.apache.lucene.index.CorruptIndexException;
|
||||||
import org.apache.lucene.index.IndexFormatTooNewException;
|
import org.apache.lucene.index.IndexFormatTooNewException;
|
||||||
import org.apache.lucene.index.IndexFormatTooOldException;
|
import org.apache.lucene.index.IndexFormatTooOldException;
|
||||||
|
@ -18,14 +19,14 @@ import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.core.Nullable;
|
|
||||||
import org.elasticsearch.common.UUIDs;
|
import org.elasticsearch.common.UUIDs;
|
||||||
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
||||||
import org.elasticsearch.core.Releasable;
|
|
||||||
import org.elasticsearch.common.logging.Loggers;
|
import org.elasticsearch.common.logging.Loggers;
|
||||||
import org.elasticsearch.common.lucene.Lucene;
|
import org.elasticsearch.common.lucene.Lucene;
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
import org.elasticsearch.common.util.CancellableThreads;
|
||||||
import org.elasticsearch.core.AbstractRefCounted;
|
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.engine.Engine;
|
||||||
import org.elasticsearch.index.mapper.MapperException;
|
import org.elasticsearch.index.mapper.MapperException;
|
||||||
import org.elasticsearch.index.seqno.ReplicationTracker;
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
|
@ -35,11 +36,14 @@ import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardNotRecoveringException;
|
import org.elasticsearch.index.shard.IndexShardNotRecoveringException;
|
||||||
import org.elasticsearch.index.shard.IndexShardState;
|
import org.elasticsearch.index.shard.IndexShardState;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
|
import org.elasticsearch.repositories.IndexId;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
import java.nio.file.Path;
|
import java.nio.file.Path;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
@ -62,6 +66,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
private final long recoveryId;
|
private final long recoveryId;
|
||||||
private final IndexShard indexShard;
|
private final IndexShard indexShard;
|
||||||
private final DiscoveryNode sourceNode;
|
private final DiscoveryNode sourceNode;
|
||||||
|
private final SnapshotFilesProvider snapshotFilesProvider;
|
||||||
private final MultiFileWriter multiFileWriter;
|
private final MultiFileWriter multiFileWriter;
|
||||||
private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker();
|
private final RecoveryRequestTracker requestTracker = new RecoveryRequestTracker();
|
||||||
private final Store store;
|
private final Store store;
|
||||||
|
@ -86,7 +91,10 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
* @param sourceNode source node of the recovery where we recover from
|
* @param sourceNode source node of the recovery where we recover from
|
||||||
* @param listener called when recovery is completed/failed
|
* @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");
|
super("recovery_status");
|
||||||
this.cancellableThreads = new CancellableThreads();
|
this.cancellableThreads = new CancellableThreads();
|
||||||
this.recoveryId = idGenerator.incrementAndGet();
|
this.recoveryId = idGenerator.incrementAndGet();
|
||||||
|
@ -94,6 +102,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
this.logger = Loggers.getLogger(getClass(), indexShard.shardId());
|
this.logger = Loggers.getLogger(getClass(), indexShard.shardId());
|
||||||
this.indexShard = indexShard;
|
this.indexShard = indexShard;
|
||||||
this.sourceNode = sourceNode;
|
this.sourceNode = sourceNode;
|
||||||
|
this.snapshotFilesProvider = snapshotFilesProvider;
|
||||||
this.shardId = indexShard.shardId();
|
this.shardId = indexShard.shardId();
|
||||||
final String tempFilePrefix = RECOVERY_PREFIX + UUIDs.randomBase64UUID() + ".";
|
final String tempFilePrefix = RECOVERY_PREFIX + UUIDs.randomBase64UUID() + ".";
|
||||||
this.multiFileWriter = new MultiFileWriter(indexShard.store(), indexShard.recoveryState().getIndex(), tempFilePrefix, logger,
|
this.multiFileWriter = new MultiFileWriter(indexShard.store(), indexShard.recoveryState().getIndex(), tempFilePrefix, logger,
|
||||||
|
@ -110,7 +119,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
* @return a copy of this recovery target
|
* @return a copy of this recovery target
|
||||||
*/
|
*/
|
||||||
public RecoveryTarget retryCopy() {
|
public RecoveryTarget retryCopy() {
|
||||||
return new RecoveryTarget(indexShard, sourceNode, listener);
|
return new RecoveryTarget(indexShard, sourceNode, snapshotFilesProvider, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -495,6 +504,30 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo fileInfo,
|
||||||
|
ActionListener<Void> 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. */
|
/** Get a temporary name for the provided file name. */
|
||||||
public String getTempNameForFile(String origFile) {
|
public String getTempNameForFile(String origFile) {
|
||||||
return multiFileWriter.getTempNameForFile(origFile);
|
return multiFileWriter.getTempNameForFile(origFile);
|
||||||
|
|
|
@ -11,9 +11,11 @@ import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
||||||
import org.elasticsearch.index.seqno.ReplicationTracker;
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
|
import org.elasticsearch.repositories.IndexId;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
|
@ -92,6 +94,17 @@ public interface RecoveryTargetHandler {
|
||||||
*/
|
*/
|
||||||
void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetadata, ActionListener<Void> listener);
|
void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetadata, ActionListener<Void> 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<Void> listener);
|
||||||
|
|
||||||
/** writes a partial file chunk to the target store */
|
/** writes a partial file chunk to the target store */
|
||||||
void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content,
|
void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content,
|
||||||
boolean lastChunk, int totalTranslogOps, ActionListener<Void> listener);
|
boolean lastChunk, int totalTranslogOps, ActionListener<Void> listener);
|
||||||
|
|
|
@ -30,9 +30,11 @@ import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||||
import org.elasticsearch.index.seqno.ReplicationTracker;
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
|
import org.elasticsearch.repositories.IndexId;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
import org.elasticsearch.transport.ConnectTransportException;
|
import org.elasticsearch.transport.ConnectTransportException;
|
||||||
import org.elasticsearch.transport.RemoteTransportException;
|
import org.elasticsearch.transport.RemoteTransportException;
|
||||||
|
@ -170,6 +172,20 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
|
||||||
executeRetryableAction(action, request, TransportRequestOptions.EMPTY, responseListener, reader);
|
executeRetryableAction(action, request, TransportRequestOptions.EMPTY, responseListener, reader);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> 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<TransportResponse.Empty> reader = in -> TransportResponse.Empty.INSTANCE;
|
||||||
|
final ActionListener<TransportResponse.Empty> responseListener = listener.map(r -> null);
|
||||||
|
executeRetryableAction(action, request, TransportRequestOptions.EMPTY, responseListener, reader);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content,
|
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content,
|
||||||
boolean lastChunk, int totalTranslogOps, ActionListener<Void> listener) {
|
boolean lastChunk, int totalTranslogOps, ActionListener<Void> listener) {
|
||||||
|
|
|
@ -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<Long> 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();
|
||||||
|
}
|
||||||
|
}
|
|
@ -130,6 +130,10 @@ public class ShardRecoveryPlan {
|
||||||
return snapshotFiles.isEmpty();
|
return snapshotFiles.isEmpty();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public List<BlobStoreIndexShardSnapshot.FileInfo> getSnapshotFiles() {
|
||||||
|
return snapshotFiles;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Iterator<BlobStoreIndexShardSnapshot.FileInfo> iterator() {
|
public Iterator<BlobStoreIndexShardSnapshot.FileInfo> iterator() {
|
||||||
return snapshotFiles.iterator();
|
return snapshotFiles.iterator();
|
||||||
|
|
|
@ -113,7 +113,10 @@ import org.elasticsearch.indices.cluster.IndicesClusterStateService;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
|
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
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.indices.store.IndicesStore;
|
||||||
import org.elasticsearch.ingest.IngestService;
|
import org.elasticsearch.ingest.IngestService;
|
||||||
import org.elasticsearch.monitor.MonitorService;
|
import org.elasticsearch.monitor.MonitorService;
|
||||||
|
@ -682,10 +685,18 @@ public class Node implements Closeable {
|
||||||
b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery());
|
b.bind(Discovery.class).toInstance(discoveryModule.getDiscovery());
|
||||||
{
|
{
|
||||||
processRecoverySettings(settingsModule.getClusterSettings(), recoverySettings);
|
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,
|
b.bind(PeerRecoverySourceService.class).toInstance(new PeerRecoverySourceService(transportService,
|
||||||
indicesService, recoverySettings, SourceOnlyRecoveryPlannerService.INSTANCE));
|
indicesService, recoverySettings, recoveryPlannerService));
|
||||||
b.bind(PeerRecoveryTargetService.class).toInstance(new PeerRecoveryTargetService(threadPool,
|
b.bind(PeerRecoveryTargetService.class).toInstance(new PeerRecoveryTargetService(threadPool,
|
||||||
transportService, recoverySettings, clusterService));
|
transportService, recoverySettings, clusterService, snapshotFilesProvider));
|
||||||
}
|
}
|
||||||
b.bind(HttpServerTransport.class).toInstance(httpServerTransport);
|
b.bind(HttpServerTransport.class).toInstance(httpServerTransport);
|
||||||
pluginComponents.forEach(p -> {
|
pluginComponents.forEach(p -> {
|
||||||
|
|
|
@ -3440,6 +3440,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public int getReadBufferSizeInBytes() {
|
||||||
|
return bufferSize;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The result of removing a snapshot from a shard folder in the repository.
|
* The result of removing a snapshot from a shard folder in the repository.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -22,9 +22,9 @@ import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.core.TimeValue;
|
|
||||||
import org.elasticsearch.common.util.iterable.Iterables;
|
import org.elasticsearch.common.util.iterable.Iterables;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.core.TimeValue;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.EngineFactory;
|
import org.elasticsearch.index.engine.EngineFactory;
|
||||||
|
@ -116,7 +116,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
||||||
thread.start();
|
thread.start();
|
||||||
IndexShard replica = shards.addReplica();
|
IndexShard replica = shards.addReplica();
|
||||||
Future<Void> future = shards.asyncRecoverReplica(replica,
|
Future<Void> future = shards.asyncRecoverReplica(replica,
|
||||||
(indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) {
|
(indexShard, node) -> new RecoveryTarget(indexShard, node, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void cleanFiles(int totalTranslogOps, long globalCheckpoint,
|
public void cleanFiles(int totalTranslogOps, long globalCheckpoint,
|
||||||
Store.MetadataSnapshot sourceMetadata, ActionListener<Void> listener) {
|
Store.MetadataSnapshot sourceMetadata, ActionListener<Void> listener) {
|
||||||
|
@ -193,7 +193,7 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
|
||||||
thread.start();
|
thread.start();
|
||||||
IndexShard replica = shards.addReplica();
|
IndexShard replica = shards.addReplica();
|
||||||
Future<Void> fut = shards.asyncRecoverReplica(replica,
|
Future<Void> fut = shards.asyncRecoverReplica(replica,
|
||||||
(shard, node) -> new RecoveryTarget(shard, node, recoveryListener) {
|
(shard, node) -> new RecoveryTarget(shard, node, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -23,10 +23,10 @@ import org.elasticsearch.cluster.metadata.IndexMetadata;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.common.bytes.BytesArray;
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
import org.elasticsearch.core.Releasable;
|
|
||||||
import org.elasticsearch.common.lucene.uid.Versions;
|
import org.elasticsearch.common.lucene.uid.Versions;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.core.Releasable;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.VersionType;
|
import org.elasticsearch.index.VersionType;
|
||||||
import org.elasticsearch.index.engine.DocIdSeqNoAndSource;
|
import org.elasticsearch.index.engine.DocIdSeqNoAndSource;
|
||||||
|
@ -423,7 +423,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
AtomicBoolean recoveryDone = new AtomicBoolean(false);
|
AtomicBoolean recoveryDone = new AtomicBoolean(false);
|
||||||
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(newReplica, (indexShard, node) -> {
|
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(newReplica, (indexShard, node) -> {
|
||||||
recoveryStart.countDown();
|
recoveryStart.countDown();
|
||||||
return new RecoveryTarget(indexShard, node, recoveryListener) {
|
return new RecoveryTarget(indexShard, node, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionListener<Void> listener) {
|
public void finalizeRecovery(long globalCheckpoint, long trimAboveSeqNo, ActionListener<Void> listener) {
|
||||||
recoveryDone.set(true);
|
recoveryDone.set(true);
|
||||||
|
@ -478,7 +478,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
final IndexShard replica = shards.addReplica();
|
final IndexShard replica = shards.addReplica();
|
||||||
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(
|
final Future<Void> recoveryFuture = shards.asyncRecoverReplica(
|
||||||
replica,
|
replica,
|
||||||
(indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener) {
|
(indexShard, node) -> new RecoveryTarget(indexShard, node, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void indexTranslogOperations(
|
public void indexTranslogOperations(
|
||||||
final List<Translog.Operation> operations,
|
final List<Translog.Operation> operations,
|
||||||
|
@ -743,7 +743,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
public BlockingTarget(RecoveryState.Stage stageToBlock, CountDownLatch recoveryBlocked, CountDownLatch releaseRecovery,
|
public BlockingTarget(RecoveryState.Stage stageToBlock, CountDownLatch recoveryBlocked, CountDownLatch releaseRecovery,
|
||||||
IndexShard shard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener,
|
IndexShard shard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener,
|
||||||
Logger logger) {
|
Logger logger) {
|
||||||
super(shard, sourceNode, listener);
|
super(shard, sourceNode, null, listener);
|
||||||
this.recoveryBlocked = recoveryBlocked;
|
this.recoveryBlocked = recoveryBlocked;
|
||||||
this.releaseRecovery = releaseRecovery;
|
this.releaseRecovery = releaseRecovery;
|
||||||
this.stageToBlock = stageToBlock;
|
this.stageToBlock = stageToBlock;
|
||||||
|
|
|
@ -2536,7 +2536,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
indexDoc(primary, "_doc", "0", "{\"foo\" : \"bar\"}");
|
indexDoc(primary, "_doc", "0", "{\"foo\" : \"bar\"}");
|
||||||
IndexShard replica = newShard(primary.shardId(), false, "n2", metadata, null);
|
IndexShard replica = newShard(primary.shardId(), false, "n2", metadata, null);
|
||||||
recoverReplica(replica, primary, (shard, discoveryNode) ->
|
recoverReplica(replica, primary, (shard, discoveryNode) ->
|
||||||
new RecoveryTarget(shard, discoveryNode, recoveryListener) {
|
new RecoveryTarget(shard, discoveryNode, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void indexTranslogOperations(
|
public void indexTranslogOperations(
|
||||||
final List<Translog.Operation> operations,
|
final List<Translog.Operation> operations,
|
||||||
|
@ -2643,7 +2643,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
// Shard is still inactive since we haven't started recovering yet
|
// Shard is still inactive since we haven't started recovering yet
|
||||||
assertFalse(replica.isActive());
|
assertFalse(replica.isActive());
|
||||||
recoverReplica(replica, primary, (shard, discoveryNode) ->
|
recoverReplica(replica, primary, (shard, discoveryNode) ->
|
||||||
new RecoveryTarget(shard, discoveryNode, recoveryListener) {
|
new RecoveryTarget(shard, discoveryNode, null, recoveryListener) {
|
||||||
@Override
|
@Override
|
||||||
public void indexTranslogOperations(
|
public void indexTranslogOperations(
|
||||||
final List<Translog.Operation> operations,
|
final List<Translog.Operation> operations,
|
||||||
|
@ -2699,7 +2699,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
||||||
assertListenerCalled.accept(replica);
|
assertListenerCalled.accept(replica);
|
||||||
recoverReplica(replica, primary, (shard, discoveryNode) ->
|
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
|
// we're only checking that listeners are called when the engine is open, before there is no point
|
||||||
@Override
|
@Override
|
||||||
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
||||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.index.seqno.RetentionLeaseSyncer;
|
||||||
import org.elasticsearch.index.shard.PrimaryReplicaSyncer;
|
import org.elasticsearch.index.shard.PrimaryReplicaSyncer;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||||
|
import org.elasticsearch.indices.recovery.SnapshotFilesProvider;
|
||||||
import org.elasticsearch.repositories.RepositoriesService;
|
import org.elasticsearch.repositories.RepositoriesService;
|
||||||
import org.elasticsearch.threadpool.TestThreadPool;
|
import org.elasticsearch.threadpool.TestThreadPool;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
@ -487,7 +488,7 @@ public class IndicesClusterStateServiceRandomUpdatesTests extends AbstractIndice
|
||||||
final RepositoriesService repositoriesService = new RepositoriesService(settings, clusterService,
|
final RepositoriesService repositoriesService = new RepositoriesService(settings, clusterService,
|
||||||
transportService, Collections.emptyMap(), Collections.emptyMap(), threadPool);
|
transportService, Collections.emptyMap(), Collections.emptyMap(), threadPool);
|
||||||
final PeerRecoveryTargetService recoveryTargetService = new PeerRecoveryTargetService(threadPool,
|
final PeerRecoveryTargetService recoveryTargetService = new PeerRecoveryTargetService(threadPool,
|
||||||
transportService, null, clusterService);
|
transportService, null, clusterService, mock(SnapshotFilesProvider.class));
|
||||||
final ShardStateAction shardStateAction = mock(ShardStateAction.class);
|
final ShardStateAction shardStateAction = mock(ShardStateAction.class);
|
||||||
final PrimaryReplicaSyncer primaryReplicaSyncer = mock(PrimaryReplicaSyncer.class);
|
final PrimaryReplicaSyncer primaryReplicaSyncer = mock(PrimaryReplicaSyncer.class);
|
||||||
final NodeClient client = mock(NodeClient.class);
|
final NodeClient client = mock(NodeClient.class);
|
||||||
|
|
|
@ -8,8 +8,12 @@
|
||||||
|
|
||||||
package org.elasticsearch.indices.recovery;
|
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.IOContext;
|
||||||
import org.apache.lucene.store.IndexInput;
|
import org.apache.lucene.store.IndexInput;
|
||||||
|
import org.apache.lucene.store.OutputStreamIndexOutput;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
|
||||||
|
@ -23,8 +27,12 @@ import org.elasticsearch.common.Randomness;
|
||||||
import org.elasticsearch.common.UUIDs;
|
import org.elasticsearch.common.UUIDs;
|
||||||
import org.elasticsearch.common.bytes.BytesArray;
|
import org.elasticsearch.common.bytes.BytesArray;
|
||||||
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
||||||
|
import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
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.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.core.Tuple;
|
||||||
import org.elasticsearch.core.internal.io.IOUtils;
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
import org.elasticsearch.index.engine.NoOpEngine;
|
import org.elasticsearch.index.engine.NoOpEngine;
|
||||||
import org.elasticsearch.index.mapper.SourceToParse;
|
import org.elasticsearch.index.mapper.SourceToParse;
|
||||||
|
@ -32,27 +40,42 @@ import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardTestCase;
|
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.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
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.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.concurrent.ArrayBlockingQueue;
|
import java.util.concurrent.ArrayBlockingQueue;
|
||||||
import java.util.concurrent.BlockingQueue;
|
import java.util.concurrent.BlockingQueue;
|
||||||
import java.util.concurrent.CyclicBarrier;
|
import java.util.concurrent.CyclicBarrier;
|
||||||
|
import java.util.concurrent.ExecutionException;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.LongStream;
|
import java.util.stream.LongStream;
|
||||||
|
|
||||||
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
import static org.hamcrest.Matchers.empty;
|
import static org.hamcrest.Matchers.empty;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
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.hamcrest.Matchers.sameInstance;
|
||||||
|
import static org.mockito.Mockito.mock;
|
||||||
|
|
||||||
public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
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 {
|
public void testWriteFileChunksConcurrently() throws Exception {
|
||||||
IndexShard sourceShard = newStartedShard(true);
|
IndexShard sourceShard = newStartedShard(true);
|
||||||
|
@ -70,7 +93,7 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
final DiscoveryNode pNode = getFakeDiscoNode(sourceShard.routingEntry().currentNodeId());
|
final DiscoveryNode pNode = getFakeDiscoNode(sourceShard.routingEntry().currentNodeId());
|
||||||
final DiscoveryNode rNode = getFakeDiscoNode(targetShard.routingEntry().currentNodeId());
|
final DiscoveryNode rNode = getFakeDiscoNode(targetShard.routingEntry().currentNodeId());
|
||||||
targetShard.markAsRecovering("test-peer-recovery", new RecoveryState(targetShard.routingEntry(), rNode, pNode));
|
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<Void> receiveFileInfoFuture = new PlainActionFuture<>();
|
final PlainActionFuture<Void> receiveFileInfoFuture = new PlainActionFuture<>();
|
||||||
recoveryTarget.receiveFileInfo(
|
recoveryTarget.receiveFileInfo(
|
||||||
mdFiles.stream().map(StoreFileMetadata::name).collect(Collectors.toList()),
|
mdFiles.stream().map(StoreFileMetadata::name).collect(Collectors.toList()),
|
||||||
|
@ -270,7 +293,7 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
shard.prepareForIndexRecovery();
|
shard.prepareForIndexRecovery();
|
||||||
long startingSeqNo = shard.recoverLocallyUpToGlobalCheckpoint();
|
long startingSeqNo = shard.recoverLocallyUpToGlobalCheckpoint();
|
||||||
shard.store().markStoreCorrupted(new IOException("simulated"));
|
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);
|
StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(logger, rNode, recoveryTarget, startingSeqNo);
|
||||||
assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO));
|
assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
assertThat(request.metadataSnapshot().size(), equalTo(0));
|
assertThat(request.metadataSnapshot().size(), equalTo(0));
|
||||||
|
@ -297,7 +320,7 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE));
|
shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE));
|
||||||
shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode));
|
shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode));
|
||||||
shard.prepareForIndexRecovery();
|
shard.prepareForIndexRecovery();
|
||||||
RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null);
|
RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null, null);
|
||||||
StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(
|
StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(
|
||||||
logger, rNode, recoveryTarget, randomNonNegativeLong());
|
logger, rNode, recoveryTarget, randomNonNegativeLong());
|
||||||
assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO));
|
assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
@ -305,4 +328,286 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
recoveryTarget.decRef();
|
recoveryTarget.decRef();
|
||||||
closeShards(shard);
|
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<StoreFileMetadata, byte[]> 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<Long> 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<Void> writeSnapshotFileFuture = PlainActionFuture.newFuture();
|
||||||
|
recoveryTarget.restoreFileFromSnapshot(repositoryName, indexId, fileInfo, writeSnapshotFileFuture);
|
||||||
|
writeSnapshotFileFuture.get();
|
||||||
|
|
||||||
|
Optional<String> 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<StoreFileMetadata, byte[]> 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<Long> 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<Void> 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<Void> 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<StoreFileMetadata, byte[]> 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<Long> 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<Void> 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<StoreFileMetadata, byte[]> 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
|
||||||
|
);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -42,6 +42,8 @@ import org.elasticsearch.common.lucene.store.IndexOutputOutputStream;
|
||||||
import org.elasticsearch.common.lucene.uid.Versions;
|
import org.elasticsearch.common.lucene.uid.Versions;
|
||||||
import org.elasticsearch.common.settings.ClusterSettings;
|
import org.elasticsearch.common.settings.ClusterSettings;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||||
|
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||||
import org.elasticsearch.common.util.CancellableThreads;
|
import org.elasticsearch.common.util.CancellableThreads;
|
||||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
@ -66,12 +68,14 @@ import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.IndexShardRelocatedException;
|
import org.elasticsearch.index.shard.IndexShardRelocatedException;
|
||||||
import org.elasticsearch.index.shard.IndexShardState;
|
import org.elasticsearch.index.shard.IndexShardState;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.indices.recovery.plan.RecoveryPlannerService;
|
import org.elasticsearch.indices.recovery.plan.RecoveryPlannerService;
|
||||||
import org.elasticsearch.indices.recovery.plan.ShardRecoveryPlan;
|
import org.elasticsearch.indices.recovery.plan.ShardRecoveryPlan;
|
||||||
import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService;
|
import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService;
|
||||||
|
import org.elasticsearch.repositories.IndexId;
|
||||||
import org.elasticsearch.test.CorruptionUtils;
|
import org.elasticsearch.test.CorruptionUtils;
|
||||||
import org.elasticsearch.test.DummyShardLock;
|
import org.elasticsearch.test.DummyShardLock;
|
||||||
import org.elasticsearch.test.ESTestCase;
|
import org.elasticsearch.test.ESTestCase;
|
||||||
|
@ -105,10 +109,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicInteger;
|
import java.util.concurrent.atomic.AtomicInteger;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.function.Function;
|
||||||
import java.util.function.IntSupplier;
|
import java.util.function.IntSupplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.zip.CRC32;
|
import java.util.zip.CRC32;
|
||||||
|
|
||||||
|
import static java.util.Collections.emptyList;
|
||||||
import static java.util.Collections.emptyMap;
|
import static java.util.Collections.emptyMap;
|
||||||
import static java.util.Collections.emptySet;
|
import static java.util.Collections.emptySet;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
|
@ -194,6 +200,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
||||||
between(1, 5),
|
between(1, 5),
|
||||||
between(1, 5),
|
between(1, 5),
|
||||||
|
between(1, 5),
|
||||||
|
false,
|
||||||
recoveryPlannerService);
|
recoveryPlannerService);
|
||||||
PlainActionFuture<Void> sendFilesFuture = new PlainActionFuture<>();
|
PlainActionFuture<Void> sendFilesFuture = new PlainActionFuture<>();
|
||||||
handler.sendFiles(store, metas.toArray(new StoreFileMetadata[0]), () -> 0, sendFilesFuture);
|
handler.sendFiles(store, metas.toArray(new StoreFileMetadata[0]), () -> 0, sendFilesFuture);
|
||||||
|
@ -257,9 +265,9 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()),
|
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<RecoverySourceHandler.SendSnapshotResult> future = new PlainActionFuture<>();
|
PlainActionFuture<RecoverySourceHandler.SendSnapshotResult> future = new PlainActionFuture<>();
|
||||||
handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()),
|
handler.phase2(startingSeqNo, endingSeqNo, newTranslogSnapshot(operations, emptyList()),
|
||||||
randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future);
|
randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future);
|
||||||
final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1);
|
final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1);
|
||||||
RecoverySourceHandler.SendSnapshotResult result = future.actionGet();
|
RecoverySourceHandler.SendSnapshotResult result = future.actionGet();
|
||||||
|
@ -299,11 +307,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()),
|
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<RecoverySourceHandler.SendSnapshotResult> future = new PlainActionFuture<>();
|
PlainActionFuture<RecoverySourceHandler.SendSnapshotResult> future = new PlainActionFuture<>();
|
||||||
final long startingSeqNo = randomLongBetween(0, ops.size() - 1L);
|
final long startingSeqNo = randomLongBetween(0, ops.size() - 1L);
|
||||||
final long endingSeqNo = randomLongBetween(startingSeqNo, 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);
|
randomNonNegativeLong(), randomNonNegativeLong(), RetentionLeases.EMPTY, randomNonNegativeLong(), future);
|
||||||
if (wasFailed.get()) {
|
if (wasFailed.get()) {
|
||||||
final RecoveryEngineException error = expectThrows(RecoveryEngineException.class, future::actionGet);
|
final RecoveryEngineException error = expectThrows(RecoveryEngineException.class, future::actionGet);
|
||||||
|
@ -353,7 +361,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
List<Translog.Operation> skipOperations = randomSubsetOf(operations);
|
List<Translog.Operation> skipOperations = randomSubsetOf(operations);
|
||||||
Translog.Snapshot snapshot = newTranslogSnapshot(operations, skipOperations);
|
Translog.Snapshot snapshot = newTranslogSnapshot(operations, skipOperations);
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(shard, new AsyncRecoveryTarget(target, recoveryExecutor),
|
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,
|
handler.phase2(startingSeqNo, endingSeqNo, snapshot, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes, retentionLeases,
|
||||||
mappingVersion, sendFuture);
|
mappingVersion, sendFuture);
|
||||||
RecoverySourceHandler.SendSnapshotResult sendSnapshotResult = sendFuture.actionGet();
|
RecoverySourceHandler.SendSnapshotResult sendSnapshotResult = sendFuture.actionGet();
|
||||||
|
@ -428,7 +437,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool,
|
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
|
@Override
|
||||||
protected void failEngine(IOException cause) {
|
protected void failEngine(IOException cause) {
|
||||||
assertFalse(failedEngine.get());
|
assertFalse(failedEngine.get());
|
||||||
|
@ -485,7 +495,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(target, recoveryExecutor), threadPool,
|
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
|
@Override
|
||||||
protected void failEngine(IOException cause) {
|
protected void failEngine(IOException cause) {
|
||||||
assertFalse(failedEngine.get());
|
assertFalse(failedEngine.get());
|
||||||
|
@ -535,13 +546,16 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
||||||
final AtomicBoolean phase2Called = new AtomicBoolean();
|
final AtomicBoolean phase2Called = new AtomicBoolean();
|
||||||
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
||||||
shard,
|
shard,
|
||||||
mock(RecoveryTargetHandler.class),
|
mock(RecoveryTargetHandler.class),
|
||||||
threadPool,
|
threadPool,
|
||||||
request,
|
request,
|
||||||
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
Math.toIntExact(recoverySettings.getChunkSize().getBytes()),
|
||||||
between(1, 8), between(1, 8),
|
between(1, 8),
|
||||||
recoveryPlannerService) {
|
between(1, 8),
|
||||||
|
between(1, 5),
|
||||||
|
false,
|
||||||
|
recoveryPlannerService) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
void phase1(IndexCommit snapshot, long startingSeqNo, IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
||||||
|
@ -619,7 +633,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
final int maxConcurrentChunks = between(1, 8);
|
final int maxConcurrentChunks = between(1, 8);
|
||||||
final int chunkSize = between(1, 32);
|
final int chunkSize = between(1, 32);
|
||||||
final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, threadPool, getStartRecoveryRequest(),
|
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);
|
Store store = newStore(createTempDir(), false);
|
||||||
List<StoreFileMetadata> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
List<StoreFileMetadata> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
||||||
int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum();
|
int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum();
|
||||||
|
@ -677,7 +691,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
final int maxConcurrentChunks = between(1, 4);
|
final int maxConcurrentChunks = between(1, 4);
|
||||||
final int chunkSize = between(1, 16);
|
final int chunkSize = between(1, 16);
|
||||||
final RecoverySourceHandler handler = new RecoverySourceHandler(null, new AsyncRecoveryTarget(recoveryTarget, recoveryExecutor),
|
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);
|
Store store = newStore(createTempDir(), false);
|
||||||
List<StoreFileMetadata> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
List<StoreFileMetadata> files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20));
|
||||||
int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum();
|
int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum();
|
||||||
|
@ -757,8 +772,16 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
final StartRecoveryRequest startRecoveryRequest = getStartRecoveryRequest();
|
final StartRecoveryRequest startRecoveryRequest = getStartRecoveryRequest();
|
||||||
final RecoverySourceHandler handler = new RecoverySourceHandler(
|
final RecoverySourceHandler handler = new RecoverySourceHandler(shard,
|
||||||
shard, recoveryTarget, threadPool, startRecoveryRequest, between(1, 16), between(1, 4), between(1, 4), recoveryPlannerService) {
|
recoveryTarget,
|
||||||
|
threadPool,
|
||||||
|
startRecoveryRequest,
|
||||||
|
between(1, 16),
|
||||||
|
between(1, 4),
|
||||||
|
between(1, 4),
|
||||||
|
between(1, 4),
|
||||||
|
false,
|
||||||
|
recoveryPlannerService) {
|
||||||
@Override
|
@Override
|
||||||
void createRetentionLease(long startingSeqNo, ActionListener<RetentionLease> listener) {
|
void createRetentionLease(long startingSeqNo, ActionListener<RetentionLease> listener) {
|
||||||
final String leaseId = ReplicationTracker.getPeerRecoveryRetentionLeaseId(startRecoveryRequest.targetNode().getId());
|
final String leaseId = ReplicationTracker.getPeerRecoveryRetentionLeaseId(startRecoveryRequest.targetNode().getId());
|
||||||
|
@ -794,6 +817,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
between(1, 16),
|
between(1, 16),
|
||||||
between(1, 4),
|
between(1, 4),
|
||||||
between(1, 4),
|
between(1, 4),
|
||||||
|
between(1, 4),
|
||||||
|
false,
|
||||||
recoveryPlannerService);
|
recoveryPlannerService);
|
||||||
|
|
||||||
String syncId = UUIDs.randomBase64UUID();
|
String syncId = UUIDs.randomBase64UUID();
|
||||||
|
@ -832,35 +857,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
writer.close();
|
writer.close();
|
||||||
when(shard.state()).thenReturn(IndexShardState.STARTED);
|
when(shard.state()).thenReturn(IndexShardState.STARTED);
|
||||||
|
|
||||||
TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() {
|
TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler();
|
||||||
@Override
|
|
||||||
public void receiveFileInfo(List<String> phase1FileNames,
|
|
||||||
List<Long> phase1FileSizes,
|
|
||||||
List<String> phase1ExistingFileNames,
|
|
||||||
List<Long> phase1ExistingFileSizes,
|
|
||||||
int totalTranslogOps,
|
|
||||||
ActionListener<Void> listener) {
|
|
||||||
listener.onResponse(null);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void cleanFiles(int totalTranslogOps,
|
|
||||||
long globalCheckpoint,
|
|
||||||
Store.MetadataSnapshot sourceMetadata,
|
|
||||||
ActionListener<Void> listener) {
|
|
||||||
listener.onResponse(null);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void writeFileChunk(StoreFileMetadata fileMetadata,
|
|
||||||
long position,
|
|
||||||
ReleasableBytesReference content,
|
|
||||||
boolean lastChunk,
|
|
||||||
int totalTranslogOps,
|
|
||||||
ActionListener<Void> listener) {
|
|
||||||
listener.onResponse(null);
|
|
||||||
}
|
|
||||||
};
|
|
||||||
AtomicReference<ShardRecoveryPlan> computedRecoveryPlanRef = new AtomicReference<>();
|
AtomicReference<ShardRecoveryPlan> computedRecoveryPlanRef = new AtomicReference<>();
|
||||||
RecoverySourceHandler handler = new RecoverySourceHandler(
|
RecoverySourceHandler handler = new RecoverySourceHandler(
|
||||||
shard,
|
shard,
|
||||||
|
@ -870,6 +867,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
between(1, 16),
|
between(1, 16),
|
||||||
between(1, 4),
|
between(1, 4),
|
||||||
between(1, 4),
|
between(1, 4),
|
||||||
|
between(1, 4),
|
||||||
|
true,
|
||||||
recoveryPlannerService
|
recoveryPlannerService
|
||||||
) {
|
) {
|
||||||
@Override
|
@Override
|
||||||
|
@ -888,10 +887,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
};
|
};
|
||||||
PlainActionFuture<RecoverySourceHandler.SendFileResult> phase1Listener = PlainActionFuture.newFuture();
|
PlainActionFuture<RecoverySourceHandler.SendFileResult> phase1Listener = PlainActionFuture.newFuture();
|
||||||
IndexCommit indexCommit = DirectoryReader.listCommits(dir).get(0);
|
IndexCommit indexCommit = DirectoryReader.listCommits(dir).get(0);
|
||||||
handler.phase1(indexCommit,
|
handler.phase1(indexCommit, 0, () -> 0, phase1Listener);
|
||||||
0,
|
|
||||||
() -> 0,
|
|
||||||
phase1Listener);
|
|
||||||
phase1Listener.get();
|
phase1Listener.get();
|
||||||
|
|
||||||
ShardRecoveryPlan computedRecoveryPlan = computedRecoveryPlanRef.get();
|
ShardRecoveryPlan computedRecoveryPlan = computedRecoveryPlanRef.get();
|
||||||
|
@ -905,6 +901,361 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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<String> fileNamesToBeRecoveredFromSnapshot = snapshotFilesToRecover.getSnapshotFiles()
|
||||||
|
.stream()
|
||||||
|
.map(fileInfo -> fileInfo.metadata().name())
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
|
||||||
|
final List<String> sourceFilesToRecover =
|
||||||
|
shardRecoveryPlan.getSourceFilesToRecover().stream().map(StoreFileMetadata::name).collect(Collectors.toList());
|
||||||
|
|
||||||
|
Set<String> filesFailedToDownload = Collections.synchronizedSet(new HashSet<>());
|
||||||
|
Set<String> filesRecoveredFromSource = Collections.synchronizedSet(new HashSet<>());
|
||||||
|
Set<String> filesRecoveredFromSnapshot = Collections.synchronizedSet(new HashSet<>());
|
||||||
|
TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() {
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> 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<Void> 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<RetentionLease> listener) {
|
||||||
|
listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
PlainActionFuture<RecoverySourceHandler.SendFileResult> future = PlainActionFuture.newFuture();
|
||||||
|
handler.recoverFilesFromSourceAndSnapshot(shardRecoveryPlan,
|
||||||
|
store,
|
||||||
|
mock(StopWatch.class),
|
||||||
|
future
|
||||||
|
);
|
||||||
|
future.actionGet();
|
||||||
|
|
||||||
|
|
||||||
|
Set<String> 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<RecoverSnapshotFileResponse> unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>();
|
||||||
|
TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() {
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
unrespondedRecoverSnapshotFiles.add(new RecoverSnapshotFileResponse(snapshotFile, listener));
|
||||||
|
recoverSnapshotFileRequests.incrementAndGet();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeFileChunk(StoreFileMetadata fileMetadata,
|
||||||
|
long position,
|
||||||
|
ReleasableBytesReference content,
|
||||||
|
boolean lastChunk,
|
||||||
|
int totalTranslogOps,
|
||||||
|
ActionListener<Void> 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<RetentionLease> listener) {
|
||||||
|
listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
PlainActionFuture<RecoverySourceHandler.SendFileResult> 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<RecoverSnapshotFileResponse> 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<RecoverSnapshotFileResponse> unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>();
|
||||||
|
TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() {
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> 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<Void> 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<RetentionLease> listener) {
|
||||||
|
listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
PlainActionFuture<RecoverySourceHandler.SendFileResult> 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<RecoverSnapshotFileResponse> unrespondedRecoverSnapshotFiles = new CopyOnWriteArrayList<>();
|
||||||
|
TestRecoveryTargetHandler recoveryTarget = new Phase1RecoveryTargetHandler() {
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
unrespondedRecoverSnapshotFiles.add(new RecoverSnapshotFileResponse(snapshotFile, listener));
|
||||||
|
downloadSnapshotFileReceived.countDown();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeFileChunk(StoreFileMetadata fileMetadata,
|
||||||
|
long position,
|
||||||
|
ReleasableBytesReference content,
|
||||||
|
boolean lastChunk,
|
||||||
|
int totalTranslogOps,
|
||||||
|
ActionListener<Void> 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<RetentionLease> listener) {
|
||||||
|
listener.onResponse(new RetentionLease("id", startingSeqNo, 0, "test"));
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
PlainActionFuture<RecoverySourceHandler.SendFileResult> 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<StoreFileMetadata> 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<StoreFileMetadata> filesToRecoverFromSource = sourceFiles.subList(0, sourceFileCount);
|
||||||
|
List<StoreFileMetadata> filesToRecoverFromSnapshot = sourceFiles.subList(sourceFileCount, sourceFiles.size());
|
||||||
|
|
||||||
|
List<BlobStoreIndexShardSnapshot.FileInfo> 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) {
|
private Store.MetadataSnapshot newMetadataSnapshot(String syncId, String localCheckpoint, String maxSeqNo, int numDocs) {
|
||||||
Map<String, String> userData = new HashMap<>();
|
Map<String, String> userData = new HashMap<>();
|
||||||
userData.put(Engine.SYNC_COMMIT_ID, syncId);
|
userData.put(Engine.SYNC_COMMIT_ID, syncId);
|
||||||
|
@ -992,12 +1343,47 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
ActionListener<Void> listener) {
|
ActionListener<Void> listener) {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository, IndexId indexId, BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk,
|
public void writeFileChunk(StoreFileMetadata fileMetadata, long position, ReleasableBytesReference content, boolean lastChunk,
|
||||||
int totalTranslogOps, ActionListener<Void> listener) {
|
int totalTranslogOps, ActionListener<Void> listener) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class Phase1RecoveryTargetHandler extends TestRecoveryTargetHandler {
|
||||||
|
@Override
|
||||||
|
public void receiveFileInfo(List<String> phase1FileNames,
|
||||||
|
List<Long> phase1FileSizes,
|
||||||
|
List<String> phase1ExistingFileNames,
|
||||||
|
List<Long> phase1ExistingFileSizes,
|
||||||
|
int totalTranslogOps,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
listener.onResponse(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void writeFileChunk(StoreFileMetadata fileMetadata,
|
||||||
|
long position,
|
||||||
|
ReleasableBytesReference content,
|
||||||
|
boolean lastChunk,
|
||||||
|
int totalTranslogOps,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
listener.onResponse(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void cleanFiles(int totalTranslogOps,
|
||||||
|
long globalCheckpoint,
|
||||||
|
Store.MetadataSnapshot sourceMetadata,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
listener.onResponse(null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Translog.Snapshot newTranslogSnapshot(List<Translog.Operation> operations, List<Translog.Operation> operationsToSkip) {
|
private Translog.Snapshot newTranslogSnapshot(List<Translog.Operation> operations, List<Translog.Operation> operationsToSkip) {
|
||||||
Iterator<Translog.Operation> iterator = operations.iterator();
|
Iterator<Translog.Operation> iterator = operations.iterator();
|
||||||
return new Translog.Snapshot() {
|
return new Translog.Snapshot() {
|
||||||
|
@ -1051,4 +1437,14 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
}
|
}
|
||||||
return operations;
|
return operations;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static class RecoverSnapshotFileResponse {
|
||||||
|
final BlobStoreIndexShardSnapshot.FileInfo fileInfo;
|
||||||
|
final ActionListener<Void> listener;
|
||||||
|
|
||||||
|
RecoverSnapshotFileResponse(BlobStoreIndexShardSnapshot.FileInfo fileInfo, ActionListener<Void> listener) {
|
||||||
|
this.fileInfo = fileInfo;
|
||||||
|
this.listener = listener;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -9,6 +9,7 @@
|
||||||
package org.elasticsearch.indices.recovery;
|
package org.elasticsearch.indices.recovery;
|
||||||
|
|
||||||
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
|
import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
|
||||||
|
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
import org.apache.lucene.index.DirectoryReader;
|
||||||
import org.apache.lucene.index.IndexCommit;
|
import org.apache.lucene.index.IndexCommit;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
|
@ -254,7 +255,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
||||||
}
|
}
|
||||||
IndexShard replicaShard = newShard(primaryShard.shardId(), false);
|
IndexShard replicaShard = newShard(primaryShard.shardId(), false);
|
||||||
updateMappings(replicaShard, primaryShard.indexSettings().getIndexMetadata());
|
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
|
@Override
|
||||||
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
public void prepareForTranslogOperations(int totalTranslogOps, ActionListener<Void> listener) {
|
||||||
super.prepareForTranslogOperations(totalTranslogOps, listener);
|
super.prepareForTranslogOperations(totalTranslogOps, listener);
|
||||||
|
@ -365,17 +367,19 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
||||||
allowShardFailures();
|
allowShardFailures();
|
||||||
IndexShard replica = group.addReplica();
|
IndexShard replica = group.addReplica();
|
||||||
expectThrows(Exception.class, () -> group.recoverReplica(replica,
|
expectThrows(Exception.class, () -> group.recoverReplica(replica,
|
||||||
(shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, new PeerRecoveryTargetService.RecoveryListener() {
|
(shard, sourceNode) -> {
|
||||||
@Override
|
return new RecoveryTarget(shard, sourceNode, null, new PeerRecoveryTargetService.RecoveryListener() {
|
||||||
public void onRecoveryDone(RecoveryState state, ShardLongFieldRange timestampMillisFieldRange) {
|
@Override
|
||||||
throw new AssertionError("recovery must fail");
|
public void onRecoveryDone(RecoveryState state, ShardLongFieldRange timestampMillisFieldRange) {
|
||||||
}
|
throw new AssertionError("recovery must fail");
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) {
|
public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) {
|
||||||
assertThat(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), equalTo("simulated"));
|
assertThat(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), equalTo("simulated"));
|
||||||
}
|
}
|
||||||
})));
|
});
|
||||||
|
}));
|
||||||
expectThrows(AlreadyClosedException.class, () -> replica.refresh("test"));
|
expectThrows(AlreadyClosedException.class, () -> replica.refresh("test"));
|
||||||
group.removeReplica(replica);
|
group.removeReplica(replica);
|
||||||
replica.store().close();
|
replica.store().close();
|
||||||
|
|
|
@ -12,13 +12,13 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.core.TimeValue;
|
import org.elasticsearch.core.TimeValue;
|
||||||
import org.elasticsearch.index.replication.ESIndexLevelReplicationTestCase;
|
import org.elasticsearch.index.replication.ESIndexLevelReplicationTestCase;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardLongFieldRange;
|
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.index.shard.ShardLongFieldRange;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
|
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||||
import org.elasticsearch.indices.recovery.RecoveriesCollection;
|
import org.elasticsearch.indices.recovery.RecoveriesCollection;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
||||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
|
||||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||||
|
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
@ -150,6 +150,6 @@ public class RecoveriesCollectionTests extends ESIndexLevelReplicationTestCase {
|
||||||
final DiscoveryNode rNode = getDiscoveryNode(indexShard.routingEntry().currentNodeId());
|
final DiscoveryNode rNode = getDiscoveryNode(indexShard.routingEntry().currentNodeId());
|
||||||
indexShard.markAsRecovering("remote", new RecoveryState(indexShard.routingEntry(), sourceNode, rNode));
|
indexShard.markAsRecovering("remote", new RecoveryState(indexShard.routingEntry(), sourceNode, rNode));
|
||||||
indexShard.prepareForIndexRecovery();
|
indexShard.prepareForIndexRecovery();
|
||||||
return collection.startRecovery(indexShard, sourceNode, listener, timeValue);
|
return collection.startRecovery(indexShard, sourceNode, null, listener, timeValue);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -155,6 +155,7 @@ import org.elasticsearch.indices.cluster.IndicesClusterStateService;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
|
import org.elasticsearch.indices.recovery.PeerRecoverySourceService;
|
||||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||||
|
import org.elasticsearch.indices.recovery.SnapshotFilesProvider;
|
||||||
import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService;
|
import org.elasticsearch.indices.recovery.plan.SourceOnlyRecoveryPlannerService;
|
||||||
import org.elasticsearch.ingest.IngestService;
|
import org.elasticsearch.ingest.IngestService;
|
||||||
import org.elasticsearch.monitor.StatusInfo;
|
import org.elasticsearch.monitor.StatusInfo;
|
||||||
|
@ -1816,12 +1817,13 @@ public class SnapshotResiliencyTests extends ESTestCase {
|
||||||
SourceOnlyRecoveryPlannerService.INSTANCE
|
SourceOnlyRecoveryPlannerService.INSTANCE
|
||||||
);
|
);
|
||||||
|
|
||||||
|
final SnapshotFilesProvider snapshotFilesProvider = new SnapshotFilesProvider(repositoriesService);
|
||||||
indicesClusterStateService = new IndicesClusterStateService(
|
indicesClusterStateService = new IndicesClusterStateService(
|
||||||
settings,
|
settings,
|
||||||
indicesService,
|
indicesService,
|
||||||
clusterService,
|
clusterService,
|
||||||
threadPool,
|
threadPool,
|
||||||
new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService),
|
new PeerRecoveryTargetService(threadPool, transportService, recoverySettings, clusterService, snapshotFilesProvider),
|
||||||
shardStateAction,
|
shardStateAction,
|
||||||
repositoriesService,
|
repositoriesService,
|
||||||
mock(SearchService.class),
|
mock(SearchService.class),
|
||||||
|
|
|
@ -50,11 +50,11 @@ import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||||
import org.elasticsearch.common.collect.Iterators;
|
import org.elasticsearch.common.collect.Iterators;
|
||||||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||||
import org.elasticsearch.common.io.stream.StreamInput;
|
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.Releasable;
|
||||||
import org.elasticsearch.core.Releasables;
|
import org.elasticsearch.core.Releasables;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
|
||||||
import org.elasticsearch.core.TimeValue;
|
import org.elasticsearch.core.TimeValue;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.engine.DocIdSeqNoAndSource;
|
import org.elasticsearch.index.engine.DocIdSeqNoAndSource;
|
||||||
|
@ -394,7 +394,8 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
}
|
}
|
||||||
|
|
||||||
public void recoverReplica(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<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier)
|
public void recoverReplica(IndexShard replica, BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier)
|
||||||
|
|
|
@ -582,7 +582,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
/** recovers a replica from the given primary **/
|
/** recovers a replica from the given primary **/
|
||||||
protected void recoverReplica(IndexShard replica, IndexShard primary, boolean startReplica) throws IOException {
|
protected void recoverReplica(IndexShard replica, IndexShard primary, boolean startReplica) throws IOException {
|
||||||
recoverReplica(replica, primary,
|
recoverReplica(replica, primary,
|
||||||
(r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener),
|
(r, sourceNode) -> new RecoveryTarget(r, sourceNode, null, recoveryListener),
|
||||||
true, startReplica);
|
true, startReplica);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -638,7 +638,7 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
final RecoveryPlannerService recoveryPlannerService = SourceOnlyRecoveryPlannerService.INSTANCE;
|
final RecoveryPlannerService recoveryPlannerService = SourceOnlyRecoveryPlannerService.INSTANCE;
|
||||||
final RecoverySourceHandler recovery = new RecoverySourceHandler(primary,
|
final RecoverySourceHandler recovery = new RecoverySourceHandler(primary,
|
||||||
new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool,
|
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,
|
primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null,
|
||||||
currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable);
|
currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable);
|
||||||
try {
|
try {
|
||||||
|
|
|
@ -12,9 +12,11 @@ import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
import org.elasticsearch.common.bytes.ReleasableBytesReference;
|
||||||
import org.elasticsearch.index.seqno.ReplicationTracker;
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
|
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.index.store.StoreFileMetadata;
|
import org.elasticsearch.index.store.StoreFileMetadata;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
|
import org.elasticsearch.repositories.IndexId;
|
||||||
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.Executor;
|
import java.util.concurrent.Executor;
|
||||||
|
@ -82,4 +84,12 @@ public class AsyncRecoveryTarget implements RecoveryTargetHandler {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void restoreFileFromSnapshot(String repository,
|
||||||
|
IndexId indexId,
|
||||||
|
BlobStoreIndexShardSnapshot.FileInfo snapshotFile,
|
||||||
|
ActionListener<Void> listener) {
|
||||||
|
executor.execute(() -> target.restoreFileFromSnapshot(repository, indexId, snapshotFile, listener));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -8,6 +8,7 @@ package org.elasticsearch.xpack.ccr.action;
|
||||||
|
|
||||||
import com.carrotsearch.hppc.LongHashSet;
|
import com.carrotsearch.hppc.LongHashSet;
|
||||||
import com.carrotsearch.hppc.LongSet;
|
import com.carrotsearch.hppc.LongSet;
|
||||||
|
|
||||||
import org.apache.lucene.store.IOContext;
|
import org.apache.lucene.store.IOContext;
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
|
@ -26,13 +27,13 @@ import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||||
import org.elasticsearch.common.UUIDs;
|
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.lucene.Lucene;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||||
import org.elasticsearch.common.unit.ByteSizeValue;
|
import org.elasticsearch.common.unit.ByteSizeValue;
|
||||||
|
import org.elasticsearch.core.Releasable;
|
||||||
import org.elasticsearch.core.TimeValue;
|
import org.elasticsearch.core.TimeValue;
|
||||||
|
import org.elasticsearch.core.Tuple;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.EngineFactory;
|
import org.elasticsearch.index.engine.EngineFactory;
|
||||||
|
@ -371,7 +372,7 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
||||||
// We need to recover the replica async to release the main thread for the following task to fill missing
|
// 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.
|
// operations between the local checkpoint and max_seq_no which the recovering replica is waiting for.
|
||||||
recoveryFuture = group.asyncRecoverReplica(newReplica,
|
recoveryFuture = group.asyncRecoverReplica(newReplica,
|
||||||
(shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, recoveryListener) {});
|
(shard, sourceNode) -> new RecoveryTarget(shard, sourceNode, null, recoveryListener) {});
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (recoveryFuture != null) {
|
if (recoveryFuture != null) {
|
||||||
|
|
Loading…
Add table
Add a link
Reference in a new issue