Skip to content

Ensure relocating shards establish peer recovery retention leases #50486

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 3 commits into from
Dec 26, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1334,11 +1334,7 @@ public synchronized void activateWithPrimaryContext(PrimaryContext primaryContex
// note that if there was no cluster state update between start of the engine of this shard and the call to
// initializeWithPrimaryContext, we might still have missed a cluster state update. This is best effort.
runAfter.run();

if (indexSettings.isSoftDeleteEnabled()) {
addPeerRecoveryRetentionLeaseForSolePrimary();
}

addPeerRecoveryRetentionLeaseForSolePrimary();
assert invariant();
}

Expand Down
54 changes: 54 additions & 0 deletions server/src/test/java/org/elasticsearch/recovery/RelocationIT.java
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,14 @@
package org.elasticsearch.recovery;

import com.carrotsearch.hppc.IntHashSet;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import com.carrotsearch.hppc.procedures.IntProcedure;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.util.English;
import org.elasticsearch.action.ActionFuture;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse;
Expand All @@ -45,6 +47,9 @@
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.seqno.ReplicationTracker;
import org.elasticsearch.index.seqno.RetentionLease;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
Expand Down Expand Up @@ -77,9 +82,12 @@
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.Stream;

import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
Expand All @@ -88,6 +96,8 @@
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.everyItem;
import static org.hamcrest.Matchers.in;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.startsWith;

Expand All @@ -103,6 +113,7 @@ protected Collection<Class<? extends Plugin>> nodePlugins() {
@Override
protected void beforeIndexDeletion() throws Exception {
super.beforeIndexDeletion();
assertActiveCopiesEstablishedPeerRecoveryRetentionLeases();
internalCluster().assertSeqNos();
internalCluster().assertSameDocIdsOnShards();
}
Expand Down Expand Up @@ -603,6 +614,49 @@ public void testRelocateWhileContinuouslyIndexingAndWaitingForRefresh() throws E
assertThat(client().prepareSearch("test").setSize(0).execute().actionGet().getHits().getTotalHits().value, equalTo(120L));
}

public void testRelocationEstablishedPeerRecoveryRetentionLeases() throws Exception {
int halfNodes = randomIntBetween(1, 3);
String indexName = "test";
Settings[] nodeSettings = Stream.concat(
Stream.generate(() -> Settings.builder().put("node.attr.color", "blue").build()).limit(halfNodes),
Stream.generate(() -> Settings.builder().put("node.attr.color", "red").build()).limit(halfNodes)).toArray(Settings[]::new);
List<String> nodes = internalCluster().startNodes(nodeSettings);
String[] blueNodes = nodes.subList(0, halfNodes).toArray(String[]::new);
String[] redNodes = nodes.subList(0, halfNodes).toArray(String[]::new);
ensureStableCluster(halfNodes * 2);
assertAcked(
client().admin().indices().prepareCreate(indexName).setSettings(Settings.builder()
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean())
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, halfNodes - 1))
.put("index.routing.allocation.include.color", "blue")));
ensureGreen("test");
assertBusy(() -> assertAllShardsOnNodes(indexName, blueNodes));
assertActiveCopiesEstablishedPeerRecoveryRetentionLeases();
client().admin().indices().prepareUpdateSettings(indexName)
.setSettings(Settings.builder().put("index.routing.allocation.include.color", "red")).get();
assertBusy(() -> assertAllShardsOnNodes(indexName, redNodes));
ensureGreen("test");
assertActiveCopiesEstablishedPeerRecoveryRetentionLeases();
}

private void assertActiveCopiesEstablishedPeerRecoveryRetentionLeases() throws Exception {
assertBusy(() -> {
for (ObjectCursor<String> it : client().admin().cluster().prepareState().get().getState().metaData().indices().keys()) {
Map<ShardId, List<ShardStats>> byShardId = Stream.of(client().admin().indices().prepareStats(it.value).get().getShards())
.collect(Collectors.groupingBy(l -> l.getShardRouting().shardId()));
for (List<ShardStats> shardStats : byShardId.values()) {
Set<String> expectedLeaseIds = shardStats.stream()
.map(s -> ReplicationTracker.getPeerRecoveryRetentionLeaseId(s.getShardRouting())).collect(Collectors.toSet());
for (ShardStats shardStat : shardStats) {
Set<String> actualLeaseIds = shardStat.getRetentionLeaseStats().retentionLeases().leases().stream()
.map(RetentionLease::id).collect(Collectors.toSet());
assertThat(expectedLeaseIds, everyItem(in(actualLeaseIds)));
}
}
}
});
}

class RecoveryCorruption implements StubbableTransport.SendRequestBehavior {

private final CountDownLatch corruptionCount;
Expand Down