Use index for peer recovery instead of translog (#45137)
Today we recover a replica by copying operations from the primary's translog. However we also retain some historical operations in the index itself, as long as soft-deletes are enabled. This commit adjusts peer recovery to use the operations in the index for recovery rather than those in the translog, and ensures that the replication group retains enough history for use in peer recovery by means of retention leases. Reverts #38904 and #42211 Relates #41536 Backport of #45136 to 7.x.
This commit is contained in:
parent
3366726ad1
commit
9ff320d967
|
@ -54,6 +54,7 @@ import org.elasticsearch.client.indices.CreateIndexResponse;
|
||||||
import org.elasticsearch.common.xcontent.XContentHelper;
|
import org.elasticsearch.common.xcontent.XContentHelper;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
|
||||||
|
@ -260,7 +261,9 @@ public class CCRIT extends ESRestHighLevelClientTestCase {
|
||||||
final Map<?, ?> shardStatsAsMap = (Map<?, ?>) shardStats.get(0);
|
final Map<?, ?> shardStatsAsMap = (Map<?, ?>) shardStats.get(0);
|
||||||
final Map<?, ?> retentionLeasesStats = (Map<?, ?>) shardStatsAsMap.get("retention_leases");
|
final Map<?, ?> retentionLeasesStats = (Map<?, ?>) shardStatsAsMap.get("retention_leases");
|
||||||
final List<?> leases = (List<?>) retentionLeasesStats.get("leases");
|
final List<?> leases = (List<?>) retentionLeasesStats.get("leases");
|
||||||
assertThat(leases, empty());
|
for (final Object lease : leases) {
|
||||||
|
assertThat(((Map<?, ?>) lease).get("source"), equalTo(ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
import org.elasticsearch.common.xcontent.json.JsonXContent;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLeaseUtils;
|
||||||
import org.elasticsearch.rest.action.document.RestBulkAction;
|
import org.elasticsearch.rest.action.document.RestBulkAction;
|
||||||
import org.elasticsearch.rest.action.document.RestGetAction;
|
import org.elasticsearch.rest.action.document.RestGetAction;
|
||||||
import org.elasticsearch.rest.action.document.RestIndexAction;
|
import org.elasticsearch.rest.action.document.RestIndexAction;
|
||||||
|
@ -89,7 +90,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
|
||||||
private String type;
|
private String type;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setIndex() throws IOException {
|
public void setIndex() {
|
||||||
index = getTestName().toLowerCase(Locale.ROOT);
|
index = getTestName().toLowerCase(Locale.ROOT);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1338,4 +1339,31 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase {
|
||||||
assertEquals("green", healthRsp.get("status"));
|
assertEquals("green", healthRsp.get("status"));
|
||||||
assertFalse((Boolean) healthRsp.get("timed_out"));
|
assertFalse((Boolean) healthRsp.get("timed_out"));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeases() throws IOException {
|
||||||
|
assumeTrue(getOldClusterVersion() + " does not support soft deletes", getOldClusterVersion().onOrAfter(Version.V_6_5_0));
|
||||||
|
if (isRunningAgainstOldCluster()) {
|
||||||
|
XContentBuilder settings = jsonBuilder();
|
||||||
|
settings.startObject();
|
||||||
|
{
|
||||||
|
settings.startObject("settings");
|
||||||
|
settings.field("number_of_shards", between(1, 5));
|
||||||
|
settings.field("number_of_replicas", between(0, 1));
|
||||||
|
if (randomBoolean() || getOldClusterVersion().before(Version.V_7_0_0)) {
|
||||||
|
// this is the default after v7.0.0, but is required before that
|
||||||
|
settings.field("soft_deletes.enabled", true);
|
||||||
|
}
|
||||||
|
settings.endObject();
|
||||||
|
}
|
||||||
|
settings.endObject();
|
||||||
|
|
||||||
|
Request createIndex = new Request("PUT", "/" + index);
|
||||||
|
createIndex.setJsonEntity(Strings.toString(settings));
|
||||||
|
client().performRequest(createIndex);
|
||||||
|
ensureGreen(index);
|
||||||
|
} else {
|
||||||
|
ensureGreen(index);
|
||||||
|
RetentionLeaseUtils.assertAllCopiesHavePeerRecoveryRetentionLeases(client(), index);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLeaseUtils;
|
||||||
import org.elasticsearch.rest.RestStatus;
|
import org.elasticsearch.rest.RestStatus;
|
||||||
import org.elasticsearch.rest.action.document.RestIndexAction;
|
import org.elasticsearch.rest.action.document.RestIndexAction;
|
||||||
import org.elasticsearch.rest.action.document.RestUpdateAction;
|
import org.elasticsearch.rest.action.document.RestUpdateAction;
|
||||||
|
@ -384,6 +385,80 @@ public class RecoveryIT extends AbstractRollingTestCase {
|
||||||
ensureGreen(index);
|
ensureGreen(index);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testRetentionLeasesEstablishedWhenPromotingPrimary() throws Exception {
|
||||||
|
final String index = "recover_and_create_leases_in_promotion";
|
||||||
|
if (CLUSTER_TYPE == ClusterType.OLD) {
|
||||||
|
Settings.Builder settings = Settings.builder()
|
||||||
|
.put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), between(1, 5))
|
||||||
|
.put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), between(1, 2)) // triggers nontrivial promotion
|
||||||
|
.put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms")
|
||||||
|
.put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0") // fail faster
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true);
|
||||||
|
createIndex(index, settings.build());
|
||||||
|
int numDocs = randomInt(10);
|
||||||
|
indexDocs(index, 0, numDocs);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
client().performRequest(new Request("POST", "/" + index + "/_flush"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
ensureGreen(index);
|
||||||
|
if (CLUSTER_TYPE == ClusterType.UPGRADED) {
|
||||||
|
assertBusy(() -> RetentionLeaseUtils.assertAllCopiesHavePeerRecoveryRetentionLeases(client(), index));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testRetentionLeasesEstablishedWhenRelocatingPrimary() throws Exception {
|
||||||
|
final String index = "recover_and_create_leases_in_relocation";
|
||||||
|
switch (CLUSTER_TYPE) {
|
||||||
|
case OLD:
|
||||||
|
Settings.Builder settings = Settings.builder()
|
||||||
|
.put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), between(1, 5))
|
||||||
|
.put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), between(0, 1))
|
||||||
|
.put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms")
|
||||||
|
.put(SETTING_ALLOCATION_MAX_RETRY.getKey(), "0") // fail faster
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true);
|
||||||
|
createIndex(index, settings.build());
|
||||||
|
int numDocs = randomInt(10);
|
||||||
|
indexDocs(index, 0, numDocs);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
client().performRequest(new Request("POST", "/" + index + "/_flush"));
|
||||||
|
}
|
||||||
|
ensureGreen(index);
|
||||||
|
break;
|
||||||
|
|
||||||
|
case MIXED:
|
||||||
|
// trigger a primary relocation by excluding the last old node with a shard filter
|
||||||
|
final Map<?, ?> nodesMap
|
||||||
|
= ObjectPath.createFromResponse(client().performRequest(new Request("GET", "/_nodes"))).evaluate("nodes");
|
||||||
|
final List<String> oldNodeNames = new ArrayList<>();
|
||||||
|
for (Object nodeDetails : nodesMap.values()) {
|
||||||
|
final Map<?, ?> nodeDetailsMap = (Map<?, ?>) nodeDetails;
|
||||||
|
final String versionString = (String) nodeDetailsMap.get("version");
|
||||||
|
if (versionString.equals(Version.CURRENT.toString()) == false) {
|
||||||
|
oldNodeNames.add((String) nodeDetailsMap.get("name"));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (oldNodeNames.size() == 1) {
|
||||||
|
final String oldNodeName = oldNodeNames.get(0);
|
||||||
|
logger.info("--> excluding index [{}] from node [{}]", index, oldNodeName);
|
||||||
|
final Request putSettingsRequest = new Request("PUT", "/" + index + "/_settings");
|
||||||
|
putSettingsRequest.setJsonEntity("{\"index.routing.allocation.exclude._name\":\"" + oldNodeName + "\"}");
|
||||||
|
assertOK(client().performRequest(putSettingsRequest));
|
||||||
|
ensureGreen(index);
|
||||||
|
assertBusy(() -> RetentionLeaseUtils.assertAllCopiesHavePeerRecoveryRetentionLeases(client(), index));
|
||||||
|
} else {
|
||||||
|
ensureGreen(index);
|
||||||
|
}
|
||||||
|
break;
|
||||||
|
|
||||||
|
case UPGRADED:
|
||||||
|
ensureGreen(index);
|
||||||
|
assertBusy(() -> RetentionLeaseUtils.assertAllCopiesHavePeerRecoveryRetentionLeases(client(), index));
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This test creates an index in the non upgraded cluster and closes it. It then checks that the index
|
* This test creates an index in the non upgraded cluster and closes it. It then checks that the index
|
||||||
* is effectively closed and potentially replicated (if the version the index was created on supports
|
* is effectively closed and potentially replicated (if the version the index was created on supports
|
||||||
|
|
|
@ -747,7 +747,7 @@ public abstract class Engine implements Closeable {
|
||||||
MapperService mapperService, long startingSeqNo) throws IOException;
|
MapperService mapperService, long startingSeqNo) throws IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its translog
|
* Checks if this engine has every operations since {@code startingSeqNo}(inclusive) in its history (either Lucene or translog)
|
||||||
*/
|
*/
|
||||||
public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException;
|
public abstract boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException;
|
||||||
|
|
||||||
|
|
|
@ -518,10 +518,15 @@ public class InternalEngine extends Engine {
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Creates a new history snapshot for reading operations since the provided seqno from the translog.
|
* Creates a new history snapshot for reading operations since the provided seqno.
|
||||||
|
* The returned snapshot can be retrieved from either Lucene index or translog files.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
public Translog.Snapshot readHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
||||||
|
if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) {
|
||||||
|
return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false);
|
||||||
|
}
|
||||||
|
|
||||||
return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo);
|
return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -529,7 +534,14 @@ public class InternalEngine extends Engine {
|
||||||
* Returns the estimated number of history operations whose seq# at least the provided seq# in this engine.
|
* Returns the estimated number of history operations whose seq# at least the provided seq# in this engine.
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) {
|
public int estimateNumberOfHistoryOperations(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
||||||
|
if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) {
|
||||||
|
try (Translog.Snapshot snapshot = newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo),
|
||||||
|
Long.MAX_VALUE, false)) {
|
||||||
|
return snapshot.totalOperations();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo);
|
return getTranslog().estimateTotalOperationsFromMinSeq(startingSeqNo);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2579,6 +2591,10 @@ public class InternalEngine extends Engine {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
||||||
|
if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) {
|
||||||
|
return getMinRetainedSeqNo() <= startingSeqNo;
|
||||||
|
}
|
||||||
|
|
||||||
final long currentLocalCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
final long currentLocalCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||||
// avoid scanning translog if not necessary
|
// avoid scanning translog if not necessary
|
||||||
if (startingSeqNo > currentLocalCheckpoint) {
|
if (startingSeqNo > currentLocalCheckpoint) {
|
||||||
|
@ -2608,15 +2624,7 @@ public class InternalEngine extends Engine {
|
||||||
@Override
|
@Override
|
||||||
public Closeable acquireRetentionLock() {
|
public Closeable acquireRetentionLock() {
|
||||||
if (softDeleteEnabled) {
|
if (softDeleteEnabled) {
|
||||||
final Releasable softDeletesRetentionLock = softDeletesPolicy.acquireRetentionLock();
|
return softDeletesPolicy.acquireRetentionLock();
|
||||||
final Closeable translogRetentionLock;
|
|
||||||
try {
|
|
||||||
translogRetentionLock = translog.acquireRetentionLock();
|
|
||||||
} catch (Exception e) {
|
|
||||||
softDeletesRetentionLock.close();
|
|
||||||
throw e;
|
|
||||||
}
|
|
||||||
return () -> IOUtils.close(translogRetentionLock, softDeletesRetentionLock);
|
|
||||||
} else {
|
} else {
|
||||||
return translog.acquireRetentionLock();
|
return translog.acquireRetentionLock();
|
||||||
}
|
}
|
||||||
|
|
|
@ -23,7 +23,9 @@ import com.carrotsearch.hppc.ObjectLongHashMap;
|
||||||
import com.carrotsearch.hppc.ObjectLongMap;
|
import com.carrotsearch.hppc.ObjectLongMap;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.GroupedActionListener;
|
||||||
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.AllocationId;
|
import org.elasticsearch.cluster.routing.AllocationId;
|
||||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
@ -59,6 +61,7 @@ import java.util.function.ToLongFunction;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.LongStream;
|
import java.util.stream.LongStream;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
import java.util.stream.StreamSupport;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This class is responsible for tracking the replication group with its progress and safety markers (local and global checkpoints).
|
* This class is responsible for tracking the replication group with its progress and safety markers (local and global checkpoints).
|
||||||
|
@ -199,6 +202,14 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
*/
|
*/
|
||||||
private long persistedRetentionLeasesVersion;
|
private long persistedRetentionLeasesVersion;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Whether there should be a peer recovery retention lease (PRRL) for every tracked shard copy. Always true on indices created from
|
||||||
|
* {@link Version#V_7_4_0} onwards, because these versions create PRRLs properly. May be false on indices created in an earlier version
|
||||||
|
* if we recently did a rolling upgrade and {@link ReplicationTracker#createMissingPeerRecoveryRetentionLeases(ActionListener)} has not
|
||||||
|
* yet completed. Is only permitted to change from false to true; can be removed once support for pre-PRRL indices is no longer needed.
|
||||||
|
*/
|
||||||
|
private boolean hasAllPeerRecoveryRetentionLeases;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Get all retention leases tracked on this shard.
|
* Get all retention leases tracked on this shard.
|
||||||
*
|
*
|
||||||
|
@ -224,10 +235,22 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
// the primary calculates the non-expired retention leases and syncs them to replicas
|
// the primary calculates the non-expired retention leases and syncs them to replicas
|
||||||
final long currentTimeMillis = currentTimeMillisSupplier.getAsLong();
|
final long currentTimeMillis = currentTimeMillisSupplier.getAsLong();
|
||||||
final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis();
|
final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis();
|
||||||
|
final Set<String> leaseIdsForCurrentPeers
|
||||||
|
= routingTable.assignedShards().stream().map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet());
|
||||||
final Map<Boolean, List<RetentionLease>> partitionByExpiration = retentionLeases
|
final Map<Boolean, List<RetentionLease>> partitionByExpiration = retentionLeases
|
||||||
.leases()
|
.leases()
|
||||||
.stream()
|
.stream()
|
||||||
.collect(Collectors.groupingBy(lease -> currentTimeMillis - lease.timestamp() > retentionLeaseMillis));
|
.collect(Collectors.groupingBy(lease -> {
|
||||||
|
if (lease.source().equals(PEER_RECOVERY_RETENTION_LEASE_SOURCE)) {
|
||||||
|
if (leaseIdsForCurrentPeers.contains(lease.id())) {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
if (routingTable.allShardsStarted()) {
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return currentTimeMillis - lease.timestamp() > retentionLeaseMillis;
|
||||||
|
}));
|
||||||
final Collection<RetentionLease> expiredLeases = partitionByExpiration.get(true);
|
final Collection<RetentionLease> expiredLeases = partitionByExpiration.get(true);
|
||||||
if (expiredLeases == null) {
|
if (expiredLeases == null) {
|
||||||
// early out as no retention leases have expired
|
// early out as no retention leases have expired
|
||||||
|
@ -249,7 +272,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
* @param source the source of the retention lease
|
* @param source the source of the retention lease
|
||||||
* @param listener the callback when the retention lease is successfully added and synced to replicas
|
* @param listener the callback when the retention lease is successfully added and synced to replicas
|
||||||
* @return the new retention lease
|
* @return the new retention lease
|
||||||
* @throws IllegalArgumentException if the specified retention lease already exists
|
* @throws RetentionLeaseAlreadyExistsException if the specified retention lease already exists
|
||||||
*/
|
*/
|
||||||
public RetentionLease addRetentionLease(
|
public RetentionLease addRetentionLease(
|
||||||
final String id,
|
final String id,
|
||||||
|
@ -260,22 +283,68 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
final RetentionLease retentionLease;
|
final RetentionLease retentionLease;
|
||||||
final RetentionLeases currentRetentionLeases;
|
final RetentionLeases currentRetentionLeases;
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
assert primaryMode;
|
retentionLease = innerAddRetentionLease(id, retainingSequenceNumber, source);
|
||||||
if (retentionLeases.contains(id)) {
|
|
||||||
throw new RetentionLeaseAlreadyExistsException(id);
|
|
||||||
}
|
|
||||||
retentionLease = new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source);
|
|
||||||
logger.debug("adding new retention lease [{}] to current retention leases [{}]", retentionLease, retentionLeases);
|
|
||||||
retentionLeases = new RetentionLeases(
|
|
||||||
operationPrimaryTerm,
|
|
||||||
retentionLeases.version() + 1,
|
|
||||||
Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList()));
|
|
||||||
currentRetentionLeases = retentionLeases;
|
currentRetentionLeases = retentionLeases;
|
||||||
}
|
}
|
||||||
onSyncRetentionLeases.accept(currentRetentionLeases, listener);
|
onSyncRetentionLeases.accept(currentRetentionLeases, listener);
|
||||||
return retentionLease;
|
return retentionLease;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Atomically clones an existing retention lease to a new ID.
|
||||||
|
*
|
||||||
|
* @param sourceLeaseId the identifier of the source retention lease
|
||||||
|
* @param targetLeaseId the identifier of the retention lease to create
|
||||||
|
* @param listener the callback when the retention lease is successfully added and synced to replicas
|
||||||
|
* @return the new retention lease
|
||||||
|
* @throws RetentionLeaseNotFoundException if the specified source retention lease does not exist
|
||||||
|
* @throws RetentionLeaseAlreadyExistsException if the specified target retention lease already exists
|
||||||
|
*/
|
||||||
|
RetentionLease cloneRetentionLease(String sourceLeaseId, String targetLeaseId, ActionListener<ReplicationResponse> listener) {
|
||||||
|
Objects.requireNonNull(listener);
|
||||||
|
final RetentionLease retentionLease;
|
||||||
|
final RetentionLeases currentRetentionLeases;
|
||||||
|
synchronized (this) {
|
||||||
|
assert primaryMode;
|
||||||
|
if (getRetentionLeases().contains(sourceLeaseId) == false) {
|
||||||
|
throw new RetentionLeaseNotFoundException(sourceLeaseId);
|
||||||
|
}
|
||||||
|
final RetentionLease sourceLease = getRetentionLeases().get(sourceLeaseId);
|
||||||
|
retentionLease = innerAddRetentionLease(targetLeaseId, sourceLease.retainingSequenceNumber(), sourceLease.source());
|
||||||
|
currentRetentionLeases = retentionLeases;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Syncing here may not be strictly necessary, because this new lease isn't retaining any extra history that wasn't previously
|
||||||
|
// retained by the source lease; however we prefer to sync anyway since we expect to do so whenever creating a new lease.
|
||||||
|
onSyncRetentionLeases.accept(currentRetentionLeases, listener);
|
||||||
|
return retentionLease;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Adds a new retention lease, but does not synchronise it with the rest of the replication group.
|
||||||
|
*
|
||||||
|
* @param id the identifier of the retention lease
|
||||||
|
* @param retainingSequenceNumber the retaining sequence number
|
||||||
|
* @param source the source of the retention lease
|
||||||
|
* @return the new retention lease
|
||||||
|
* @throws RetentionLeaseAlreadyExistsException if the specified retention lease already exists
|
||||||
|
*/
|
||||||
|
private RetentionLease innerAddRetentionLease(String id, long retainingSequenceNumber, String source) {
|
||||||
|
assert Thread.holdsLock(this);
|
||||||
|
assert primaryMode : id + "/" + retainingSequenceNumber + "/" + source;
|
||||||
|
if (retentionLeases.contains(id)) {
|
||||||
|
throw new RetentionLeaseAlreadyExistsException(id);
|
||||||
|
}
|
||||||
|
final RetentionLease retentionLease
|
||||||
|
= new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source);
|
||||||
|
logger.debug("adding new retention lease [{}] to current retention leases [{}]", retentionLease, retentionLeases);
|
||||||
|
retentionLeases = new RetentionLeases(
|
||||||
|
operationPrimaryTerm,
|
||||||
|
retentionLeases.version() + 1,
|
||||||
|
Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList()));
|
||||||
|
return retentionLease;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Renews an existing retention lease.
|
* Renews an existing retention lease.
|
||||||
*
|
*
|
||||||
|
@ -283,7 +352,7 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
* @param retainingSequenceNumber the retaining sequence number
|
* @param retainingSequenceNumber the retaining sequence number
|
||||||
* @param source the source of the retention lease
|
* @param source the source of the retention lease
|
||||||
* @return the renewed retention lease
|
* @return the renewed retention lease
|
||||||
* @throws IllegalArgumentException if the specified retention lease does not exist
|
* @throws RetentionLeaseNotFoundException if the specified retention lease does not exist
|
||||||
*/
|
*/
|
||||||
public synchronized RetentionLease renewRetentionLease(final String id, final long retainingSequenceNumber, final String source) {
|
public synchronized RetentionLease renewRetentionLease(final String id, final long retainingSequenceNumber, final String source) {
|
||||||
assert primaryMode;
|
assert primaryMode;
|
||||||
|
@ -397,6 +466,108 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Retention leases for peer recovery have source {@link ReplicationTracker#PEER_RECOVERY_RETENTION_LEASE_SOURCE}, a lease ID
|
||||||
|
* containing the persistent node ID calculated by {@link ReplicationTracker#getPeerRecoveryRetentionLeaseId}, and retain operations
|
||||||
|
* with sequence numbers strictly greater than the given global checkpoint.
|
||||||
|
*/
|
||||||
|
public RetentionLease addPeerRecoveryRetentionLease(String nodeId, long globalCheckpoint,
|
||||||
|
ActionListener<ReplicationResponse> listener) {
|
||||||
|
return addRetentionLease(getPeerRecoveryRetentionLeaseId(nodeId), globalCheckpoint + 1,
|
||||||
|
PEER_RECOVERY_RETENTION_LEASE_SOURCE, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetentionLease cloneLocalPeerRecoveryRetentionLease(String nodeId, ActionListener<ReplicationResponse> listener) {
|
||||||
|
return cloneRetentionLease(
|
||||||
|
getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()),
|
||||||
|
getPeerRecoveryRetentionLeaseId(nodeId), listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void removePeerRecoveryRetentionLease(String nodeId, ActionListener<ReplicationResponse> listener) {
|
||||||
|
removeRetentionLease(getPeerRecoveryRetentionLeaseId(nodeId), listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Source for peer recovery retention leases; see {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
|
||||||
|
*/
|
||||||
|
public static final String PEER_RECOVERY_RETENTION_LEASE_SOURCE = "peer recovery";
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Id for a peer recovery retention lease for the given node. See {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
|
||||||
|
*/
|
||||||
|
static String getPeerRecoveryRetentionLeaseId(String nodeId) {
|
||||||
|
return "peer_recovery/" + nodeId;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Id for a peer recovery retention lease for the given {@link ShardRouting}.
|
||||||
|
* See {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
|
||||||
|
*/
|
||||||
|
public static String getPeerRecoveryRetentionLeaseId(ShardRouting shardRouting) {
|
||||||
|
return getPeerRecoveryRetentionLeaseId(shardRouting.currentNodeId());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Advance the peer-recovery retention leases for all assigned shard copies to discard history below the corresponding global
|
||||||
|
* checkpoint, and renew any leases that are approaching expiry.
|
||||||
|
*/
|
||||||
|
public synchronized void renewPeerRecoveryRetentionLeases() {
|
||||||
|
assert primaryMode;
|
||||||
|
assert invariant();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Peer-recovery retention leases never expire while the associated shard is assigned, but we must still renew them occasionally in
|
||||||
|
* case the associated shard is temporarily unassigned. However we must not renew them too often, since each renewal must be
|
||||||
|
* persisted and the resulting IO can be expensive on nodes with large numbers of shards (see #42299). We choose to renew them after
|
||||||
|
* half the expiry time, so that by default the cluster has at least 6 hours to recover before these leases start to expire.
|
||||||
|
*/
|
||||||
|
final long renewalTimeMillis = currentTimeMillisSupplier.getAsLong() - indexSettings.getRetentionLeaseMillis() / 2;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* If any of the peer-recovery retention leases need renewal, it's a good opportunity to renew them all.
|
||||||
|
*/
|
||||||
|
final boolean renewalNeeded = StreamSupport.stream(routingTable.spliterator(), false).filter(ShardRouting::assignedToNode)
|
||||||
|
.anyMatch(shardRouting -> {
|
||||||
|
final RetentionLease retentionLease = retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting));
|
||||||
|
if (retentionLease == null) {
|
||||||
|
/*
|
||||||
|
* If this shard copy is tracked then we got here here via a rolling upgrade from an older version that doesn't
|
||||||
|
* create peer recovery retention leases for every shard copy.
|
||||||
|
*/
|
||||||
|
assert checkpoints.get(shardRouting.allocationId().getId()).tracked == false
|
||||||
|
|| hasAllPeerRecoveryRetentionLeases == false;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
return retentionLease.timestamp() <= renewalTimeMillis
|
||||||
|
|| retentionLease.retainingSequenceNumber() <= checkpoints.get(shardRouting.allocationId().getId()).globalCheckpoint;
|
||||||
|
});
|
||||||
|
|
||||||
|
if (renewalNeeded) {
|
||||||
|
for (ShardRouting shardRouting : routingTable) {
|
||||||
|
if (shardRouting.assignedToNode()) {
|
||||||
|
final RetentionLease retentionLease = retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting));
|
||||||
|
if (retentionLease != null) {
|
||||||
|
final CheckpointState checkpointState = checkpoints.get(shardRouting.allocationId().getId());
|
||||||
|
final long newRetainedSequenceNumber = Math.max(0L, checkpointState.globalCheckpoint + 1L);
|
||||||
|
if (retentionLease.retainingSequenceNumber() <= newRetainedSequenceNumber) {
|
||||||
|
renewRetentionLease(getPeerRecoveryRetentionLeaseId(shardRouting), newRetainedSequenceNumber,
|
||||||
|
PEER_RECOVERY_RETENTION_LEASE_SOURCE);
|
||||||
|
} else {
|
||||||
|
// the retention lease is tied to the node, not the shard copy, so it's possible a copy was removed and now
|
||||||
|
// we are in the process of recovering it again, or maybe we were just promoted and have not yet received the
|
||||||
|
// global checkpoints from our peers.
|
||||||
|
assert checkpointState.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO :
|
||||||
|
"cannot renew " + retentionLease + " according to " + checkpointState + " for " + shardRouting;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
assert invariant();
|
||||||
|
}
|
||||||
|
|
||||||
public static class CheckpointState implements Writeable {
|
public static class CheckpointState implements Writeable {
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -625,6 +796,23 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked";
|
assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (primaryMode
|
||||||
|
&& indexSettings.isSoftDeleteEnabled()
|
||||||
|
&& indexSettings.getIndexMetaData().getState() == IndexMetaData.State.OPEN
|
||||||
|
&& hasAllPeerRecoveryRetentionLeases) {
|
||||||
|
// all tracked shard copies have a corresponding peer-recovery retention lease
|
||||||
|
for (final ShardRouting shardRouting : routingTable.assignedShards()) {
|
||||||
|
if (checkpoints.get(shardRouting.allocationId().getId()).tracked) {
|
||||||
|
assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting))
|
||||||
|
: "no retention lease for tracked shard [" + shardRouting + "] in " + retentionLeases;
|
||||||
|
assert PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(
|
||||||
|
retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source())
|
||||||
|
: "incorrect source [" + retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)).source()
|
||||||
|
+ "] for [" + shardRouting + "] in " + retentionLeases;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -678,6 +866,8 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
this.pendingInSync = new HashSet<>();
|
this.pendingInSync = new HashSet<>();
|
||||||
this.routingTable = null;
|
this.routingTable = null;
|
||||||
this.replicationGroup = null;
|
this.replicationGroup = null;
|
||||||
|
this.hasAllPeerRecoveryRetentionLeases = indexSettings.getIndexVersionCreated().onOrAfter(Version.V_7_4_0);
|
||||||
|
assert Version.V_EMPTY.equals(indexSettings.getIndexVersionCreated()) == false;
|
||||||
assert invariant();
|
assert invariant();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -769,9 +959,55 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
primaryMode = true;
|
primaryMode = true;
|
||||||
updateLocalCheckpoint(shardAllocationId, checkpoints.get(shardAllocationId), localCheckpoint);
|
updateLocalCheckpoint(shardAllocationId, checkpoints.get(shardAllocationId), localCheckpoint);
|
||||||
updateGlobalCheckpointOnPrimary();
|
updateGlobalCheckpointOnPrimary();
|
||||||
|
|
||||||
|
if (indexSettings.isSoftDeleteEnabled()) {
|
||||||
|
addPeerRecoveryRetentionLeaseForSolePrimary();
|
||||||
|
}
|
||||||
|
|
||||||
assert invariant();
|
assert invariant();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Creates a peer recovery retention lease for this shard, if one does not already exist and this shard is the sole shard copy in the
|
||||||
|
* replication group. If one does not already exist and yet there are other shard copies in this group then we must have just done
|
||||||
|
* a rolling upgrade from a version before {@link Version#V_7_4_0}, in which case the missing leases should be created asynchronously
|
||||||
|
* by the caller using {@link ReplicationTracker#createMissingPeerRecoveryRetentionLeases(ActionListener)}.
|
||||||
|
*/
|
||||||
|
private void addPeerRecoveryRetentionLeaseForSolePrimary() {
|
||||||
|
assert primaryMode;
|
||||||
|
assert Thread.holdsLock(this);
|
||||||
|
|
||||||
|
if (indexSettings().getIndexMetaData().getState() == IndexMetaData.State.OPEN) {
|
||||||
|
final ShardRouting primaryShard = routingTable.primaryShard();
|
||||||
|
final String leaseId = getPeerRecoveryRetentionLeaseId(primaryShard);
|
||||||
|
if (retentionLeases.get(leaseId) == null) {
|
||||||
|
if (replicationGroup.getReplicationTargets().equals(Collections.singletonList(primaryShard))) {
|
||||||
|
assert primaryShard.allocationId().getId().equals(shardAllocationId)
|
||||||
|
: routingTable.assignedShards() + " vs " + shardAllocationId;
|
||||||
|
// Safe to call innerAddRetentionLease() without a subsequent sync since there are no other members of this replication
|
||||||
|
// group.
|
||||||
|
logger.trace("addPeerRecoveryRetentionLeaseForSolePrimary: adding lease [{}]", leaseId);
|
||||||
|
innerAddRetentionLease(leaseId, Math.max(0L, checkpoints.get(shardAllocationId).globalCheckpoint + 1),
|
||||||
|
PEER_RECOVERY_RETENTION_LEASE_SOURCE);
|
||||||
|
hasAllPeerRecoveryRetentionLeases = true;
|
||||||
|
} else {
|
||||||
|
/*
|
||||||
|
* We got here here via a rolling upgrade from an older version that doesn't create peer recovery retention
|
||||||
|
* leases for every shard copy, but in this case we do not expect any leases to exist.
|
||||||
|
*/
|
||||||
|
assert hasAllPeerRecoveryRetentionLeases == false : routingTable + " vs " + retentionLeases;
|
||||||
|
logger.debug("{} becoming primary of {} with missing lease: {}", primaryShard, routingTable, retentionLeases);
|
||||||
|
}
|
||||||
|
} else if (hasAllPeerRecoveryRetentionLeases == false && routingTable.assignedShards().stream().allMatch(shardRouting ->
|
||||||
|
retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting))
|
||||||
|
|| checkpoints.get(shardRouting.allocationId().getId()).tracked == false)) {
|
||||||
|
// Although this index is old enough not to have all the expected peer recovery retention leases, in fact it does, so we
|
||||||
|
// don't need to do any more work.
|
||||||
|
hasAllPeerRecoveryRetentionLeases = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Notifies the tracker of the current allocation IDs in the cluster state.
|
* Notifies the tracker of the current allocation IDs in the cluster state.
|
||||||
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
|
* @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master
|
||||||
|
@ -1072,9 +1308,57 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
||||||
// note that if there was no cluster state update between start of the engine of this shard and the call to
|
// 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.
|
// initializeWithPrimaryContext, we might still have missed a cluster state update. This is best effort.
|
||||||
runAfter.run();
|
runAfter.run();
|
||||||
|
|
||||||
|
if (indexSettings.isSoftDeleteEnabled()) {
|
||||||
|
addPeerRecoveryRetentionLeaseForSolePrimary();
|
||||||
|
}
|
||||||
|
|
||||||
assert invariant();
|
assert invariant();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private synchronized void setHasAllPeerRecoveryRetentionLeases() {
|
||||||
|
hasAllPeerRecoveryRetentionLeases = true;
|
||||||
|
assert invariant();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Create any required peer-recovery retention leases that do not currently exist because we just did a rolling upgrade from a version
|
||||||
|
* prior to {@link Version#V_7_4_0} that does not create peer-recovery retention leases.
|
||||||
|
*/
|
||||||
|
public synchronized void createMissingPeerRecoveryRetentionLeases(ActionListener<Void> listener) {
|
||||||
|
if (indexSettings().isSoftDeleteEnabled()
|
||||||
|
&& indexSettings().getIndexMetaData().getState() == IndexMetaData.State.OPEN
|
||||||
|
&& hasAllPeerRecoveryRetentionLeases == false) {
|
||||||
|
|
||||||
|
final List<ShardRouting> shardRoutings = routingTable.assignedShards();
|
||||||
|
final GroupedActionListener<ReplicationResponse> groupedActionListener = new GroupedActionListener<>(ActionListener.wrap(vs -> {
|
||||||
|
setHasAllPeerRecoveryRetentionLeases();
|
||||||
|
listener.onResponse(null);
|
||||||
|
}, listener::onFailure), shardRoutings.size());
|
||||||
|
for (ShardRouting shardRouting : shardRoutings) {
|
||||||
|
if (retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting))) {
|
||||||
|
groupedActionListener.onResponse(null);
|
||||||
|
} else {
|
||||||
|
final CheckpointState checkpointState = checkpoints.get(shardRouting.allocationId().getId());
|
||||||
|
if (checkpointState.tracked == false) {
|
||||||
|
groupedActionListener.onResponse(null);
|
||||||
|
} else {
|
||||||
|
logger.trace("createMissingPeerRecoveryRetentionLeases: adding missing lease for {}", shardRouting);
|
||||||
|
try {
|
||||||
|
addPeerRecoveryRetentionLease(shardRouting.currentNodeId(),
|
||||||
|
Math.max(SequenceNumbers.NO_OPS_PERFORMED, checkpointState.globalCheckpoint), groupedActionListener);
|
||||||
|
} catch (Exception e) {
|
||||||
|
groupedActionListener.onFailure(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
logger.trace("createMissingPeerRecoveryRetentionLeases: nothing to do");
|
||||||
|
listener.onResponse(null);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private Runnable getMasterUpdateOperationFromCurrentState() {
|
private Runnable getMasterUpdateOperationFromCurrentState() {
|
||||||
assert primaryMode == false;
|
assert primaryMode == false;
|
||||||
final long lastAppliedClusterStateVersion = appliedClusterStateVersion;
|
final long lastAppliedClusterStateVersion = appliedClusterStateVersion;
|
||||||
|
|
|
@ -44,7 +44,7 @@ public interface RetentionLeaseSyncer {
|
||||||
RetentionLeaseSyncer EMPTY = new RetentionLeaseSyncer() {
|
RetentionLeaseSyncer EMPTY = new RetentionLeaseSyncer() {
|
||||||
@Override
|
@Override
|
||||||
public void sync(final ShardId shardId, final RetentionLeases retentionLeases, final ActionListener<ReplicationResponse> listener) {
|
public void sync(final ShardId shardId, final RetentionLeases retentionLeases, final ActionListener<ReplicationResponse> listener) {
|
||||||
|
listener.onResponse(new ReplicationResponse());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -274,14 +274,5 @@ public class RetentionLeases implements ToXContentFragment, Writeable {
|
||||||
LinkedHashMap::new));
|
LinkedHashMap::new));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* A utility method to convert a retention lease collection to a map from retention lease ID to retention lease.
|
|
||||||
*
|
|
||||||
* @param retentionLeases the retention lease collection
|
|
||||||
* @return the map from retention lease ID to retention lease
|
|
||||||
*/
|
|
||||||
static Map<String, RetentionLease> toMap(final RetentionLeases retentionLeases) {
|
|
||||||
return retentionLeases.leases;
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
||||||
import org.elasticsearch.Assertions;
|
import org.elasticsearch.Assertions;
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
|
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;
|
||||||
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
|
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
|
||||||
|
@ -159,6 +160,7 @@ import java.util.List;
|
||||||
import java.util.Locale;
|
import java.util.Locale;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.CopyOnWriteArrayList;
|
import java.util.concurrent.CopyOnWriteArrayList;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
@ -170,6 +172,7 @@ import java.util.concurrent.atomic.AtomicReference;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
|
import java.util.function.LongSupplier;
|
||||||
import java.util.function.Supplier;
|
import java.util.function.Supplier;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.StreamSupport;
|
import java.util.stream.StreamSupport;
|
||||||
|
@ -484,6 +487,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) {
|
if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) {
|
||||||
// the master started a recovering primary, activate primary mode.
|
// the master started a recovering primary, activate primary mode.
|
||||||
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
|
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
|
||||||
|
ensurePeerRecoveryRetentionLeasesExist();
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
assert currentRouting.primary() == false : "term is only increased as part of primary promotion";
|
assert currentRouting.primary() == false : "term is only increased as part of primary promotion";
|
||||||
|
@ -526,6 +530,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
assert getOperationPrimaryTerm() == newPrimaryTerm;
|
assert getOperationPrimaryTerm() == newPrimaryTerm;
|
||||||
try {
|
try {
|
||||||
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
|
replicationTracker.activatePrimaryMode(getLocalCheckpoint());
|
||||||
|
ensurePeerRecoveryRetentionLeasesExist();
|
||||||
/*
|
/*
|
||||||
* If this shard was serving as a replica shard when another shard was promoted to primary then
|
* If this shard was serving as a replica shard when another shard was promoted to primary then
|
||||||
* its Lucene index was reset during the primary term transition. In particular, the Lucene index
|
* its Lucene index was reset during the primary term transition. In particular, the Lucene index
|
||||||
|
@ -1357,6 +1362,88 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
assert currentEngineReference.get() == null;
|
assert currentEngineReference.get() == null;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A best effort to bring up this shard to the global checkpoint using the local translog before performing a peer recovery.
|
||||||
|
*
|
||||||
|
* @return a sequence number that an operation-based peer recovery can start with.
|
||||||
|
* This is the first operation after the local checkpoint of the safe commit if exists.
|
||||||
|
*/
|
||||||
|
public long recoverLocallyUpToGlobalCheckpoint() {
|
||||||
|
if (state != IndexShardState.RECOVERING) {
|
||||||
|
throw new IndexShardNotRecoveringException(shardId, state);
|
||||||
|
}
|
||||||
|
assert recoveryState.getStage() == RecoveryState.Stage.INDEX : "unexpected recovery stage [" + recoveryState.getStage() + "]";
|
||||||
|
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
|
||||||
|
final Optional<SequenceNumbers.CommitInfo> safeCommit;
|
||||||
|
final long globalCheckpoint;
|
||||||
|
try {
|
||||||
|
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
|
||||||
|
globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID);
|
||||||
|
safeCommit = store.findSafeIndexCommit(globalCheckpoint);
|
||||||
|
} catch (org.apache.lucene.index.IndexNotFoundException e) {
|
||||||
|
logger.trace("skip local recovery as no index commit found");
|
||||||
|
return UNASSIGNED_SEQ_NO;
|
||||||
|
} catch (Exception e) {
|
||||||
|
logger.debug("skip local recovery as failed to find the safe commit", e);
|
||||||
|
return UNASSIGNED_SEQ_NO;
|
||||||
|
}
|
||||||
|
if (safeCommit.isPresent() == false) {
|
||||||
|
logger.trace("skip local recovery as no safe commit found");
|
||||||
|
return UNASSIGNED_SEQ_NO;
|
||||||
|
}
|
||||||
|
assert safeCommit.get().localCheckpoint <= globalCheckpoint : safeCommit.get().localCheckpoint + " > " + globalCheckpoint;
|
||||||
|
try {
|
||||||
|
maybeCheckIndex(); // check index here and won't do it again if ops-based recovery occurs
|
||||||
|
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
|
||||||
|
if (safeCommit.get().localCheckpoint == globalCheckpoint) {
|
||||||
|
logger.trace("skip local recovery as the safe commit is up to date; safe commit {} global checkpoint {}",
|
||||||
|
safeCommit.get(), globalCheckpoint);
|
||||||
|
recoveryState.getTranslog().totalLocal(0);
|
||||||
|
return globalCheckpoint + 1;
|
||||||
|
}
|
||||||
|
if (indexSettings.getIndexMetaData().getState() == IndexMetaData.State.CLOSE ||
|
||||||
|
IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.get(indexSettings.getSettings())) {
|
||||||
|
logger.trace("skip local recovery as the index was closed or not allowed to write; safe commit {} global checkpoint {}",
|
||||||
|
safeCommit.get(), globalCheckpoint);
|
||||||
|
recoveryState.getTranslog().totalLocal(0);
|
||||||
|
return safeCommit.get().localCheckpoint + 1;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
final Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> {
|
||||||
|
recoveryState.getTranslog().totalLocal(snapshot.totalOperations());
|
||||||
|
final int recoveredOps = runTranslogRecovery(engine, snapshot, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY,
|
||||||
|
recoveryState.getTranslog()::incrementRecoveredOperations);
|
||||||
|
recoveryState.getTranslog().totalLocal(recoveredOps); // adjust the total local to reflect the actual count
|
||||||
|
return recoveredOps;
|
||||||
|
};
|
||||||
|
innerOpenEngineAndTranslog(() -> globalCheckpoint);
|
||||||
|
getEngine().recoverFromTranslog(translogRecoveryRunner, globalCheckpoint);
|
||||||
|
logger.trace("shard locally recovered up to {}", getEngine().getSeqNoStats(globalCheckpoint));
|
||||||
|
} finally {
|
||||||
|
synchronized (mutex) {
|
||||||
|
IOUtils.close(currentEngineReference.getAndSet(null));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} catch (Exception e) {
|
||||||
|
logger.debug(new ParameterizedMessage("failed to recover shard locally up to global checkpoint {}", globalCheckpoint), e);
|
||||||
|
return UNASSIGNED_SEQ_NO;
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
// we need to find the safe commit again as we should have created a new one during the local recovery
|
||||||
|
final Optional<SequenceNumbers.CommitInfo> newSafeCommit = store.findSafeIndexCommit(globalCheckpoint);
|
||||||
|
assert newSafeCommit.isPresent() : "no safe commit found after local recovery";
|
||||||
|
return newSafeCommit.get().localCheckpoint + 1;
|
||||||
|
} catch (Exception e) {
|
||||||
|
if (Assertions.ENABLED) {
|
||||||
|
throw new AssertionError(
|
||||||
|
"failed to find the safe commit after recovering shard locally up to global checkpoint " + globalCheckpoint, e);
|
||||||
|
}
|
||||||
|
logger.debug(new ParameterizedMessage(
|
||||||
|
"failed to find the safe commit after recovering shard locally up to global checkpoint {}", globalCheckpoint), e);
|
||||||
|
return UNASSIGNED_SEQ_NO;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
public void trimOperationOfPreviousPrimaryTerms(long aboveSeqNo) {
|
public void trimOperationOfPreviousPrimaryTerms(long aboveSeqNo) {
|
||||||
getEngine().trimOperationsFromTranslog(getOperationPrimaryTerm(), aboveSeqNo);
|
getEngine().trimOperationsFromTranslog(getOperationPrimaryTerm(), aboveSeqNo);
|
||||||
}
|
}
|
||||||
|
@ -1456,11 +1543,23 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return opsRecovered;
|
return opsRecovered;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void loadGlobalCheckpointToReplicationTracker() throws IOException {
|
||||||
|
// we have to set it before we open an engine and recover from the translog because
|
||||||
|
// acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in,
|
||||||
|
// and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in.
|
||||||
|
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
|
||||||
|
final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID);
|
||||||
|
replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint");
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* opens the engine on top of the existing lucene engine and translog.
|
* opens the engine on top of the existing lucene engine and translog.
|
||||||
* Operations from the translog will be replayed to bring lucene up to date.
|
* Operations from the translog will be replayed to bring lucene up to date.
|
||||||
**/
|
**/
|
||||||
public void openEngineAndRecoverFromTranslog() throws IOException {
|
public void openEngineAndRecoverFromTranslog() throws IOException {
|
||||||
|
assert recoveryState.getStage() == RecoveryState.Stage.INDEX : "unexpected recovery stage [" + recoveryState.getStage() + "]";
|
||||||
|
maybeCheckIndex();
|
||||||
|
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
|
||||||
final RecoveryState.Translog translogRecoveryStats = recoveryState.getTranslog();
|
final RecoveryState.Translog translogRecoveryStats = recoveryState.getTranslog();
|
||||||
final Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> {
|
final Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> {
|
||||||
translogRecoveryStats.totalOperations(snapshot.totalOperations());
|
translogRecoveryStats.totalOperations(snapshot.totalOperations());
|
||||||
|
@ -1468,7 +1567,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return runTranslogRecovery(engine, snapshot, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY,
|
return runTranslogRecovery(engine, snapshot, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY,
|
||||||
translogRecoveryStats::incrementRecoveredOperations);
|
translogRecoveryStats::incrementRecoveredOperations);
|
||||||
};
|
};
|
||||||
innerOpenEngineAndTranslog();
|
loadGlobalCheckpointToReplicationTracker();
|
||||||
|
innerOpenEngineAndTranslog(replicationTracker);
|
||||||
getEngine().recoverFromTranslog(translogRecoveryRunner, Long.MAX_VALUE);
|
getEngine().recoverFromTranslog(translogRecoveryRunner, Long.MAX_VALUE);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1477,36 +1577,22 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
* The translog is kept but its operations won't be replayed.
|
* The translog is kept but its operations won't be replayed.
|
||||||
*/
|
*/
|
||||||
public void openEngineAndSkipTranslogRecovery() throws IOException {
|
public void openEngineAndSkipTranslogRecovery() throws IOException {
|
||||||
innerOpenEngineAndTranslog();
|
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
|
||||||
|
assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "unexpected recovery stage [" + recoveryState.getStage() + "]";
|
||||||
|
loadGlobalCheckpointToReplicationTracker();
|
||||||
|
innerOpenEngineAndTranslog(replicationTracker);
|
||||||
getEngine().skipTranslogRecovery();
|
getEngine().skipTranslogRecovery();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void innerOpenEngineAndTranslog() throws IOException {
|
private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) throws IOException {
|
||||||
if (state != IndexShardState.RECOVERING) {
|
if (state != IndexShardState.RECOVERING) {
|
||||||
throw new IndexShardNotRecoveringException(shardId, state);
|
throw new IndexShardNotRecoveringException(shardId, state);
|
||||||
}
|
}
|
||||||
recoveryState.setStage(RecoveryState.Stage.VERIFY_INDEX);
|
final EngineConfig config = newEngineConfig(globalCheckpointSupplier);
|
||||||
// also check here, before we apply the translog
|
|
||||||
if (Booleans.isTrue(checkIndexOnStartup) || "checksum".equals(checkIndexOnStartup)) {
|
|
||||||
try {
|
|
||||||
checkIndex();
|
|
||||||
} catch (IOException ex) {
|
|
||||||
throw new RecoveryFailedException(recoveryState, "check index failed", ex);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
recoveryState.setStage(RecoveryState.Stage.TRANSLOG);
|
|
||||||
|
|
||||||
final EngineConfig config = newEngineConfig();
|
|
||||||
|
|
||||||
// we disable deletes since we allow for operations to be executed against the shard while recovering
|
// we disable deletes since we allow for operations to be executed against the shard while recovering
|
||||||
// but we need to make sure we don't loose deletes until we are done recovering
|
// but we need to make sure we don't loose deletes until we are done recovering
|
||||||
config.setEnableGcDeletes(false);
|
config.setEnableGcDeletes(false);
|
||||||
// we have to set it before we open an engine and recover from the translog because
|
|
||||||
// acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in,
|
|
||||||
// and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in.
|
|
||||||
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
|
|
||||||
final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID);
|
|
||||||
replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint");
|
|
||||||
updateRetentionLeasesOnReplica(loadRetentionLeases());
|
updateRetentionLeasesOnReplica(loadRetentionLeases());
|
||||||
assert recoveryState.getRecoverySource().expectEmptyRetentionLeases() == false || getRetentionLeases().leases().isEmpty()
|
assert recoveryState.getRecoverySource().expectEmptyRetentionLeases() == false || getRetentionLeases().leases().isEmpty()
|
||||||
: "expected empty set of retention leases with recovery source [" + recoveryState.getRecoverySource()
|
: "expected empty set of retention leases with recovery source [" + recoveryState.getRecoverySource()
|
||||||
|
@ -1551,16 +1637,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
*/
|
*/
|
||||||
public void performRecoveryRestart() throws IOException {
|
public void performRecoveryRestart() throws IOException {
|
||||||
synchronized (mutex) {
|
synchronized (mutex) {
|
||||||
if (state != IndexShardState.RECOVERING) {
|
|
||||||
throw new IndexShardNotRecoveringException(shardId, state);
|
|
||||||
}
|
|
||||||
assert refreshListeners.pendingCount() == 0 : "we can't restart with pending listeners";
|
assert refreshListeners.pendingCount() == 0 : "we can't restart with pending listeners";
|
||||||
final Engine engine = this.currentEngineReference.getAndSet(null);
|
IOUtils.close(currentEngineReference.getAndSet(null));
|
||||||
IOUtils.close(engine);
|
resetRecoveryStage();
|
||||||
recoveryState().setStage(RecoveryState.Stage.INIT);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* If a file-based recovery occurs, a recovery target calls this method to reset the recovery stage.
|
||||||
|
*/
|
||||||
|
public void resetRecoveryStage() {
|
||||||
|
assert routingEntry().recoverySource().getType() == RecoverySource.Type.PEER : "not a peer recovery [" + routingEntry() + "]";
|
||||||
|
assert currentEngineReference.get() == null;
|
||||||
|
if (state != IndexShardState.RECOVERING) {
|
||||||
|
throw new IndexShardNotRecoveringException(shardId, state);
|
||||||
|
}
|
||||||
|
recoveryState().setStage(RecoveryState.Stage.INIT);
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* returns stats about ongoing recoveries, both source and target
|
* returns stats about ongoing recoveries, both source and target
|
||||||
*/
|
*/
|
||||||
|
@ -2099,6 +2193,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
assert assertPrimaryMode();
|
assert assertPrimaryMode();
|
||||||
verifyNotClosed();
|
verifyNotClosed();
|
||||||
ensureSoftDeletesEnabled("retention leases");
|
ensureSoftDeletesEnabled("retention leases");
|
||||||
|
replicationTracker.renewPeerRecoveryRetentionLeases();
|
||||||
final Tuple<Boolean, RetentionLeases> retentionLeases = getRetentionLeases(true);
|
final Tuple<Boolean, RetentionLeases> retentionLeases = getRetentionLeases(true);
|
||||||
if (retentionLeases.v1()) {
|
if (retentionLeases.v1()) {
|
||||||
logger.trace("syncing retention leases [{}] after expiration check", retentionLeases.v2());
|
logger.trace("syncing retention leases [{}] after expiration check", retentionLeases.v2());
|
||||||
|
@ -2266,6 +2361,13 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
synchronized (mutex) {
|
synchronized (mutex) {
|
||||||
replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex
|
replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex
|
||||||
}
|
}
|
||||||
|
ensurePeerRecoveryRetentionLeasesExist();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void ensurePeerRecoveryRetentionLeasesExist() {
|
||||||
|
threadPool.generic().execute(() -> replicationTracker.createMissingPeerRecoveryRetentionLeases(ActionListener.wrap(
|
||||||
|
r -> logger.trace("created missing peer recovery retention leases"),
|
||||||
|
e -> logger.debug("failed creating missing peer recovery retention leases", e))));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -2287,6 +2389,17 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
internalIndexingStats.noopUpdate(type);
|
internalIndexingStats.noopUpdate(type);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void maybeCheckIndex() {
|
||||||
|
recoveryState.setStage(RecoveryState.Stage.VERIFY_INDEX);
|
||||||
|
if (Booleans.isTrue(checkIndexOnStartup) || "checksum".equals(checkIndexOnStartup)) {
|
||||||
|
try {
|
||||||
|
checkIndex();
|
||||||
|
} catch (IOException ex) {
|
||||||
|
throw new RecoveryFailedException(recoveryState, "check index failed", ex);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
void checkIndex() throws IOException {
|
void checkIndex() throws IOException {
|
||||||
if (store.tryIncRef()) {
|
if (store.tryIncRef()) {
|
||||||
try {
|
try {
|
||||||
|
@ -2485,6 +2598,24 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
return replicationTracker.isRelocated();
|
return replicationTracker.isRelocated();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public RetentionLease addPeerRecoveryRetentionLease(String nodeId, long globalCheckpoint,
|
||||||
|
ActionListener<ReplicationResponse> listener) {
|
||||||
|
assert assertPrimaryMode();
|
||||||
|
// only needed for BWC reasons involving rolling upgrades from versions that do not support PRRLs:
|
||||||
|
assert indexSettings.getIndexVersionCreated().before(Version.V_7_4_0);
|
||||||
|
return replicationTracker.addPeerRecoveryRetentionLease(nodeId, globalCheckpoint, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public RetentionLease cloneLocalPeerRecoveryRetentionLease(String nodeId, ActionListener<ReplicationResponse> listener) {
|
||||||
|
assert assertPrimaryMode();
|
||||||
|
return replicationTracker.cloneLocalPeerRecoveryRetentionLease(nodeId, listener);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void removePeerRecoveryRetentionLease(String nodeId, ActionListener<ReplicationResponse> listener) {
|
||||||
|
assert assertPrimaryMode();
|
||||||
|
replicationTracker.removePeerRecoveryRetentionLease(nodeId, listener);
|
||||||
|
}
|
||||||
|
|
||||||
class ShardEventListener implements Engine.EventListener {
|
class ShardEventListener implements Engine.EventListener {
|
||||||
private final CopyOnWriteArrayList<Consumer<ShardFailure>> delegates = new CopyOnWriteArrayList<>();
|
private final CopyOnWriteArrayList<Consumer<ShardFailure>> delegates = new CopyOnWriteArrayList<>();
|
||||||
|
|
||||||
|
@ -2538,7 +2669,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
mapperService.resolveDocumentType(type));
|
mapperService.resolveDocumentType(type));
|
||||||
}
|
}
|
||||||
|
|
||||||
private EngineConfig newEngineConfig() {
|
private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) {
|
||||||
Sort indexSort = indexSortSupplier.get();
|
Sort indexSort = indexSortSupplier.get();
|
||||||
return new EngineConfig(shardId, shardRouting.allocationId().getId(),
|
return new EngineConfig(shardId, shardRouting.allocationId().getId(),
|
||||||
threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(),
|
threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(),
|
||||||
|
@ -2548,7 +2679,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()),
|
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()),
|
||||||
Collections.singletonList(refreshListeners),
|
Collections.singletonList(refreshListeners),
|
||||||
Collections.singletonList(new RefreshMetricUpdater(refreshMetric)),
|
Collections.singletonList(new RefreshMetricUpdater(refreshMetric)),
|
||||||
indexSort, circuitBreakerService, replicationTracker, replicationTracker::getRetentionLeases,
|
indexSort, circuitBreakerService, globalCheckpointSupplier, replicationTracker::getRetentionLeases,
|
||||||
() -> getOperationPrimaryTerm(), tombstoneDocSupplier());
|
() -> getOperationPrimaryTerm(), tombstoneDocSupplier());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -3185,7 +3316,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
// we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata,
|
// we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata,
|
||||||
// acquireXXXCommit and close works.
|
// acquireXXXCommit and close works.
|
||||||
final Engine readOnlyEngine =
|
final Engine readOnlyEngine =
|
||||||
new ReadOnlyEngine(newEngineConfig(), seqNoStats, translogStats, false, Function.identity()) {
|
new ReadOnlyEngine(newEngineConfig(replicationTracker), seqNoStats, translogStats, false, Function.identity()) {
|
||||||
@Override
|
@Override
|
||||||
public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) {
|
public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) {
|
||||||
synchronized (mutex) {
|
synchronized (mutex) {
|
||||||
|
@ -3214,7 +3345,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine));
|
IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine));
|
||||||
newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig()));
|
newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker)));
|
||||||
onNewEngine(newEngineReference.get());
|
onNewEngine(newEngineReference.get());
|
||||||
}
|
}
|
||||||
final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery(
|
final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery(
|
||||||
|
|
|
@ -97,6 +97,7 @@ import java.util.Iterator;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
|
@ -1583,6 +1584,22 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns a {@link org.elasticsearch.index.seqno.SequenceNumbers.CommitInfo} of the safe commit if exists.
|
||||||
|
*/
|
||||||
|
public Optional<SequenceNumbers.CommitInfo> findSafeIndexCommit(long globalCheckpoint) throws IOException {
|
||||||
|
final List<IndexCommit> commits = DirectoryReader.listCommits(directory);
|
||||||
|
assert commits.isEmpty() == false : "no commit found";
|
||||||
|
final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(commits, globalCheckpoint);
|
||||||
|
final SequenceNumbers.CommitInfo commitInfo = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(safeCommit.getUserData().entrySet());
|
||||||
|
// all operations of the safe commit must be at most the global checkpoint.
|
||||||
|
if (commitInfo.maxSeqNo <= globalCheckpoint) {
|
||||||
|
return Optional.of(commitInfo);
|
||||||
|
} else {
|
||||||
|
return Optional.empty();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
private static void updateCommitData(IndexWriter writer, Map<String, String> keysToUpdate) throws IOException {
|
private static void updateCommitData(IndexWriter writer, Map<String, String> keysToUpdate) throws IOException {
|
||||||
final Map<String, String> userData = getUserData(writer);
|
final Map<String, String> userData = getUserData(writer);
|
||||||
userData.putAll(keysToUpdate);
|
userData.putAll(keysToUpdate);
|
||||||
|
|
|
@ -22,8 +22,6 @@ package org.elasticsearch.indices.recovery;
|
||||||
import org.apache.logging.log4j.LogManager;
|
import org.apache.logging.log4j.LogManager;
|
||||||
import org.apache.logging.log4j.Logger;
|
import org.apache.logging.log4j.Logger;
|
||||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
|
||||||
import org.apache.lucene.index.IndexCommit;
|
|
||||||
import org.apache.lucene.store.AlreadyClosedException;
|
import org.apache.lucene.store.AlreadyClosedException;
|
||||||
import org.apache.lucene.store.RateLimiter;
|
import org.apache.lucene.store.RateLimiter;
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
|
@ -44,18 +42,14 @@ import org.elasticsearch.common.unit.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.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
import org.elasticsearch.index.engine.CombinedDeletionPolicy;
|
|
||||||
import org.elasticsearch.index.engine.RecoveryEngineException;
|
import org.elasticsearch.index.engine.RecoveryEngineException;
|
||||||
import org.elasticsearch.index.mapper.MapperException;
|
import org.elasticsearch.index.mapper.MapperException;
|
||||||
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.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
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.TranslogCorruptedException;
|
|
||||||
import org.elasticsearch.indices.recovery.RecoveriesCollection.RecoveryRef;
|
import org.elasticsearch.indices.recovery.RecoveriesCollection.RecoveryRef;
|
||||||
import org.elasticsearch.tasks.Task;
|
import org.elasticsearch.tasks.Task;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
@ -68,12 +62,11 @@ import org.elasticsearch.transport.TransportResponseHandler;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.List;
|
|
||||||
import java.util.StringJoiner;
|
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.function.Consumer;
|
import java.util.function.Consumer;
|
||||||
|
|
||||||
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
import static org.elasticsearch.common.unit.TimeValue.timeValueMillis;
|
||||||
|
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The recovery target handles recoveries of peer shards of the shard+node to recover to.
|
* The recovery target handles recoveries of peer shards of the shard+node to recover to.
|
||||||
|
@ -178,9 +171,12 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
cancellableThreads = recoveryTarget.cancellableThreads();
|
cancellableThreads = recoveryTarget.cancellableThreads();
|
||||||
try {
|
try {
|
||||||
assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node";
|
assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node";
|
||||||
request = getStartRecoveryRequest(recoveryTarget);
|
|
||||||
logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId());
|
logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId());
|
||||||
recoveryTarget.indexShard().prepareForIndexRecovery();
|
recoveryTarget.indexShard().prepareForIndexRecovery();
|
||||||
|
final long startingSeqNo = recoveryTarget.indexShard().recoverLocallyUpToGlobalCheckpoint();
|
||||||
|
assert startingSeqNo == UNASSIGNED_SEQ_NO || recoveryTarget.state().getStage() == RecoveryState.Stage.TRANSLOG :
|
||||||
|
"unexpected recovery stage [" + recoveryTarget.state().getStage() + "] starting seqno [ " + startingSeqNo + "]";
|
||||||
|
request = getStartRecoveryRequest(logger, clusterService.localNode(), recoveryTarget, startingSeqNo);
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
// this will be logged as warning later on...
|
// this will be logged as warning later on...
|
||||||
logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e);
|
logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e);
|
||||||
|
@ -313,60 +309,45 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Obtains a snapshot of the store metadata for the recovery target.
|
|
||||||
*
|
|
||||||
* @param recoveryTarget the target of the recovery
|
|
||||||
* @return a snapshot of the store metadata
|
|
||||||
*/
|
|
||||||
private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) {
|
|
||||||
try {
|
|
||||||
return recoveryTarget.indexShard().snapshotStoreMetadata();
|
|
||||||
} catch (final org.apache.lucene.index.IndexNotFoundException e) {
|
|
||||||
// happens on an empty folder. no need to log
|
|
||||||
logger.trace("{} shard folder empty, recovering all files", recoveryTarget);
|
|
||||||
return Store.MetadataSnapshot.EMPTY;
|
|
||||||
} catch (final IOException e) {
|
|
||||||
logger.warn("error while listing local files, recovering as if there are none", e);
|
|
||||||
return Store.MetadataSnapshot.EMPTY;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Prepare the start recovery request.
|
* Prepare the start recovery request.
|
||||||
*
|
*
|
||||||
|
* @param logger the logger
|
||||||
|
* @param localNode the local node of the recovery target
|
||||||
* @param recoveryTarget the target of the recovery
|
* @param recoveryTarget the target of the recovery
|
||||||
|
* @param startingSeqNo a sequence number that an operation-based peer recovery can start with.
|
||||||
|
* This is the first operation after the local checkpoint of the safe commit if exists.
|
||||||
* @return a start recovery request
|
* @return a start recovery request
|
||||||
*/
|
*/
|
||||||
private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recoveryTarget) {
|
public static StartRecoveryRequest getStartRecoveryRequest(Logger logger, DiscoveryNode localNode,
|
||||||
|
RecoveryTarget recoveryTarget, long startingSeqNo) {
|
||||||
final StartRecoveryRequest request;
|
final StartRecoveryRequest request;
|
||||||
logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode());
|
logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode());
|
||||||
|
|
||||||
final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget);
|
Store.MetadataSnapshot metadataSnapshot;
|
||||||
|
try {
|
||||||
|
metadataSnapshot = recoveryTarget.indexShard().snapshotStoreMetadata();
|
||||||
|
} catch (final org.apache.lucene.index.IndexNotFoundException e) {
|
||||||
|
// happens on an empty folder. no need to log
|
||||||
|
assert startingSeqNo == UNASSIGNED_SEQ_NO : startingSeqNo;
|
||||||
|
logger.trace("{} shard folder empty, recovering all files", recoveryTarget);
|
||||||
|
metadataSnapshot = Store.MetadataSnapshot.EMPTY;
|
||||||
|
} catch (final IOException e) {
|
||||||
|
if (startingSeqNo != UNASSIGNED_SEQ_NO) {
|
||||||
|
logger.warn(new ParameterizedMessage("error while listing local files, resetting the starting sequence number from {} " +
|
||||||
|
"to unassigned and recovering as if there are none", startingSeqNo), e);
|
||||||
|
startingSeqNo = UNASSIGNED_SEQ_NO;
|
||||||
|
} else {
|
||||||
|
logger.warn("error while listing local files, recovering as if there are none", e);
|
||||||
|
}
|
||||||
|
metadataSnapshot = Store.MetadataSnapshot.EMPTY;
|
||||||
|
}
|
||||||
logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size());
|
logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size());
|
||||||
|
|
||||||
final long startingSeqNo;
|
|
||||||
if (metadataSnapshot.size() > 0) {
|
|
||||||
startingSeqNo = getStartingSeqNo(logger, recoveryTarget);
|
|
||||||
} else {
|
|
||||||
startingSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (startingSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO) {
|
|
||||||
logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode());
|
|
||||||
} else {
|
|
||||||
logger.trace(
|
|
||||||
"{} preparing for sequence-number-based recovery starting at sequence number [{}] from [{}]",
|
|
||||||
recoveryTarget.shardId(),
|
|
||||||
startingSeqNo,
|
|
||||||
recoveryTarget.sourceNode());
|
|
||||||
}
|
|
||||||
|
|
||||||
request = new StartRecoveryRequest(
|
request = new StartRecoveryRequest(
|
||||||
recoveryTarget.shardId(),
|
recoveryTarget.shardId(),
|
||||||
recoveryTarget.indexShard().routingEntry().allocationId().getId(),
|
recoveryTarget.indexShard().routingEntry().allocationId().getId(),
|
||||||
recoveryTarget.sourceNode(),
|
recoveryTarget.sourceNode(),
|
||||||
clusterService.localNode(),
|
localNode,
|
||||||
metadataSnapshot,
|
metadataSnapshot,
|
||||||
recoveryTarget.state().getPrimary(),
|
recoveryTarget.state().getPrimary(),
|
||||||
recoveryTarget.recoveryId(),
|
recoveryTarget.recoveryId(),
|
||||||
|
@ -374,50 +355,6 @@ public class PeerRecoveryTargetService implements IndexEventListener {
|
||||||
return request;
|
return request;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Get the starting sequence number for a sequence-number-based request.
|
|
||||||
*
|
|
||||||
* @param recoveryTarget the target of the recovery
|
|
||||||
* @return the starting sequence number or {@link SequenceNumbers#UNASSIGNED_SEQ_NO} if obtaining the starting sequence number
|
|
||||||
* failed
|
|
||||||
*/
|
|
||||||
public static long getStartingSeqNo(final Logger logger, final RecoveryTarget recoveryTarget) {
|
|
||||||
try {
|
|
||||||
final Store store = recoveryTarget.store();
|
|
||||||
final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY);
|
|
||||||
final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation(), translogUUID);
|
|
||||||
final List<IndexCommit> existingCommits = DirectoryReader.listCommits(store.directory());
|
|
||||||
final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, globalCheckpoint);
|
|
||||||
final SequenceNumbers.CommitInfo seqNoStats = Store.loadSeqNoInfo(safeCommit);
|
|
||||||
if (logger.isTraceEnabled()) {
|
|
||||||
final StringJoiner descriptionOfExistingCommits = new StringJoiner(",");
|
|
||||||
for (IndexCommit commit : existingCommits) {
|
|
||||||
descriptionOfExistingCommits.add(CombinedDeletionPolicy.commitDescription(commit));
|
|
||||||
}
|
|
||||||
logger.trace("Calculate starting seqno based on global checkpoint [{}], safe commit [{}], existing commits [{}]",
|
|
||||||
globalCheckpoint, CombinedDeletionPolicy.commitDescription(safeCommit), descriptionOfExistingCommits);
|
|
||||||
}
|
|
||||||
if (seqNoStats.maxSeqNo <= globalCheckpoint) {
|
|
||||||
assert seqNoStats.localCheckpoint <= globalCheckpoint;
|
|
||||||
/*
|
|
||||||
* Commit point is good for sequence-number based recovery as the maximum sequence number included in it is below the global
|
|
||||||
* checkpoint (i.e., it excludes any operations that may not be on the primary). Recovery will start at the first operation
|
|
||||||
* after the local checkpoint stored in the commit.
|
|
||||||
*/
|
|
||||||
return seqNoStats.localCheckpoint + 1;
|
|
||||||
} else {
|
|
||||||
return SequenceNumbers.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
|
||||||
} catch (final TranslogCorruptedException | IOException e) {
|
|
||||||
/*
|
|
||||||
* This can happen, for example, if a phase one of the recovery completed successfully, a network partition happens before the
|
|
||||||
* translog on the recovery target is opened, the recovery enters a retry loop seeing now that the index files are on disk and
|
|
||||||
* proceeds to attempt a sequence-number-based recovery.
|
|
||||||
*/
|
|
||||||
return SequenceNumbers.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public interface RecoveryListener {
|
public interface RecoveryListener {
|
||||||
void onRecoveryDone(RecoveryState state);
|
void onRecoveryDone(RecoveryState state);
|
||||||
|
|
||||||
|
|
|
@ -29,10 +29,14 @@ import org.apache.lucene.store.IOContext;
|
||||||
import org.apache.lucene.store.IndexInput;
|
import org.apache.lucene.store.IndexInput;
|
||||||
import org.apache.lucene.store.RateLimiter;
|
import org.apache.lucene.store.RateLimiter;
|
||||||
import org.apache.lucene.util.ArrayUtil;
|
import org.apache.lucene.util.ArrayUtil;
|
||||||
|
import org.apache.lucene.util.SetOnce;
|
||||||
import org.elasticsearch.ExceptionsHelper;
|
import org.elasticsearch.ExceptionsHelper;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.StepListener;
|
import org.elasticsearch.action.StepListener;
|
||||||
|
import org.elasticsearch.action.support.ThreadedActionListener;
|
||||||
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
import org.elasticsearch.common.CheckedSupplier;
|
import org.elasticsearch.common.CheckedSupplier;
|
||||||
|
@ -51,6 +55,9 @@ import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||||
import org.elasticsearch.core.internal.io.IOUtils;
|
import org.elasticsearch.core.internal.io.IOUtils;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.engine.RecoveryEngineException;
|
import org.elasticsearch.index.engine.RecoveryEngineException;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
|
@ -142,8 +149,14 @@ public class RecoverySourceHandler {
|
||||||
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
||||||
throw e;
|
throw e;
|
||||||
});
|
});
|
||||||
final Consumer<Exception> onFailure = e ->
|
final Consumer<Exception> onFailure = e -> {
|
||||||
|
assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[onFailure]");
|
||||||
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e));
|
||||||
|
};
|
||||||
|
|
||||||
|
final boolean useRetentionLeases = shard.indexSettings().isSoftDeleteEnabled()
|
||||||
|
&& shard.indexSettings().getIndexMetaData().getState() != IndexMetaData.State.CLOSE;
|
||||||
|
final SetOnce<RetentionLease> retentionLeaseRef = new SetOnce<>();
|
||||||
|
|
||||||
runUnderPrimaryPermit(() -> {
|
runUnderPrimaryPermit(() -> {
|
||||||
final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable();
|
final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable();
|
||||||
|
@ -154,13 +167,35 @@ public class RecoverySourceHandler {
|
||||||
throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node");
|
throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node");
|
||||||
}
|
}
|
||||||
assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting;
|
assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting;
|
||||||
|
retentionLeaseRef.set(useRetentionLeases ? shard.getRetentionLeases().get(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting)) : null);
|
||||||
}, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ",
|
}, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ",
|
||||||
shard, cancellableThreads, logger);
|
shard, cancellableThreads, logger);
|
||||||
final Closeable retentionLock = shard.acquireRetentionLock();
|
final Closeable retentionLock = shard.acquireRetentionLock();
|
||||||
resources.add(retentionLock);
|
resources.add(retentionLock);
|
||||||
final long startingSeqNo;
|
final long startingSeqNo;
|
||||||
final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO &&
|
final boolean isSequenceNumberBasedRecovery
|
||||||
isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo());
|
= request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO
|
||||||
|
&& isTargetSameHistory()
|
||||||
|
&& shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo())
|
||||||
|
&& (useRetentionLeases == false
|
||||||
|
|| (retentionLeaseRef.get() != null && retentionLeaseRef.get().retainingSequenceNumber() <= request.startingSeqNo()));
|
||||||
|
// NB check hasCompleteHistoryOperations when computing isSequenceNumberBasedRecovery, even if there is a retention lease,
|
||||||
|
// because when doing a rolling upgrade from earlier than 7.4 we may create some leases that are initially unsatisfied. It's
|
||||||
|
// possible there are other cases where we cannot satisfy all leases, because that's not a property we currently expect to hold.
|
||||||
|
// Also it's pretty cheap when soft deletes are enabled, and it'd be a disaster if we tried a sequence-number-based recovery
|
||||||
|
// without having a complete history.
|
||||||
|
|
||||||
|
if (isSequenceNumberBasedRecovery && useRetentionLeases) {
|
||||||
|
// all the history we need is retained by an existing retention lease, so we do not need a separate retention lock
|
||||||
|
retentionLock.close();
|
||||||
|
logger.trace("history is retained by {}", retentionLeaseRef.get());
|
||||||
|
} else {
|
||||||
|
// all the history we need is retained by the retention lock, obtained before calling shard.hasCompleteHistoryOperations()
|
||||||
|
// and before acquiring the safe commit we'll be using, so we can be certain that all operations after the safe commit's
|
||||||
|
// local checkpoint will be retained for the duration of this recovery.
|
||||||
|
logger.trace("history is retained by retention lock");
|
||||||
|
}
|
||||||
|
|
||||||
final StepListener<SendFileResult> sendFileStep = new StepListener<>();
|
final StepListener<SendFileResult> sendFileStep = new StepListener<>();
|
||||||
final StepListener<TimeValue> prepareEngineStep = new StepListener<>();
|
final StepListener<TimeValue> prepareEngineStep = new StepListener<>();
|
||||||
|
@ -179,9 +214,22 @@ public class RecoverySourceHandler {
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e);
|
throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e);
|
||||||
}
|
}
|
||||||
// We need to set this to 0 to create a translog roughly according to the retention policy on the target. Note that it will
|
|
||||||
// still filter out legacy operations without seqNo.
|
// Try and copy enough operations to the recovering peer so that if it is promoted to primary then it has a chance of being
|
||||||
startingSeqNo = 0;
|
// able to recover other replicas using operations-based recoveries. If we are not using retention leases then we
|
||||||
|
// conservatively copy all available operations. If we are using retention leases then "enough operations" is just the
|
||||||
|
// operations from the local checkpoint of the safe commit onwards, because when using soft deletes the safe commit retains
|
||||||
|
// at least as much history as anything else. The safe commit will often contain all the history retained by the current set
|
||||||
|
// of retention leases, but this is not guaranteed: an earlier peer recovery from a different primary might have created a
|
||||||
|
// retention lease for some history that this primary already discarded, since we discard history when the global checkpoint
|
||||||
|
// advances and not when creating a new safe commit. In any case this is a best-effort thing since future recoveries can
|
||||||
|
// always fall back to file-based ones, and only really presents a problem if this primary fails before things have settled
|
||||||
|
// down.
|
||||||
|
startingSeqNo = useRetentionLeases
|
||||||
|
? Long.parseLong(safeCommitRef.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1L
|
||||||
|
: 0;
|
||||||
|
logger.trace("performing file-based recovery followed by history replay starting at [{}]", startingSeqNo);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo);
|
final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo);
|
||||||
shard.store().incRef();
|
shard.store().incRef();
|
||||||
|
@ -194,7 +242,40 @@ public class RecoverySourceHandler {
|
||||||
logger.warn("releasing snapshot caused exception", ex);
|
logger.warn("releasing snapshot caused exception", ex);
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
phase1(safeCommitRef.getIndexCommit(), shard.getLastKnownGlobalCheckpoint(), () -> estimateNumOps, sendFileStep);
|
|
||||||
|
final StepListener<ReplicationResponse> deleteRetentionLeaseStep = new StepListener<>();
|
||||||
|
if (useRetentionLeases) {
|
||||||
|
runUnderPrimaryPermit(() -> {
|
||||||
|
try {
|
||||||
|
// If the target previously had a copy of this shard then a file-based recovery might move its global
|
||||||
|
// checkpoint backwards. We must therefore remove any existing retention lease so that we can create a
|
||||||
|
// new one later on in the recovery.
|
||||||
|
shard.removePeerRecoveryRetentionLease(request.targetNode().getId(),
|
||||||
|
new ThreadedActionListener<>(logger, shard.getThreadPool(), ThreadPool.Names.GENERIC,
|
||||||
|
deleteRetentionLeaseStep, false));
|
||||||
|
} catch (RetentionLeaseNotFoundException e) {
|
||||||
|
logger.debug("no peer-recovery retention lease for " + request.targetAllocationId());
|
||||||
|
deleteRetentionLeaseStep.onResponse(null);
|
||||||
|
}
|
||||||
|
}, shardId + " removing retention leaes for [" + request.targetAllocationId() + "]",
|
||||||
|
shard, cancellableThreads, logger);
|
||||||
|
} else {
|
||||||
|
deleteRetentionLeaseStep.onResponse(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
deleteRetentionLeaseStep.whenComplete(ignored -> {
|
||||||
|
assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[phase1]");
|
||||||
|
|
||||||
|
final Consumer<ActionListener<RetentionLease>> createRetentionLeaseAsync;
|
||||||
|
if (useRetentionLeases) {
|
||||||
|
createRetentionLeaseAsync = l -> createRetentionLease(startingSeqNo, l);
|
||||||
|
} else {
|
||||||
|
createRetentionLeaseAsync = l -> l.onResponse(null);
|
||||||
|
}
|
||||||
|
|
||||||
|
phase1(safeCommitRef.getIndexCommit(), createRetentionLeaseAsync, () -> estimateNumOps, sendFileStep);
|
||||||
|
}, onFailure);
|
||||||
|
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e);
|
throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e);
|
||||||
}
|
}
|
||||||
|
@ -202,12 +283,14 @@ public class RecoverySourceHandler {
|
||||||
assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo;
|
assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo;
|
||||||
|
|
||||||
sendFileStep.whenComplete(r -> {
|
sendFileStep.whenComplete(r -> {
|
||||||
|
assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[prepareTargetForTranslog]");
|
||||||
// For a sequence based recovery, the target can keep its local translog
|
// For a sequence based recovery, the target can keep its local translog
|
||||||
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false,
|
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false,
|
||||||
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep);
|
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep);
|
||||||
}, onFailure);
|
}, onFailure);
|
||||||
|
|
||||||
prepareEngineStep.whenComplete(prepareEngineTime -> {
|
prepareEngineStep.whenComplete(prepareEngineTime -> {
|
||||||
|
assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[phase2]");
|
||||||
/*
|
/*
|
||||||
* add shard to replication group (shard will receive replication requests from this point on) now that engine is open.
|
* add shard to replication group (shard will receive replication requests from this point on) now that engine is open.
|
||||||
* This means that any document indexed into the primary after this will be replicated to this replica as well
|
* This means that any document indexed into the primary after this will be replicated to this replica as well
|
||||||
|
@ -218,14 +301,16 @@ public class RecoverySourceHandler {
|
||||||
shardId + " initiating tracking of " + request.targetAllocationId(), shard, cancellableThreads, logger);
|
shardId + " initiating tracking of " + request.targetAllocationId(), shard, cancellableThreads, logger);
|
||||||
|
|
||||||
final long endingSeqNo = shard.seqNoStats().getMaxSeqNo();
|
final long endingSeqNo = shard.seqNoStats().getMaxSeqNo();
|
||||||
if (logger.isTraceEnabled()) {
|
logger.trace("snapshot translog for recovery; current size is [{}]",
|
||||||
logger.trace("snapshot translog for recovery; current size is [{}]",
|
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo));
|
||||||
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo));
|
|
||||||
}
|
|
||||||
final Translog.Snapshot phase2Snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo);
|
final Translog.Snapshot phase2Snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo);
|
||||||
resources.add(phase2Snapshot);
|
resources.add(phase2Snapshot);
|
||||||
// we can release the retention lock here because the snapshot itself will retain the required operations.
|
|
||||||
retentionLock.close();
|
if (useRetentionLeases == false || isSequenceNumberBasedRecovery == false) {
|
||||||
|
// we can release the retention lock here because the snapshot itself will retain the required operations.
|
||||||
|
retentionLock.close();
|
||||||
|
}
|
||||||
|
|
||||||
// we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values
|
// we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values
|
||||||
// are at least as high as the corresponding values on the primary when any of these operations were executed on it.
|
// are at least as high as the corresponding values on the primary when any of these operations were executed on it.
|
||||||
final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp();
|
final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp();
|
||||||
|
@ -345,16 +430,9 @@ public class RecoverySourceHandler {
|
||||||
* segments that are missing. Only segments that have the same size and
|
* segments that are missing. Only segments that have the same size and
|
||||||
* checksum can be reused
|
* checksum can be reused
|
||||||
*/
|
*/
|
||||||
void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
void phase1(IndexCommit snapshot, Consumer<ActionListener<RetentionLease>> createRetentionLease,
|
||||||
|
IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
||||||
cancellableThreads.checkForCancel();
|
cancellableThreads.checkForCancel();
|
||||||
// Total size of segment files that are recovered
|
|
||||||
long totalSizeInBytes = 0;
|
|
||||||
// Total size of segment files that were able to be re-used
|
|
||||||
long existingTotalSizeInBytes = 0;
|
|
||||||
final List<String> phase1FileNames = new ArrayList<>();
|
|
||||||
final List<Long> phase1FileSizes = new ArrayList<>();
|
|
||||||
final List<String> phase1ExistingFileNames = new ArrayList<>();
|
|
||||||
final List<Long> phase1ExistingFileSizes = new ArrayList<>();
|
|
||||||
final Store store = shard.store();
|
final Store store = shard.store();
|
||||||
try {
|
try {
|
||||||
StopWatch stopWatch = new StopWatch().start();
|
StopWatch stopWatch = new StopWatch().start();
|
||||||
|
@ -374,6 +452,16 @@ public class RecoverySourceHandler {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (canSkipPhase1(recoverySourceMetadata, request.metadataSnapshot()) == false) {
|
if (canSkipPhase1(recoverySourceMetadata, request.metadataSnapshot()) == false) {
|
||||||
|
final List<String> phase1FileNames = new ArrayList<>();
|
||||||
|
final List<Long> phase1FileSizes = new ArrayList<>();
|
||||||
|
final List<String> phase1ExistingFileNames = new ArrayList<>();
|
||||||
|
final List<Long> phase1ExistingFileSizes = new ArrayList<>();
|
||||||
|
|
||||||
|
// Total size of segment files that are recovered
|
||||||
|
long totalSizeInBytes = 0;
|
||||||
|
// Total size of segment files that were able to be re-used
|
||||||
|
long existingTotalSizeInBytes = 0;
|
||||||
|
|
||||||
// Generate a "diff" of all the identical, different, and missing
|
// Generate a "diff" of all the identical, different, and missing
|
||||||
// segment files on the target node, using the existing files on
|
// segment files on the target node, using the existing files on
|
||||||
// the source node
|
// the source node
|
||||||
|
@ -408,6 +496,7 @@ public class RecoverySourceHandler {
|
||||||
phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes));
|
phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes));
|
||||||
final StepListener<Void> sendFileInfoStep = new StepListener<>();
|
final StepListener<Void> sendFileInfoStep = new StepListener<>();
|
||||||
final StepListener<Void> sendFilesStep = new StepListener<>();
|
final StepListener<Void> sendFilesStep = new StepListener<>();
|
||||||
|
final StepListener<RetentionLease> createRetentionLeaseStep = new StepListener<>();
|
||||||
final StepListener<Void> cleanFilesStep = new StepListener<>();
|
final StepListener<Void> cleanFilesStep = new StepListener<>();
|
||||||
cancellableThreads.execute(() ->
|
cancellableThreads.execute(() ->
|
||||||
recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames,
|
recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames,
|
||||||
|
@ -416,8 +505,20 @@ public class RecoverySourceHandler {
|
||||||
sendFileInfoStep.whenComplete(r ->
|
sendFileInfoStep.whenComplete(r ->
|
||||||
sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFilesStep), listener::onFailure);
|
sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFilesStep), listener::onFailure);
|
||||||
|
|
||||||
sendFilesStep.whenComplete(r ->
|
sendFilesStep.whenComplete(r -> createRetentionLease.accept(createRetentionLeaseStep), listener::onFailure);
|
||||||
cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep), listener::onFailure);
|
|
||||||
|
createRetentionLeaseStep.whenComplete(retentionLease ->
|
||||||
|
{
|
||||||
|
final long lastKnownGlobalCheckpoint = shard.getLastKnownGlobalCheckpoint();
|
||||||
|
assert retentionLease == null || retentionLease.retainingSequenceNumber() - 1 <= lastKnownGlobalCheckpoint
|
||||||
|
: retentionLease + " vs " + lastKnownGlobalCheckpoint;
|
||||||
|
// Establishes new empty translog on the replica with global checkpoint set to lastKnownGlobalCheckpoint. We want
|
||||||
|
// the commit we just copied to be a safe commit on the replica, so why not set the global checkpoint on the replica
|
||||||
|
// to the max seqno of this commit? Because (in rare corner cases) this commit might not be a safe commit here on
|
||||||
|
// the primary, and in these cases the max seqno would be too high to be valid as a global checkpoint.
|
||||||
|
cleanFiles(store, recoverySourceMetadata, translogOps, lastKnownGlobalCheckpoint, cleanFilesStep);
|
||||||
|
},
|
||||||
|
listener::onFailure);
|
||||||
|
|
||||||
final long totalSize = totalSizeInBytes;
|
final long totalSize = totalSizeInBytes;
|
||||||
final long existingTotalSize = existingTotalSizeInBytes;
|
final long existingTotalSize = existingTotalSizeInBytes;
|
||||||
|
@ -428,18 +529,59 @@ public class RecoverySourceHandler {
|
||||||
phase1ExistingFileSizes, existingTotalSize, took));
|
phase1ExistingFileSizes, existingTotalSize, took));
|
||||||
}, listener::onFailure);
|
}, listener::onFailure);
|
||||||
} else {
|
} else {
|
||||||
logger.trace("skipping [phase1]- identical sync id [{}] found on both source and target",
|
logger.trace("skipping [phase1] since source and target have identical sync id [{}]", recoverySourceMetadata.getSyncId());
|
||||||
recoverySourceMetadata.getSyncId());
|
|
||||||
final TimeValue took = stopWatch.totalTime();
|
// but we must still create a retention lease
|
||||||
logger.trace("recovery [phase1]: took [{}]", took);
|
final StepListener<RetentionLease> createRetentionLeaseStep = new StepListener<>();
|
||||||
listener.onResponse(new SendFileResult(phase1FileNames, phase1FileSizes, totalSizeInBytes, phase1ExistingFileNames,
|
createRetentionLease.accept(createRetentionLeaseStep);
|
||||||
phase1ExistingFileSizes, existingTotalSizeInBytes, took));
|
createRetentionLeaseStep.whenComplete(retentionLease -> {
|
||||||
|
final TimeValue took = stopWatch.totalTime();
|
||||||
|
logger.trace("recovery [phase1]: took [{}]", took);
|
||||||
|
listener.onResponse(new SendFileResult(Collections.emptyList(), Collections.emptyList(), 0L, Collections.emptyList(),
|
||||||
|
Collections.emptyList(), 0L, took));
|
||||||
|
}, listener::onFailure);
|
||||||
|
|
||||||
}
|
}
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
throw new RecoverFilesRecoveryException(request.shardId(), phase1FileNames.size(), new ByteSizeValue(totalSizeInBytes), e);
|
throw new RecoverFilesRecoveryException(request.shardId(), 0, new ByteSizeValue(0L), e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private void createRetentionLease(final long startingSeqNo, ActionListener<RetentionLease> listener) {
|
||||||
|
runUnderPrimaryPermit(() -> {
|
||||||
|
// Clone the peer recovery retention lease belonging to the source shard. We are retaining history between the the local
|
||||||
|
// checkpoint of the safe commit we're creating and this lease's retained seqno with the retention lock, and by cloning an
|
||||||
|
// existing lease we (approximately) know that all our peers are also retaining history as requested by the cloned lease. If
|
||||||
|
// the recovery now fails before copying enough history over then a subsequent attempt will find this lease, determine it is
|
||||||
|
// not enough, and fall back to a file-based recovery.
|
||||||
|
//
|
||||||
|
// (approximately) because we do not guarantee to be able to satisfy every lease on every peer.
|
||||||
|
logger.trace("cloning primary's retention lease");
|
||||||
|
try {
|
||||||
|
final StepListener<ReplicationResponse> cloneRetentionLeaseStep = new StepListener<>();
|
||||||
|
final RetentionLease clonedLease
|
||||||
|
= shard.cloneLocalPeerRecoveryRetentionLease(request.targetNode().getId(),
|
||||||
|
new ThreadedActionListener<>(logger, shard.getThreadPool(),
|
||||||
|
ThreadPool.Names.GENERIC, cloneRetentionLeaseStep, false));
|
||||||
|
logger.trace("cloned primary's retention lease as [{}]", clonedLease);
|
||||||
|
cloneRetentionLeaseStep.whenComplete(rr -> listener.onResponse(clonedLease), listener::onFailure);
|
||||||
|
} catch (RetentionLeaseNotFoundException e) {
|
||||||
|
// it's possible that the primary has no retention lease yet if we are doing a rolling upgrade from a version before
|
||||||
|
// 7.4, and in that case we just create a lease using the local checkpoint of the safe commit which we're using for
|
||||||
|
// recovery as a conservative estimate for the global checkpoint.
|
||||||
|
assert shard.indexSettings().getIndexVersionCreated().before(Version.V_7_4_0);
|
||||||
|
final StepListener<ReplicationResponse> addRetentionLeaseStep = new StepListener<>();
|
||||||
|
final long estimatedGlobalCheckpoint = startingSeqNo - 1;
|
||||||
|
final RetentionLease newLease = shard.addPeerRecoveryRetentionLease(request.targetNode().getId(),
|
||||||
|
estimatedGlobalCheckpoint, new ThreadedActionListener<>(logger, shard.getThreadPool(),
|
||||||
|
ThreadPool.Names.GENERIC, addRetentionLeaseStep, false));
|
||||||
|
addRetentionLeaseStep.whenComplete(rr -> listener.onResponse(newLease), listener::onFailure);
|
||||||
|
logger.trace("created retention lease with estimated checkpoint of [{}]", estimatedGlobalCheckpoint);
|
||||||
|
}
|
||||||
|
}, shardId + " establishing retention lease for [" + request.targetAllocationId() + "]",
|
||||||
|
shard, cancellableThreads, logger);
|
||||||
|
}
|
||||||
|
|
||||||
boolean canSkipPhase1(Store.MetadataSnapshot source, Store.MetadataSnapshot target) {
|
boolean canSkipPhase1(Store.MetadataSnapshot source, Store.MetadataSnapshot target) {
|
||||||
if (source.getSyncId() == null || source.getSyncId().equals(target.getSyncId()) == false) {
|
if (source.getSyncId() == null || source.getSyncId().equals(target.getSyncId()) == false) {
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
|
|
||||||
package org.elasticsearch.indices.recovery;
|
package org.elasticsearch.indices.recovery;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
@ -33,6 +34,7 @@ import org.elasticsearch.common.xcontent.ToXContentFragment;
|
||||||
import org.elasticsearch.common.xcontent.ToXContentObject;
|
import org.elasticsearch.common.xcontent.ToXContentObject;
|
||||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -461,6 +463,7 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
private int recovered;
|
private int recovered;
|
||||||
private int total = UNKNOWN;
|
private int total = UNKNOWN;
|
||||||
private int totalOnStart = UNKNOWN;
|
private int totalOnStart = UNKNOWN;
|
||||||
|
private int totalLocal = UNKNOWN;
|
||||||
|
|
||||||
public Translog() {
|
public Translog() {
|
||||||
}
|
}
|
||||||
|
@ -470,6 +473,9 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
recovered = in.readVInt();
|
recovered = in.readVInt();
|
||||||
total = in.readVInt();
|
total = in.readVInt();
|
||||||
totalOnStart = in.readVInt();
|
totalOnStart = in.readVInt();
|
||||||
|
if (in.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||||
|
totalLocal = in.readVInt();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -478,6 +484,9 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
out.writeVInt(recovered);
|
out.writeVInt(recovered);
|
||||||
out.writeVInt(total);
|
out.writeVInt(total);
|
||||||
out.writeVInt(totalOnStart);
|
out.writeVInt(totalOnStart);
|
||||||
|
if (out.getVersion().onOrAfter(Version.V_7_4_0)) {
|
||||||
|
out.writeVInt(totalLocal);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void reset() {
|
public synchronized void reset() {
|
||||||
|
@ -485,6 +494,7 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
recovered = 0;
|
recovered = 0;
|
||||||
total = UNKNOWN;
|
total = UNKNOWN;
|
||||||
totalOnStart = UNKNOWN;
|
totalOnStart = UNKNOWN;
|
||||||
|
totalLocal = UNKNOWN;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void incrementRecoveredOperations() {
|
public synchronized void incrementRecoveredOperations() {
|
||||||
|
@ -526,8 +536,8 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void totalOperations(int total) {
|
public synchronized void totalOperations(int total) {
|
||||||
this.total = total;
|
this.total = totalLocal == UNKNOWN ? total : totalLocal + total;
|
||||||
assert total == UNKNOWN || total >= recovered : "total, if known, should be > recovered. total [" + total +
|
assert total == UNKNOWN || this.total >= recovered : "total, if known, should be > recovered. total [" + total +
|
||||||
"], recovered [" + recovered + "]";
|
"], recovered [" + recovered + "]";
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -542,7 +552,20 @@ public class RecoveryState implements ToXContentFragment, Writeable {
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void totalOperationsOnStart(int total) {
|
public synchronized void totalOperationsOnStart(int total) {
|
||||||
this.totalOnStart = total;
|
this.totalOnStart = totalLocal == UNKNOWN ? total : totalLocal + total;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Sets the total number of translog operations to be recovered locally before performing peer recovery
|
||||||
|
* @see IndexShard#recoverLocallyUpToGlobalCheckpoint()
|
||||||
|
*/
|
||||||
|
public synchronized void totalLocal(int totalLocal) {
|
||||||
|
assert totalLocal >= recovered : totalLocal + " < " + recovered;
|
||||||
|
this.totalLocal = totalLocal;
|
||||||
|
}
|
||||||
|
|
||||||
|
public synchronized int totalLocal() {
|
||||||
|
return totalLocal;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized float recoveredPercent() {
|
public synchronized float recoveredPercent() {
|
||||||
|
|
|
@ -377,6 +377,8 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
int totalTranslogOps,
|
int totalTranslogOps,
|
||||||
ActionListener<Void> listener) {
|
ActionListener<Void> listener) {
|
||||||
ActionListener.completeWith(listener, () -> {
|
ActionListener.completeWith(listener, () -> {
|
||||||
|
indexShard.resetRecoveryStage();
|
||||||
|
indexShard.prepareForIndexRecovery();
|
||||||
final RecoveryState.Index index = state().getIndex();
|
final RecoveryState.Index index = state().getIndex();
|
||||||
for (int i = 0; i < phase1ExistingFileNames.size(); i++) {
|
for (int i = 0; i < phase1ExistingFileNames.size(); i++) {
|
||||||
index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true);
|
index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true);
|
||||||
|
@ -417,7 +419,8 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
||||||
} else {
|
} else {
|
||||||
assert indexShard.assertRetentionLeasesPersisted();
|
assert indexShard.assertRetentionLeasesPersisted();
|
||||||
}
|
}
|
||||||
|
indexShard.maybeCheckIndex();
|
||||||
|
state().setStage(RecoveryState.Stage.TRANSLOG);
|
||||||
} catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) {
|
} catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) {
|
||||||
// this is a fatal exception at this stage.
|
// this is a fatal exception at this stage.
|
||||||
// this means we transferred files from the remote that have not be checksummed and they are
|
// this means we transferred files from the remote that have not be checksummed and they are
|
||||||
|
|
|
@ -40,6 +40,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.env.NodeEnvironment;
|
import org.elasticsearch.env.NodeEnvironment;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
|
import org.elasticsearch.index.IndexService;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.MergePolicyConfig;
|
import org.elasticsearch.index.MergePolicyConfig;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
|
@ -427,8 +428,12 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
|
||||||
.setSettings(Settings.builder()
|
.setSettings(Settings.builder()
|
||||||
.put("number_of_shards", 1)
|
.put("number_of_shards", 1)
|
||||||
.put("number_of_replicas", 1)
|
.put("number_of_replicas", 1)
|
||||||
|
|
||||||
// disable merges to keep segments the same
|
// disable merges to keep segments the same
|
||||||
.put(MergePolicyConfig.INDEX_MERGE_ENABLED, "false")
|
.put(MergePolicyConfig.INDEX_MERGE_ENABLED, false)
|
||||||
|
|
||||||
|
// expire retention leases quickly
|
||||||
|
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "100ms")
|
||||||
).get();
|
).get();
|
||||||
|
|
||||||
logger.info("--> indexing docs");
|
logger.info("--> indexing docs");
|
||||||
|
@ -472,10 +477,13 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
|
||||||
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1")
|
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1")
|
||||||
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1")
|
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1")
|
||||||
.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)
|
.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0)
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.getKey(), "0s")
|
||||||
).get();
|
).get();
|
||||||
client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get();
|
assertBusy(() -> assertThat(client().admin().indices().prepareStats("test").get().getShards()[0]
|
||||||
|
.getRetentionLeaseStats().retentionLeases().leases().size(), equalTo(1)));
|
||||||
|
client().admin().indices().prepareFlush("test").setForce(true).get();
|
||||||
if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy
|
if (softDeleteEnabled) { // We need an extra flush to advance the min_retained_seqno of the SoftDeletesPolicy
|
||||||
client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get();
|
client().admin().indices().prepareFlush("test").setForce(true).get();
|
||||||
}
|
}
|
||||||
return super.onNodeStopped(nodeName);
|
return super.onNodeStopped(nodeName);
|
||||||
}
|
}
|
||||||
|
|
|
@ -73,6 +73,7 @@ import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.FixedBitSet;
|
import org.apache.lucene.util.FixedBitSet;
|
||||||
import org.elasticsearch.ElasticsearchException;
|
import org.elasticsearch.ElasticsearchException;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.support.TransportActions;
|
import org.elasticsearch.action.support.TransportActions;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
@ -2357,7 +2358,7 @@ public class InternalEngineTests extends EngineTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testSeqNoAndCheckpoints() throws IOException {
|
public void testSeqNoAndCheckpoints() throws IOException, InterruptedException {
|
||||||
final int opCount = randomIntBetween(1, 256);
|
final int opCount = randomIntBetween(1, 256);
|
||||||
long primarySeqNo = SequenceNumbers.NO_OPS_PERFORMED;
|
long primarySeqNo = SequenceNumbers.NO_OPS_PERFORMED;
|
||||||
final String[] ids = new String[]{"1", "2", "3"};
|
final String[] ids = new String[]{"1", "2", "3"};
|
||||||
|
@ -2375,13 +2376,27 @@ public class InternalEngineTests extends EngineTestCase {
|
||||||
final ShardRouting primary = TestShardRouting.newShardRouting("test",
|
final ShardRouting primary = TestShardRouting.newShardRouting("test",
|
||||||
shardId.id(), "node1", null, true,
|
shardId.id(), "node1", null, true,
|
||||||
ShardRoutingState.STARTED, allocationId);
|
ShardRoutingState.STARTED, allocationId);
|
||||||
final ShardRouting replica =
|
final ShardRouting initializingReplica =
|
||||||
TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED);
|
TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.INITIALIZING);
|
||||||
|
|
||||||
ReplicationTracker gcpTracker = (ReplicationTracker) initialEngine.config().getGlobalCheckpointSupplier();
|
ReplicationTracker gcpTracker = (ReplicationTracker) initialEngine.config().getGlobalCheckpointSupplier();
|
||||||
gcpTracker.updateFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(),
|
gcpTracker.updateFromMaster(1L, new HashSet<>(Collections.singletonList(primary.allocationId().getId())),
|
||||||
replica.allocationId().getId())),
|
new IndexShardRoutingTable.Builder(shardId).addShard(primary).build());
|
||||||
new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build());
|
|
||||||
gcpTracker.activatePrimaryMode(primarySeqNo);
|
gcpTracker.activatePrimaryMode(primarySeqNo);
|
||||||
|
if (defaultSettings.isSoftDeleteEnabled()) {
|
||||||
|
final CountDownLatch countDownLatch = new CountDownLatch(1);
|
||||||
|
gcpTracker.addPeerRecoveryRetentionLease(initializingReplica.currentNodeId(),
|
||||||
|
SequenceNumbers.NO_OPS_PERFORMED, ActionListener.wrap(countDownLatch::countDown));
|
||||||
|
countDownLatch.await();
|
||||||
|
}
|
||||||
|
gcpTracker.updateFromMaster(2L, new HashSet<>(Collections.singletonList(primary.allocationId().getId())),
|
||||||
|
new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(initializingReplica).build());
|
||||||
|
gcpTracker.initiateTracking(initializingReplica.allocationId().getId());
|
||||||
|
gcpTracker.markAllocationIdAsInSync(initializingReplica.allocationId().getId(), replicaLocalCheckpoint);
|
||||||
|
final ShardRouting replica = initializingReplica.moveToStarted();
|
||||||
|
gcpTracker.updateFromMaster(3L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), replica.allocationId().getId())),
|
||||||
|
new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build());
|
||||||
|
|
||||||
for (int op = 0; op < opCount; op++) {
|
for (int op = 0; op < opCount; op++) {
|
||||||
final String id;
|
final String id;
|
||||||
// mostly index, sometimes delete
|
// mostly index, sometimes delete
|
||||||
|
|
|
@ -40,7 +40,8 @@ public class NoOpEngineRecoveryTests extends IndexShardTestCase {
|
||||||
indexShard.close("test", true);
|
indexShard.close("test", true);
|
||||||
|
|
||||||
final ShardRouting shardRouting = indexShard.routingEntry();
|
final ShardRouting shardRouting = indexShard.routingEntry();
|
||||||
IndexShard primary = reinitShard(indexShard, initWithSameId(shardRouting, ExistingStoreRecoverySource.INSTANCE), NoOpEngine::new);
|
IndexShard primary = reinitShard(indexShard, initWithSameId(shardRouting, ExistingStoreRecoverySource.INSTANCE),
|
||||||
|
indexShard.indexSettings().getIndexMetaData(), NoOpEngine::new);
|
||||||
recoverShardFromStore(primary);
|
recoverShardFromStore(primary);
|
||||||
assertEquals(primary.seqNoStats().getMaxSeqNo(), primary.getMaxSeqNoOfUpdatesOrDeletes());
|
assertEquals(primary.seqNoStats().getMaxSeqNo(), primary.getMaxSeqNoOfUpdatesOrDeletes());
|
||||||
assertEquals(nbDocs, primary.docStats().getCount());
|
assertEquals(nbDocs, primary.docStats().getCount());
|
||||||
|
|
|
@ -49,6 +49,7 @@ import org.elasticsearch.index.engine.EngineTestCase;
|
||||||
import org.elasticsearch.index.engine.InternalEngineFactory;
|
import org.elasticsearch.index.engine.InternalEngineFactory;
|
||||||
import org.elasticsearch.index.engine.InternalEngineTests;
|
import org.elasticsearch.index.engine.InternalEngineTests;
|
||||||
import org.elasticsearch.index.mapper.SourceToParse;
|
import org.elasticsearch.index.mapper.SourceToParse;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
|
@ -66,6 +67,7 @@ import java.util.Collections;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.Future;
|
import java.util.concurrent.Future;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -78,6 +80,7 @@ import static org.hamcrest.Matchers.anyOf;
|
||||||
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.everyItem;
|
import static org.hamcrest.Matchers.everyItem;
|
||||||
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
import static org.hamcrest.Matchers.is;
|
import static org.hamcrest.Matchers.is;
|
||||||
import static org.hamcrest.Matchers.isIn;
|
import static org.hamcrest.Matchers.isIn;
|
||||||
|
@ -114,31 +117,26 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
int docs = shards.indexDocs(randomInt(50));
|
int docs = shards.indexDocs(randomInt(50));
|
||||||
shards.flush();
|
shards.flush();
|
||||||
final IndexShard originalReplica = shards.getReplicas().get(0);
|
final IndexShard originalReplica = shards.getReplicas().get(0);
|
||||||
long replicaCommittedLocalCheckpoint = docs - 1;
|
|
||||||
boolean replicaHasDocsSinceLastFlushedCheckpoint = false;
|
|
||||||
for (int i = 0; i < randomInt(2); i++) {
|
for (int i = 0; i < randomInt(2); i++) {
|
||||||
final int indexedDocs = shards.indexDocs(randomInt(5));
|
final int indexedDocs = shards.indexDocs(randomInt(5));
|
||||||
docs += indexedDocs;
|
docs += indexedDocs;
|
||||||
if (indexedDocs > 0) {
|
|
||||||
replicaHasDocsSinceLastFlushedCheckpoint = true;
|
|
||||||
}
|
|
||||||
|
|
||||||
final boolean flush = randomBoolean();
|
final boolean flush = randomBoolean();
|
||||||
if (flush) {
|
if (flush) {
|
||||||
originalReplica.flush(new FlushRequest());
|
originalReplica.flush(new FlushRequest());
|
||||||
replicaHasDocsSinceLastFlushedCheckpoint = false;
|
|
||||||
replicaCommittedLocalCheckpoint = docs - 1;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
// simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
|
||||||
shards.syncGlobalCheckpoint();
|
shards.syncGlobalCheckpoint();
|
||||||
|
long globalCheckpointOnReplica = originalReplica.getLastSyncedGlobalCheckpoint();
|
||||||
|
Optional<SequenceNumbers.CommitInfo> safeCommitOnReplica =
|
||||||
|
originalReplica.store().findSafeIndexCommit(globalCheckpointOnReplica);
|
||||||
|
assertTrue(safeCommitOnReplica.isPresent());
|
||||||
shards.removeReplica(originalReplica);
|
shards.removeReplica(originalReplica);
|
||||||
|
|
||||||
final int missingOnReplica = shards.indexDocs(randomInt(5));
|
final int missingOnReplica = shards.indexDocs(randomInt(5));
|
||||||
docs += missingOnReplica;
|
docs += missingOnReplica;
|
||||||
replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0;
|
|
||||||
|
|
||||||
final boolean translogTrimmed;
|
final boolean translogTrimmed;
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
|
@ -157,14 +155,15 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
final IndexShard recoveredReplica =
|
final IndexShard recoveredReplica =
|
||||||
shards.addReplicaWithExistingPath(originalReplica.shardPath(), originalReplica.routingEntry().currentNodeId());
|
shards.addReplicaWithExistingPath(originalReplica.shardPath(), originalReplica.routingEntry().currentNodeId());
|
||||||
shards.recoverReplica(recoveredReplica);
|
shards.recoverReplica(recoveredReplica);
|
||||||
if (translogTrimmed && replicaHasDocsSinceLastFlushedCheckpoint) {
|
if (translogTrimmed && missingOnReplica > 0) {
|
||||||
// replica has something to catch up with, but since we trimmed the primary translog, we should fall back to full recovery
|
// replica has something to catch up with, but since we trimmed the primary translog, we should fall back to full recovery
|
||||||
assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
||||||
} else {
|
} else {
|
||||||
assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty());
|
assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty());
|
||||||
assertThat(
|
assertThat(recoveredReplica.recoveryState().getTranslog().recoveredOperations(),
|
||||||
recoveredReplica.recoveryState().getTranslog().recoveredOperations(),
|
equalTo(Math.toIntExact(docs - 1 - safeCommitOnReplica.get().localCheckpoint)));
|
||||||
equalTo(Math.toIntExact(docs - (replicaCommittedLocalCheckpoint + 1))));
|
assertThat(recoveredReplica.recoveryState().getTranslog().totalLocal(),
|
||||||
|
equalTo(Math.toIntExact(globalCheckpointOnReplica - safeCommitOnReplica.get().localCheckpoint)));
|
||||||
}
|
}
|
||||||
|
|
||||||
docs += shards.indexDocs(randomInt(5));
|
docs += shards.indexDocs(randomInt(5));
|
||||||
|
@ -231,10 +230,9 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
try (ReplicationGroup shards = createGroup(2)) {
|
try (ReplicationGroup shards = createGroup(2)) {
|
||||||
shards.startAll();
|
shards.startAll();
|
||||||
int totalDocs = shards.indexDocs(randomInt(10));
|
int totalDocs = shards.indexDocs(randomInt(10));
|
||||||
int committedDocs = 0;
|
shards.syncGlobalCheckpoint();
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
shards.flush();
|
shards.flush();
|
||||||
committedDocs = totalDocs;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final IndexShard oldPrimary = shards.getPrimary();
|
final IndexShard oldPrimary = shards.getPrimary();
|
||||||
|
@ -254,7 +252,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
oldPrimary.flush(new FlushRequest(index.getName()));
|
oldPrimary.flush(new FlushRequest(index.getName()));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
long globalCheckpointOnOldPrimary = oldPrimary.getLastSyncedGlobalCheckpoint();
|
||||||
|
Optional<SequenceNumbers.CommitInfo> safeCommitOnOldPrimary =
|
||||||
|
oldPrimary.store().findSafeIndexCommit(globalCheckpointOnOldPrimary);
|
||||||
|
assertTrue(safeCommitOnOldPrimary.isPresent());
|
||||||
shards.promoteReplicaToPrimary(newPrimary).get();
|
shards.promoteReplicaToPrimary(newPrimary).get();
|
||||||
|
|
||||||
// check that local checkpoint of new primary is properly tracked after primary promotion
|
// check that local checkpoint of new primary is properly tracked after primary promotion
|
||||||
|
@ -291,6 +292,15 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
// We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen.
|
// We need an extra flush to advance the min_retained_seqno on the new primary so ops-based won't happen.
|
||||||
// The min_retained_seqno only advances when a merge asks for the retention query.
|
// The min_retained_seqno only advances when a merge asks for the retention query.
|
||||||
newPrimary.flush(new FlushRequest().force(true));
|
newPrimary.flush(new FlushRequest().force(true));
|
||||||
|
|
||||||
|
// We also need to make sure that there is no retention lease holding on to any history. The lease for the old primary
|
||||||
|
// expires since there are no unassigned shards in this replication group).
|
||||||
|
assertBusy(() -> {
|
||||||
|
newPrimary.syncRetentionLeases();
|
||||||
|
//noinspection OptionalGetWithoutIsPresent since there must be at least one lease
|
||||||
|
assertThat(newPrimary.getRetentionLeases().leases().stream().mapToLong(RetentionLease::retainingSequenceNumber)
|
||||||
|
.min().getAsLong(), greaterThan(newPrimary.seqNoStats().getMaxSeqNo()));
|
||||||
|
});
|
||||||
}
|
}
|
||||||
uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10));
|
uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10));
|
||||||
totalDocs += uncommittedOpsOnPrimary;
|
totalDocs += uncommittedOpsOnPrimary;
|
||||||
|
@ -310,7 +320,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
||||||
|
|
||||||
if (expectSeqNoRecovery) {
|
if (expectSeqNoRecovery) {
|
||||||
assertThat(newReplica.recoveryState().getIndex().fileDetails(), empty());
|
assertThat(newReplica.recoveryState().getIndex().fileDetails(), empty());
|
||||||
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs - committedDocs));
|
assertThat(newReplica.recoveryState().getTranslog().totalLocal(),
|
||||||
|
equalTo(Math.toIntExact(globalCheckpointOnOldPrimary - safeCommitOnOldPrimary.get().localCheckpoint)));
|
||||||
|
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(),
|
||||||
|
equalTo(Math.toIntExact(totalDocs - 1 - safeCommitOnOldPrimary.get().localCheckpoint)));
|
||||||
} else {
|
} else {
|
||||||
assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
||||||
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedOpsOnPrimary));
|
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedOpsOnPrimary));
|
||||||
|
|
|
@ -28,6 +28,7 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.seqno.RetentionLease;
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseSyncAction;
|
import org.elasticsearch.index.seqno.RetentionLeaseSyncAction;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLeaseUtils;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
@ -61,9 +62,10 @@ public class RetentionLeasesReplicationTests extends ESIndexLevelReplicationTest
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases();
|
RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases();
|
||||||
assertThat(leasesOnPrimary.version(), equalTo((long) iterations));
|
assertThat(leasesOnPrimary.version(), equalTo(iterations + group.getReplicas().size() + 1L));
|
||||||
assertThat(leasesOnPrimary.primaryTerm(), equalTo(group.getPrimary().getOperationPrimaryTerm()));
|
assertThat(leasesOnPrimary.primaryTerm(), equalTo(group.getPrimary().getOperationPrimaryTerm()));
|
||||||
assertThat(leasesOnPrimary.leases(), containsInAnyOrder(leases.toArray(new RetentionLease[0])));
|
assertThat(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(leasesOnPrimary).values(),
|
||||||
|
containsInAnyOrder(leases.toArray(new RetentionLease[0])));
|
||||||
latch.await();
|
latch.await();
|
||||||
for (IndexShard replica : group.getReplicas()) {
|
for (IndexShard replica : group.getReplicas()) {
|
||||||
assertThat(replica.getRetentionLeases(), equalTo(leasesOnPrimary));
|
assertThat(replica.getRetentionLeases(), equalTo(leasesOnPrimary));
|
||||||
|
@ -109,9 +111,12 @@ public class RetentionLeasesReplicationTests extends ESIndexLevelReplicationTest
|
||||||
}
|
}
|
||||||
}) {
|
}) {
|
||||||
group.startAll();
|
group.startAll();
|
||||||
|
for (IndexShard replica : group.getReplicas()) {
|
||||||
|
replica.updateRetentionLeasesOnReplica(group.getPrimary().getRetentionLeases());
|
||||||
|
}
|
||||||
int numLeases = between(1, 100);
|
int numLeases = between(1, 100);
|
||||||
IndexShard newPrimary = randomFrom(group.getReplicas());
|
IndexShard newPrimary = randomFrom(group.getReplicas());
|
||||||
RetentionLeases latestRetentionLeasesOnNewPrimary = RetentionLeases.EMPTY;
|
RetentionLeases latestRetentionLeasesOnNewPrimary = newPrimary.getRetentionLeases();
|
||||||
for (int i = 0; i < numLeases; i++) {
|
for (int i = 0; i < numLeases; i++) {
|
||||||
PlainActionFuture<ReplicationResponse> addLeaseFuture = new PlainActionFuture<>();
|
PlainActionFuture<ReplicationResponse> addLeaseFuture = new PlainActionFuture<>();
|
||||||
group.addRetentionLease(Integer.toString(i), randomNonNegativeLong(), "test-" + i, addLeaseFuture);
|
group.addRetentionLease(Integer.toString(i), randomNonNegativeLong(), "test-" + i, addLeaseFuture);
|
||||||
|
@ -130,7 +135,7 @@ public class RetentionLeasesReplicationTests extends ESIndexLevelReplicationTest
|
||||||
group.addRetentionLease("new-lease-after-promotion", randomNonNegativeLong(), "test", newLeaseFuture);
|
group.addRetentionLease("new-lease-after-promotion", randomNonNegativeLong(), "test", newLeaseFuture);
|
||||||
RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases();
|
RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases();
|
||||||
assertThat(leasesOnPrimary.primaryTerm(), equalTo(group.getPrimary().getOperationPrimaryTerm()));
|
assertThat(leasesOnPrimary.primaryTerm(), equalTo(group.getPrimary().getOperationPrimaryTerm()));
|
||||||
assertThat(leasesOnPrimary.version(), equalTo(latestRetentionLeasesOnNewPrimary.version() + 1L));
|
assertThat(leasesOnPrimary.version(), equalTo(latestRetentionLeasesOnNewPrimary.version() + 1));
|
||||||
assertThat(leasesOnPrimary.leases(), hasSize(latestRetentionLeasesOnNewPrimary.leases().size() + 1));
|
assertThat(leasesOnPrimary.leases(), hasSize(latestRetentionLeasesOnNewPrimary.leases().size() + 1));
|
||||||
RetentionLeaseSyncAction.Request request = ((SyncRetentionLeasesResponse) newLeaseFuture.actionGet()).syncRequest;
|
RetentionLeaseSyncAction.Request request = ((SyncRetentionLeasesResponse) newLeaseFuture.actionGet()).syncRequest;
|
||||||
for (IndexShard replica : group.getReplicas()) {
|
for (IndexShard replica : group.getReplicas()) {
|
||||||
|
|
|
@ -0,0 +1,87 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
|
import org.elasticsearch.Version;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
|
import org.elasticsearch.common.UUIDs;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.env.NodeEnvironment;
|
||||||
|
import org.elasticsearch.env.NodeMetaData;
|
||||||
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
import org.elasticsearch.test.InternalTestCluster;
|
||||||
|
import org.elasticsearch.test.VersionUtils;
|
||||||
|
|
||||||
|
import java.nio.file.Path;
|
||||||
|
|
||||||
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
@ESIntegTestCase.ClusterScope(numDataNodes = 0)
|
||||||
|
public class PeerRecoveryRetentionLeaseCreationIT extends ESIntegTestCase {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected boolean forbidPrivateIndexSettings() {
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCanRecoverFromStoreWithoutPeerRecoveryRetentionLease() throws Exception {
|
||||||
|
/*
|
||||||
|
* In a full cluster restart from a version without peer-recovery retention leases, the leases on disk will not include a lease for
|
||||||
|
* the local node. The same sort of thing can happen in weird situations involving dangling indices. This test ensures that a
|
||||||
|
* primary that is recovering from store creates a lease for itself.
|
||||||
|
*/
|
||||||
|
|
||||||
|
internalCluster().startMasterOnlyNode();
|
||||||
|
final String dataNode = internalCluster().startDataOnlyNode();
|
||||||
|
final Path[] nodeDataPaths = internalCluster().getInstance(NodeEnvironment.class, dataNode).nodeDataPaths();
|
||||||
|
|
||||||
|
assertAcked(prepareCreate("index").setSettings(Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)
|
||||||
|
.put(IndexMetaData.SETTING_VERSION_CREATED,
|
||||||
|
// simulate a version which supports soft deletes (v6.5.0-and-later) with which this node is compatible
|
||||||
|
VersionUtils.randomVersionBetween(random(),
|
||||||
|
Version.max(Version.CURRENT.minimumIndexCompatibilityVersion(), Version.V_6_5_0), Version.CURRENT))));
|
||||||
|
ensureGreen("index");
|
||||||
|
|
||||||
|
// Change the node ID so that the persisted retention lease no longer applies.
|
||||||
|
final String oldNodeId = client().admin().cluster().prepareNodesInfo(dataNode).clear().get().getNodes().get(0).getNode().getId();
|
||||||
|
final String newNodeId = randomValueOtherThan(oldNodeId, () -> UUIDs.randomBase64UUID(random()));
|
||||||
|
|
||||||
|
internalCluster().restartNode(dataNode, new InternalTestCluster.RestartCallback() {
|
||||||
|
@Override
|
||||||
|
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||||
|
final NodeMetaData nodeMetaData = new NodeMetaData(newNodeId, Version.CURRENT);
|
||||||
|
NodeMetaData.FORMAT.writeAndCleanup(nodeMetaData, nodeDataPaths);
|
||||||
|
return Settings.EMPTY;
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
ensureGreen("index");
|
||||||
|
assertThat(client().admin().cluster().prepareNodesInfo(dataNode).clear().get().getNodes().get(0).getNode().getId(),
|
||||||
|
equalTo(newNodeId));
|
||||||
|
final RetentionLeases retentionLeases = client().admin().indices().prepareStats("index").get().getShards()[0]
|
||||||
|
.getRetentionLeaseStats().retentionLeases();
|
||||||
|
assertTrue("expected lease for [" + newNodeId + "] in " + retentionLeases,
|
||||||
|
retentionLeases.contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(newNodeId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -0,0 +1,179 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
|
import org.elasticsearch.cluster.routing.AllocationId;
|
||||||
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.common.collect.Tuple;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.test.IndexSettingsModule;
|
||||||
|
import org.junit.Before;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
|
|
||||||
|
public class PeerRecoveryRetentionLeaseExpiryTests extends ReplicationTrackerTestCase {
|
||||||
|
|
||||||
|
private static final ActionListener<ReplicationResponse> EMPTY_LISTENER = ActionListener.wrap(() -> { });
|
||||||
|
|
||||||
|
private ReplicationTracker replicationTracker;
|
||||||
|
private AtomicLong currentTimeMillis;
|
||||||
|
private Settings settings;
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUpReplicationTracker() throws InterruptedException {
|
||||||
|
final AllocationId primaryAllocationId = AllocationId.newInitializing();
|
||||||
|
currentTimeMillis = new AtomicLong(randomLongBetween(0, 1024));
|
||||||
|
|
||||||
|
if (randomBoolean()) {
|
||||||
|
settings = Settings.builder()
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.getKey(),
|
||||||
|
TimeValue.timeValueMillis(randomLongBetween(1, TimeValue.timeValueHours(12).millis())))
|
||||||
|
.build();
|
||||||
|
} else {
|
||||||
|
settings = Settings.EMPTY;
|
||||||
|
}
|
||||||
|
|
||||||
|
final long primaryTerm = randomLongBetween(1, Long.MAX_VALUE);
|
||||||
|
replicationTracker = new ReplicationTracker(
|
||||||
|
new ShardId("test", "_na", 0),
|
||||||
|
primaryAllocationId.getId(),
|
||||||
|
IndexSettingsModule.newIndexSettings("test", settings),
|
||||||
|
primaryTerm,
|
||||||
|
UNASSIGNED_SEQ_NO,
|
||||||
|
value -> { },
|
||||||
|
currentTimeMillis::get,
|
||||||
|
(leases, listener) -> { });
|
||||||
|
replicationTracker.updateFromMaster(1L, Collections.singleton(primaryAllocationId.getId()),
|
||||||
|
routingTable(Collections.emptySet(), primaryAllocationId));
|
||||||
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
|
||||||
|
final AllocationId replicaAllocationId = AllocationId.newInitializing();
|
||||||
|
final IndexShardRoutingTable routingTableWithReplica
|
||||||
|
= routingTable(Collections.singleton(replicaAllocationId), primaryAllocationId);
|
||||||
|
replicationTracker.updateFromMaster(2L, Collections.singleton(primaryAllocationId.getId()), routingTableWithReplica);
|
||||||
|
replicationTracker.addPeerRecoveryRetentionLease(
|
||||||
|
routingTableWithReplica.getByAllocationId(replicaAllocationId.getId()).currentNodeId(), randomCheckpoint(),
|
||||||
|
EMPTY_LISTENER);
|
||||||
|
|
||||||
|
replicationTracker.initiateTracking(replicaAllocationId.getId());
|
||||||
|
replicationTracker.markAllocationIdAsInSync(replicaAllocationId.getId(), randomCheckpoint());
|
||||||
|
}
|
||||||
|
|
||||||
|
private long randomCheckpoint() {
|
||||||
|
return randomBoolean() ? SequenceNumbers.NO_OPS_PERFORMED : randomNonNegativeLong();
|
||||||
|
}
|
||||||
|
|
||||||
|
private void startReplica() {
|
||||||
|
final ShardRouting replicaShardRouting = replicationTracker.routingTable.replicaShards().get(0);
|
||||||
|
final IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(replicationTracker.routingTable);
|
||||||
|
builder.removeShard(replicaShardRouting);
|
||||||
|
builder.addShard(replicaShardRouting.moveToStarted());
|
||||||
|
replicationTracker.updateFromMaster(replicationTracker.appliedClusterStateVersion + 1,
|
||||||
|
replicationTracker.routingTable.shards().stream().map(sr -> sr.allocationId().getId()).collect(Collectors.toSet()),
|
||||||
|
builder.build());
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeasesForAssignedCopiesDoNotEverExpire() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
startReplica();
|
||||||
|
}
|
||||||
|
|
||||||
|
currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(0, Long.MAX_VALUE - currentTimeMillis.get()));
|
||||||
|
|
||||||
|
final Tuple<Boolean, RetentionLeases> retentionLeases = replicationTracker.getRetentionLeases(true);
|
||||||
|
assertFalse(retentionLeases.v1());
|
||||||
|
|
||||||
|
final Set<String> leaseIds = retentionLeases.v2().leases().stream().map(RetentionLease::id).collect(Collectors.toSet());
|
||||||
|
assertThat(leaseIds, hasSize(2));
|
||||||
|
assertThat(leaseIds, equalTo(replicationTracker.routingTable.shards().stream()
|
||||||
|
.map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet())));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeasesForUnassignedCopiesDoNotExpireImmediatelyIfShardsNotAllStarted() {
|
||||||
|
final String unknownNodeId = randomAlphaOfLength(10);
|
||||||
|
replicationTracker.addPeerRecoveryRetentionLease(unknownNodeId, randomCheckpoint(), EMPTY_LISTENER);
|
||||||
|
|
||||||
|
currentTimeMillis.set(currentTimeMillis.get()
|
||||||
|
+ randomLongBetween(0, IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.get(settings).millis()));
|
||||||
|
|
||||||
|
final Tuple<Boolean, RetentionLeases> retentionLeases = replicationTracker.getRetentionLeases(true);
|
||||||
|
assertFalse("should not have expired anything", retentionLeases.v1());
|
||||||
|
|
||||||
|
final Set<String> leaseIds = retentionLeases.v2().leases().stream().map(RetentionLease::id).collect(Collectors.toSet());
|
||||||
|
assertThat(leaseIds, hasSize(3));
|
||||||
|
assertThat(leaseIds, equalTo(Stream.concat(Stream.of(ReplicationTracker.getPeerRecoveryRetentionLeaseId(unknownNodeId)),
|
||||||
|
replicationTracker.routingTable.shards().stream()
|
||||||
|
.map(ReplicationTracker::getPeerRecoveryRetentionLeaseId)).collect(Collectors.toSet())));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeasesForUnassignedCopiesExpireEventually() {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
startReplica();
|
||||||
|
}
|
||||||
|
|
||||||
|
final String unknownNodeId = randomAlphaOfLength(10);
|
||||||
|
replicationTracker.addPeerRecoveryRetentionLease(unknownNodeId, randomCheckpoint(), EMPTY_LISTENER);
|
||||||
|
|
||||||
|
currentTimeMillis.set(randomLongBetween(
|
||||||
|
currentTimeMillis.get() + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.get(settings).millis() + 1,
|
||||||
|
Long.MAX_VALUE));
|
||||||
|
|
||||||
|
final Tuple<Boolean, RetentionLeases> retentionLeases = replicationTracker.getRetentionLeases(true);
|
||||||
|
assertTrue("should have expired something", retentionLeases.v1());
|
||||||
|
|
||||||
|
final Set<String> leaseIds = retentionLeases.v2().leases().stream().map(RetentionLease::id).collect(Collectors.toSet());
|
||||||
|
assertThat(leaseIds, hasSize(2));
|
||||||
|
assertThat(leaseIds, equalTo(replicationTracker.routingTable.shards().stream()
|
||||||
|
.map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet())));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeasesForUnassignedCopiesExpireImmediatelyIfShardsAllStarted() {
|
||||||
|
final String unknownNodeId = randomAlphaOfLength(10);
|
||||||
|
replicationTracker.addPeerRecoveryRetentionLease(unknownNodeId, randomCheckpoint(), EMPTY_LISTENER);
|
||||||
|
|
||||||
|
startReplica();
|
||||||
|
|
||||||
|
currentTimeMillis.set(currentTimeMillis.get() +
|
||||||
|
(usually()
|
||||||
|
? randomLongBetween(0, IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.get(settings).millis())
|
||||||
|
: randomLongBetween(0, Long.MAX_VALUE - currentTimeMillis.get())));
|
||||||
|
|
||||||
|
final Tuple<Boolean, RetentionLeases> retentionLeases = replicationTracker.getRetentionLeases(true);
|
||||||
|
assertTrue(retentionLeases.v1());
|
||||||
|
|
||||||
|
final Set<String> leaseIds = retentionLeases.v2().leases().stream().map(RetentionLease::id).collect(Collectors.toSet());
|
||||||
|
assertThat(leaseIds, hasSize(2));
|
||||||
|
assertThat(leaseIds, equalTo(replicationTracker.routingTable.shards().stream()
|
||||||
|
.map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet())));
|
||||||
|
}
|
||||||
|
}
|
|
@ -20,6 +20,8 @@
|
||||||
package org.elasticsearch.index.seqno;
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
import org.elasticsearch.action.ActionListener;
|
import org.elasticsearch.action.ActionListener;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
import org.elasticsearch.cluster.routing.AllocationId;
|
import org.elasticsearch.cluster.routing.AllocationId;
|
||||||
import org.elasticsearch.common.collect.Tuple;
|
import org.elasticsearch.common.collect.Tuple;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
@ -43,6 +45,7 @@ import java.util.concurrent.CyclicBarrier;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
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.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
|
@ -83,7 +86,7 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE);
|
minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE);
|
||||||
replicationTracker.addRetentionLease(
|
replicationTracker.addRetentionLease(
|
||||||
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
||||||
assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 1 + i, true, false);
|
assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 2 + i, true, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int i = 0; i < length; i++) {
|
for (int i = 0; i < length; i++) {
|
||||||
|
@ -93,7 +96,7 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
}
|
}
|
||||||
minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE);
|
minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE);
|
||||||
replicationTracker.renewRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i);
|
replicationTracker.renewRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i);
|
||||||
assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers, primaryTerm, 1 + length + i, true, false);
|
assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers, primaryTerm, 2 + length + i, true, false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -178,6 +181,7 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
Collections.singleton(allocationId.getId()),
|
Collections.singleton(allocationId.getId()),
|
||||||
routingTable(Collections.emptySet(), allocationId));
|
routingTable(Collections.emptySet(), allocationId));
|
||||||
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
retainingSequenceNumbers.put(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L);
|
||||||
|
|
||||||
final int length = randomIntBetween(0, 8);
|
final int length = randomIntBetween(0, 8);
|
||||||
for (int i = 0; i < length; i++) {
|
for (int i = 0; i < length; i++) {
|
||||||
|
@ -239,12 +243,109 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
length - i - 1,
|
length - i - 1,
|
||||||
minimumRetainingSequenceNumbers,
|
minimumRetainingSequenceNumbers,
|
||||||
primaryTerm,
|
primaryTerm,
|
||||||
1 + length + i,
|
2 + length + i,
|
||||||
true,
|
true,
|
||||||
false);
|
false);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testCloneRetentionLease() {
|
||||||
|
final AllocationId allocationId = AllocationId.newInitializing();
|
||||||
|
final AtomicReference<ReplicationTracker> replicationTrackerRef = new AtomicReference<>();
|
||||||
|
final AtomicLong timeReference = new AtomicLong();
|
||||||
|
final AtomicBoolean synced = new AtomicBoolean();
|
||||||
|
final ReplicationTracker replicationTracker = new ReplicationTracker(
|
||||||
|
new ShardId("test", "_na", 0),
|
||||||
|
allocationId.getId(),
|
||||||
|
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY),
|
||||||
|
randomLongBetween(1, Long.MAX_VALUE),
|
||||||
|
UNASSIGNED_SEQ_NO,
|
||||||
|
value -> {},
|
||||||
|
timeReference::get,
|
||||||
|
(leases, listener) -> {
|
||||||
|
assertFalse(Thread.holdsLock(replicationTrackerRef.get()));
|
||||||
|
assertTrue(synced.compareAndSet(false, true));
|
||||||
|
listener.onResponse(new ReplicationResponse());
|
||||||
|
});
|
||||||
|
replicationTrackerRef.set(replicationTracker);
|
||||||
|
replicationTracker.updateFromMaster(
|
||||||
|
randomNonNegativeLong(),
|
||||||
|
Collections.singleton(allocationId.getId()),
|
||||||
|
routingTable(Collections.emptySet(), allocationId));
|
||||||
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
|
||||||
|
final long addTime = randomLongBetween(timeReference.get(), Long.MAX_VALUE);
|
||||||
|
timeReference.set(addTime);
|
||||||
|
final long minimumRetainingSequenceNumber = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE);
|
||||||
|
final PlainActionFuture<ReplicationResponse> addFuture = new PlainActionFuture<>();
|
||||||
|
replicationTracker.addRetentionLease("source", minimumRetainingSequenceNumber, "test-source", addFuture);
|
||||||
|
addFuture.actionGet();
|
||||||
|
assertTrue(synced.get());
|
||||||
|
synced.set(false);
|
||||||
|
|
||||||
|
final long cloneTime = randomLongBetween(timeReference.get(), Long.MAX_VALUE);
|
||||||
|
timeReference.set(cloneTime);
|
||||||
|
final PlainActionFuture<ReplicationResponse> cloneFuture = new PlainActionFuture<>();
|
||||||
|
final RetentionLease clonedLease = replicationTracker.cloneRetentionLease("source", "target", cloneFuture);
|
||||||
|
cloneFuture.actionGet();
|
||||||
|
assertTrue(synced.get());
|
||||||
|
synced.set(false);
|
||||||
|
|
||||||
|
assertThat(clonedLease.id(), equalTo("target"));
|
||||||
|
assertThat(clonedLease.retainingSequenceNumber(), equalTo(minimumRetainingSequenceNumber));
|
||||||
|
assertThat(clonedLease.timestamp(), equalTo(cloneTime));
|
||||||
|
assertThat(clonedLease.source(), equalTo("test-source"));
|
||||||
|
|
||||||
|
assertThat(replicationTracker.getRetentionLeases().get("target"), equalTo(clonedLease));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCloneNonexistentRetentionLease() {
|
||||||
|
final AllocationId allocationId = AllocationId.newInitializing();
|
||||||
|
final ReplicationTracker replicationTracker = new ReplicationTracker(
|
||||||
|
new ShardId("test", "_na", 0),
|
||||||
|
allocationId.getId(),
|
||||||
|
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY),
|
||||||
|
randomLongBetween(1, Long.MAX_VALUE),
|
||||||
|
UNASSIGNED_SEQ_NO,
|
||||||
|
value -> {},
|
||||||
|
() -> 0L,
|
||||||
|
(leases, listener) -> { });
|
||||||
|
replicationTracker.updateFromMaster(
|
||||||
|
randomNonNegativeLong(),
|
||||||
|
Collections.singleton(allocationId.getId()),
|
||||||
|
routingTable(Collections.emptySet(), allocationId));
|
||||||
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
|
||||||
|
assertThat(expectThrows(RetentionLeaseNotFoundException.class,
|
||||||
|
() -> replicationTracker.cloneRetentionLease("nonexistent-lease-id", "target", ActionListener.wrap(() -> {}))).getMessage(),
|
||||||
|
equalTo("retention lease with ID [nonexistent-lease-id] not found"));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testCloneDuplicateRetentionLease() {
|
||||||
|
final AllocationId allocationId = AllocationId.newInitializing();
|
||||||
|
final ReplicationTracker replicationTracker = new ReplicationTracker(
|
||||||
|
new ShardId("test", "_na", 0),
|
||||||
|
allocationId.getId(),
|
||||||
|
IndexSettingsModule.newIndexSettings("test", Settings.EMPTY),
|
||||||
|
randomLongBetween(1, Long.MAX_VALUE),
|
||||||
|
UNASSIGNED_SEQ_NO,
|
||||||
|
value -> {},
|
||||||
|
() -> 0L,
|
||||||
|
(leases, listener) -> { });
|
||||||
|
replicationTracker.updateFromMaster(
|
||||||
|
randomNonNegativeLong(),
|
||||||
|
Collections.singleton(allocationId.getId()),
|
||||||
|
routingTable(Collections.emptySet(), allocationId));
|
||||||
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
|
||||||
|
replicationTracker.addRetentionLease("source", randomLongBetween(0L, Long.MAX_VALUE), "test-source", ActionListener.wrap(() -> {}));
|
||||||
|
replicationTracker.addRetentionLease("exists", randomLongBetween(0L, Long.MAX_VALUE), "test-source", ActionListener.wrap(() -> {}));
|
||||||
|
|
||||||
|
assertThat(expectThrows(RetentionLeaseAlreadyExistsException.class,
|
||||||
|
() -> replicationTracker.cloneRetentionLease("source", "exists", ActionListener.wrap(() -> {}))).getMessage(),
|
||||||
|
equalTo("retention lease with ID [exists] already exists"));
|
||||||
|
}
|
||||||
|
|
||||||
public void testRemoveNotFound() {
|
public void testRemoveNotFound() {
|
||||||
final AllocationId allocationId = AllocationId.newInitializing();
|
final AllocationId allocationId = AllocationId.newInitializing();
|
||||||
long primaryTerm = randomLongBetween(1, Long.MAX_VALUE);
|
long primaryTerm = randomLongBetween(1, Long.MAX_VALUE);
|
||||||
|
@ -298,6 +399,7 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
Collections.singleton(allocationId.getId()),
|
Collections.singleton(allocationId.getId()),
|
||||||
routingTable(Collections.emptySet(), allocationId));
|
routingTable(Collections.emptySet(), allocationId));
|
||||||
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED);
|
||||||
|
retainingSequenceNumbers.put(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L);
|
||||||
|
|
||||||
final int length = randomIntBetween(0, 8);
|
final int length = randomIntBetween(0, 8);
|
||||||
for (int i = 0; i < length; i++) {
|
for (int i = 0; i < length; i++) {
|
||||||
|
@ -365,11 +467,12 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
|
|
||||||
{
|
{
|
||||||
final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases();
|
final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases();
|
||||||
assertThat(retentionLeases.version(), equalTo(1L));
|
final long expectedVersion = primaryMode ? 2L : 1L;
|
||||||
assertThat(retentionLeases.leases(), hasSize(1));
|
assertThat(retentionLeases.version(), equalTo(expectedVersion));
|
||||||
final RetentionLease retentionLease = retentionLeases.leases().iterator().next();
|
assertThat(retentionLeases.leases(), hasSize(primaryMode ? 2 : 1));
|
||||||
|
final RetentionLease retentionLease = retentionLeases.get("0");
|
||||||
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
||||||
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 1, primaryMode, false);
|
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, expectedVersion, primaryMode, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// renew the lease
|
// renew the lease
|
||||||
|
@ -387,18 +490,19 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
|
|
||||||
{
|
{
|
||||||
final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases();
|
final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases();
|
||||||
assertThat(retentionLeases.version(), equalTo(2L));
|
final long expectedVersion = primaryMode ? 3L : 2L;
|
||||||
assertThat(retentionLeases.leases(), hasSize(1));
|
assertThat(retentionLeases.version(), equalTo(expectedVersion));
|
||||||
final RetentionLease retentionLease = retentionLeases.leases().iterator().next();
|
assertThat(retentionLeases.leases(), hasSize(primaryMode ? 2 : 1));
|
||||||
|
final RetentionLease retentionLease = retentionLeases.get("0");
|
||||||
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
||||||
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, primaryMode, false);
|
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, expectedVersion, primaryMode, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// now force the lease to expire
|
// now force the lease to expire
|
||||||
currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get()));
|
currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get()));
|
||||||
if (primaryMode) {
|
if (primaryMode) {
|
||||||
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, true, false);
|
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 3, true, false);
|
||||||
assertRetentionLeases(replicationTracker, 0, new long[0], primaryTerm, 3, true, true);
|
assertRetentionLeases(replicationTracker, 0, new long[0], primaryTerm, 4, true, true);
|
||||||
} else {
|
} else {
|
||||||
// leases do not expire on replicas until synced from the primary
|
// leases do not expire on replicas until synced from the primary
|
||||||
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, false, false);
|
assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, false, false);
|
||||||
|
@ -625,10 +729,9 @@ public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTes
|
||||||
}
|
}
|
||||||
assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm));
|
assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm));
|
||||||
assertThat(retentionLeases.version(), equalTo(version));
|
assertThat(retentionLeases.version(), equalTo(version));
|
||||||
final Map<String, RetentionLease> idToRetentionLease = new HashMap<>();
|
final Map<String, RetentionLease> idToRetentionLease = retentionLeases.leases().stream()
|
||||||
for (final RetentionLease retentionLease : retentionLeases.leases()) {
|
.filter(retentionLease -> ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()) == false)
|
||||||
idToRetentionLease.put(retentionLease.id(), retentionLease);
|
.collect(Collectors.toMap(RetentionLease::id, Function.identity()));
|
||||||
}
|
|
||||||
|
|
||||||
assertThat(idToRetentionLease.entrySet(), hasSize(size));
|
assertThat(idToRetentionLease.entrySet(), hasSize(size));
|
||||||
for (int i = 0; i < size; i++) {
|
for (int i = 0; i < size; i++) {
|
||||||
|
|
|
@ -52,10 +52,14 @@ public abstract class ReplicationTrackerTestCase extends ESTestCase {
|
||||||
(leases, listener) -> {});
|
(leases, listener) -> {});
|
||||||
}
|
}
|
||||||
|
|
||||||
|
static String nodeIdFromAllocationId(final AllocationId allocationId) {
|
||||||
|
return "n-" + allocationId.getId().substring(0, 8);
|
||||||
|
}
|
||||||
|
|
||||||
static IndexShardRoutingTable routingTable(final Set<AllocationId> initializingIds, final AllocationId primaryId) {
|
static IndexShardRoutingTable routingTable(final Set<AllocationId> initializingIds, final AllocationId primaryId) {
|
||||||
final ShardId shardId = new ShardId("test", "_na_", 0);
|
final ShardId shardId = new ShardId("test", "_na_", 0);
|
||||||
final ShardRouting primaryShard =
|
final ShardRouting primaryShard = TestShardRouting.newShardRouting(
|
||||||
TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(10), null, true, ShardRoutingState.STARTED, primaryId);
|
shardId, nodeIdFromAllocationId(primaryId), null, true, ShardRoutingState.STARTED, primaryId);
|
||||||
return routingTable(initializingIds, primaryShard);
|
return routingTable(initializingIds, primaryShard);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -65,7 +69,7 @@ public abstract class ReplicationTrackerTestCase extends ESTestCase {
|
||||||
final IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId);
|
final IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId);
|
||||||
for (final AllocationId initializingId : initializingIds) {
|
for (final AllocationId initializingId : initializingIds) {
|
||||||
builder.addShard(TestShardRouting.newShardRouting(
|
builder.addShard(TestShardRouting.newShardRouting(
|
||||||
shardId, randomAlphaOfLength(10), null, false, ShardRoutingState.INITIALIZING, initializingId));
|
shardId, nodeIdFromAllocationId(initializingId), null, false, ShardRoutingState.INITIALIZING, initializingId));
|
||||||
}
|
}
|
||||||
|
|
||||||
builder.addShard(primaryShard);
|
builder.addShard(primaryShard);
|
||||||
|
|
|
@ -37,6 +37,7 @@ import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.test.IndexSettingsModule;
|
import org.elasticsearch.test.IndexSettingsModule;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
@ -50,6 +51,7 @@ import java.util.concurrent.CyclicBarrier;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
import java.util.concurrent.atomic.AtomicBoolean;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
import java.util.function.BiConsumer;
|
import java.util.function.BiConsumer;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.function.Function;
|
import java.util.function.Function;
|
||||||
import java.util.function.LongConsumer;
|
import java.util.function.LongConsumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -61,6 +63,9 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED;
|
||||||
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.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.greaterThan;
|
import static org.hamcrest.Matchers.greaterThan;
|
||||||
|
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||||
|
import static org.hamcrest.Matchers.hasItem;
|
||||||
|
import static org.hamcrest.Matchers.lessThanOrEqualTo;
|
||||||
import static org.hamcrest.Matchers.not;
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
|
@ -149,6 +154,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
newInitializing.add(extraId);
|
newInitializing.add(extraId);
|
||||||
tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId));
|
tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId));
|
||||||
|
|
||||||
|
addPeerRecoveryRetentionLease(tracker, extraId);
|
||||||
tracker.initiateTracking(extraId.getId());
|
tracker.initiateTracking(extraId.getId());
|
||||||
|
|
||||||
// now notify for the new id
|
// now notify for the new id
|
||||||
|
@ -190,6 +196,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId));
|
tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId));
|
||||||
final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1);
|
final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1);
|
||||||
tracker.activatePrimaryMode(localCheckpoint);
|
tracker.activatePrimaryMode(localCheckpoint);
|
||||||
|
addPeerRecoveryRetentionLease(tracker, replicaId);
|
||||||
tracker.initiateTracking(replicaId.getId());
|
tracker.initiateTracking(replicaId.getId());
|
||||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
|
@ -357,6 +364,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()),
|
tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()),
|
||||||
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId));
|
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId));
|
||||||
tracker.activatePrimaryMode(globalCheckpoint);
|
tracker.activatePrimaryMode(globalCheckpoint);
|
||||||
|
addPeerRecoveryRetentionLease(tracker, trackingAllocationId);
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
// synchronize starting with the test thread
|
// synchronize starting with the test thread
|
||||||
|
@ -421,6 +429,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()),
|
||||||
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId));
|
routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId));
|
||||||
tracker.activatePrimaryMode(globalCheckpoint);
|
tracker.activatePrimaryMode(globalCheckpoint);
|
||||||
|
addPeerRecoveryRetentionLease(tracker, trackingAllocationId);
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
// synchronize starting with the test thread
|
// synchronize starting with the test thread
|
||||||
|
@ -563,6 +572,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
initialClusterStateVersion + 3,
|
initialClusterStateVersion + 3,
|
||||||
ids(newActiveAllocationIds),
|
ids(newActiveAllocationIds),
|
||||||
routingTable(newInitializingAllocationIds, primaryId));
|
routingTable(newInitializingAllocationIds, primaryId));
|
||||||
|
addPeerRecoveryRetentionLease(tracker, newSyncingAllocationId);
|
||||||
final CyclicBarrier barrier = new CyclicBarrier(2);
|
final CyclicBarrier barrier = new CyclicBarrier(2);
|
||||||
final Thread thread = new Thread(() -> {
|
final Thread thread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
|
@ -610,7 +620,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
* allocation ID to the in-sync set and removing it from pending, the local checkpoint update that freed the thread waiting for the
|
* allocation ID to the in-sync set and removing it from pending, the local checkpoint update that freed the thread waiting for the
|
||||||
* local checkpoint to advance could miss updating the global checkpoint in a race if the waiting thread did not add the allocation
|
* local checkpoint to advance could miss updating the global checkpoint in a race if the waiting thread did not add the allocation
|
||||||
* ID to the in-sync set and remove it from the pending set before the local checkpoint updating thread executed the global checkpoint
|
* ID to the in-sync set and remove it from the pending set before the local checkpoint updating thread executed the global checkpoint
|
||||||
* update. This test fails without an additional call to {@link ReplicationTracker#updateGlobalCheckpointOnPrimary()} after
|
* update. This test fails without an additional call to {@code ReplicationTracker#updateGlobalCheckpointOnPrimary()} after
|
||||||
* removing the allocation ID from the pending set in {@link ReplicationTracker#markAllocationIdAsInSync(String, long)} (even if a
|
* removing the allocation ID from the pending set in {@link ReplicationTracker#markAllocationIdAsInSync(String, long)} (even if a
|
||||||
* call is added after notifying all waiters in {@link ReplicationTracker#updateLocalCheckpoint(String, long)}).
|
* call is added after notifying all waiters in {@link ReplicationTracker#updateLocalCheckpoint(String, long)}).
|
||||||
*
|
*
|
||||||
|
@ -630,6 +640,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
Collections.singleton(active.getId()),
|
Collections.singleton(active.getId()),
|
||||||
routingTable(Collections.singleton(initializing), active));
|
routingTable(Collections.singleton(initializing), active));
|
||||||
tracker.activatePrimaryMode(activeLocalCheckpoint);
|
tracker.activatePrimaryMode(activeLocalCheckpoint);
|
||||||
|
addPeerRecoveryRetentionLease(tracker, initializing);
|
||||||
final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE);
|
final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE);
|
||||||
final Thread activeThread = new Thread(() -> {
|
final Thread activeThread = new Thread(() -> {
|
||||||
try {
|
try {
|
||||||
|
@ -693,7 +704,9 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
clusterState.apply(oldPrimary);
|
clusterState.apply(oldPrimary);
|
||||||
clusterState.apply(newPrimary);
|
clusterState.apply(newPrimary);
|
||||||
|
|
||||||
activatePrimary(oldPrimary);
|
oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
||||||
|
addPeerRecoveryRetentionLease(oldPrimary, newPrimary.shardAllocationId);
|
||||||
|
newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases());
|
||||||
|
|
||||||
final int numUpdates = randomInt(10);
|
final int numUpdates = randomInt(10);
|
||||||
for (int i = 0; i < numUpdates; i++) {
|
for (int i = 0; i < numUpdates; i++) {
|
||||||
|
@ -706,7 +719,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
randomLocalCheckpointUpdate(oldPrimary);
|
randomLocalCheckpointUpdate(oldPrimary);
|
||||||
}
|
}
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
randomMarkInSync(oldPrimary);
|
randomMarkInSync(oldPrimary, newPrimary);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -738,7 +751,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
randomLocalCheckpointUpdate(oldPrimary);
|
randomLocalCheckpointUpdate(oldPrimary);
|
||||||
}
|
}
|
||||||
if (randomBoolean()) {
|
if (randomBoolean()) {
|
||||||
randomMarkInSync(oldPrimary);
|
randomMarkInSync(oldPrimary, newPrimary);
|
||||||
}
|
}
|
||||||
|
|
||||||
// do another handoff
|
// do another handoff
|
||||||
|
@ -876,7 +889,10 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
final ShardId shardId = new ShardId("test", "_na_", 0);
|
final ShardId shardId = new ShardId("test", "_na_", 0);
|
||||||
final ShardRouting primaryShard =
|
final ShardRouting primaryShard =
|
||||||
TestShardRouting.newShardRouting(
|
TestShardRouting.newShardRouting(
|
||||||
shardId, randomAlphaOfLength(10), randomAlphaOfLength(10), true, ShardRoutingState.RELOCATING, relocatingId);
|
shardId,
|
||||||
|
nodeIdFromAllocationId(relocatingId),
|
||||||
|
nodeIdFromAllocationId(AllocationId.newInitializing(relocatingId.getRelocationId())),
|
||||||
|
true, ShardRoutingState.RELOCATING, relocatingId);
|
||||||
|
|
||||||
return new FakeClusterState(
|
return new FakeClusterState(
|
||||||
initialClusterStateVersion,
|
initialClusterStateVersion,
|
||||||
|
@ -884,20 +900,17 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
routingTable(initializingAllocationIds, primaryShard));
|
routingTable(initializingAllocationIds, primaryShard));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void activatePrimary(ReplicationTracker gcp) {
|
|
||||||
gcp.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10));
|
|
||||||
}
|
|
||||||
|
|
||||||
private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) {
|
private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) {
|
||||||
String allocationId = randomFrom(gcp.checkpoints.keySet());
|
String allocationId = randomFrom(gcp.checkpoints.keySet());
|
||||||
long currentLocalCheckpoint = gcp.checkpoints.get(allocationId).getLocalCheckpoint();
|
long currentLocalCheckpoint = gcp.checkpoints.get(allocationId).getLocalCheckpoint();
|
||||||
gcp.updateLocalCheckpoint(allocationId, Math.max(SequenceNumbers.NO_OPS_PERFORMED, currentLocalCheckpoint + randomInt(5)));
|
gcp.updateLocalCheckpoint(allocationId, Math.max(SequenceNumbers.NO_OPS_PERFORMED, currentLocalCheckpoint + randomInt(5)));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static void randomMarkInSync(ReplicationTracker gcp) {
|
private static void randomMarkInSync(ReplicationTracker oldPrimary, ReplicationTracker newPrimary) {
|
||||||
String allocationId = randomFrom(gcp.checkpoints.keySet());
|
final String allocationId = randomFrom(oldPrimary.checkpoints.keySet());
|
||||||
long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, gcp.getGlobalCheckpoint() + randomInt(5));
|
final long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, oldPrimary.getGlobalCheckpoint() + randomInt(5));
|
||||||
markAsTrackingAndInSyncQuietly(gcp, allocationId, newLocalCheckpoint);
|
markAsTrackingAndInSyncQuietly(oldPrimary, allocationId, newLocalCheckpoint);
|
||||||
|
newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases());
|
||||||
}
|
}
|
||||||
|
|
||||||
private static FakeClusterState randomUpdateClusterState(Set<String> allocationIds, FakeClusterState clusterState) {
|
private static FakeClusterState randomUpdateClusterState(Set<String> allocationIds, FakeClusterState clusterState) {
|
||||||
|
@ -908,11 +921,14 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
final Set<AllocationId> inSyncIdsToRemove = new HashSet<>(
|
final Set<AllocationId> inSyncIdsToRemove = new HashSet<>(
|
||||||
exclude(randomSubsetOf(randomInt(clusterState.inSyncIds.size()), clusterState.inSyncIds), allocationIds));
|
exclude(randomSubsetOf(randomInt(clusterState.inSyncIds.size()), clusterState.inSyncIds), allocationIds));
|
||||||
final Set<AllocationId> remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove);
|
final Set<AllocationId> remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove);
|
||||||
|
final Set<AllocationId> initializingIdsExceptRelocationTargets = exclude(clusterState.initializingIds(),
|
||||||
|
clusterState.routingTable.activeShards().stream().filter(ShardRouting::relocating)
|
||||||
|
.map(s -> s.allocationId().getRelocationId()).collect(Collectors.toSet()));
|
||||||
return new FakeClusterState(
|
return new FakeClusterState(
|
||||||
clusterState.version + randomIntBetween(1, 5),
|
clusterState.version + randomIntBetween(1, 5),
|
||||||
remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds,
|
remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds,
|
||||||
routingTable(
|
routingTable(
|
||||||
Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove),
|
Sets.difference(Sets.union(initializingIdsExceptRelocationTargets, initializingIdsToAdd), initializingIdsToRemove),
|
||||||
clusterState.routingTable.primaryShard()));
|
clusterState.routingTable.primaryShard()));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -945,6 +961,7 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
private static void markAsTrackingAndInSyncQuietly(
|
private static void markAsTrackingAndInSyncQuietly(
|
||||||
final ReplicationTracker tracker, final String allocationId, final long localCheckpoint) {
|
final ReplicationTracker tracker, final String allocationId, final long localCheckpoint) {
|
||||||
try {
|
try {
|
||||||
|
addPeerRecoveryRetentionLease(tracker, allocationId);
|
||||||
tracker.initiateTracking(allocationId);
|
tracker.initiateTracking(allocationId);
|
||||||
tracker.markAllocationIdAsInSync(allocationId, localCheckpoint);
|
tracker.markAllocationIdAsInSync(allocationId, localCheckpoint);
|
||||||
} catch (final InterruptedException e) {
|
} catch (final InterruptedException e) {
|
||||||
|
@ -952,4 +969,169 @@ public class ReplicationTrackerTests extends ReplicationTrackerTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private static void addPeerRecoveryRetentionLease(final ReplicationTracker tracker, final AllocationId allocationId) {
|
||||||
|
final String nodeId = nodeIdFromAllocationId(allocationId);
|
||||||
|
if (tracker.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeId)) == false) {
|
||||||
|
tracker.addPeerRecoveryRetentionLease(nodeId, NO_OPS_PERFORMED, ActionListener.wrap(() -> { }));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static void addPeerRecoveryRetentionLease(final ReplicationTracker tracker, final String allocationId) {
|
||||||
|
addPeerRecoveryRetentionLease(tracker, AllocationId.newInitializing(allocationId));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPeerRecoveryRetentionLeaseCreationAndRenewal() {
|
||||||
|
|
||||||
|
final int numberOfActiveAllocationsIds = randomIntBetween(1, 8);
|
||||||
|
final int numberOfInitializingIds = randomIntBetween(0, 8);
|
||||||
|
final Tuple<Set<AllocationId>, Set<AllocationId>> activeAndInitializingAllocationIds =
|
||||||
|
randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds);
|
||||||
|
final Set<AllocationId> activeAllocationIds = activeAndInitializingAllocationIds.v1();
|
||||||
|
final Set<AllocationId> initializingAllocationIds = activeAndInitializingAllocationIds.v2();
|
||||||
|
|
||||||
|
final AllocationId primaryId = activeAllocationIds.iterator().next();
|
||||||
|
|
||||||
|
final long initialClusterStateVersion = randomNonNegativeLong();
|
||||||
|
|
||||||
|
final AtomicLong currentTimeMillis = new AtomicLong(0L);
|
||||||
|
final ReplicationTracker tracker = newTracker(primaryId, updatedGlobalCheckpoint::set, currentTimeMillis::get);
|
||||||
|
|
||||||
|
final long retentionLeaseExpiryTimeMillis = tracker.indexSettings().getRetentionLeaseMillis();
|
||||||
|
final long peerRecoveryRetentionLeaseRenewalTimeMillis = retentionLeaseExpiryTimeMillis / 2;
|
||||||
|
|
||||||
|
final long maximumTestTimeMillis = 13 * retentionLeaseExpiryTimeMillis;
|
||||||
|
final long testStartTimeMillis = randomLongBetween(0L, Long.MAX_VALUE - maximumTestTimeMillis);
|
||||||
|
currentTimeMillis.set(testStartTimeMillis);
|
||||||
|
|
||||||
|
final Function<AllocationId, RetentionLease> retentionLeaseFromAllocationId = allocationId
|
||||||
|
-> new RetentionLease(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)),
|
||||||
|
0L, currentTimeMillis.get(), ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE);
|
||||||
|
|
||||||
|
final List<RetentionLease> initialLeases = new ArrayList<>();
|
||||||
|
if (randomBoolean()) {
|
||||||
|
initialLeases.add(retentionLeaseFromAllocationId.apply(primaryId));
|
||||||
|
}
|
||||||
|
for (final AllocationId replicaId : initializingAllocationIds) {
|
||||||
|
if (randomBoolean()) {
|
||||||
|
initialLeases.add(retentionLeaseFromAllocationId.apply(replicaId));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (int i = randomIntBetween(0, 5); i > 0; i--) {
|
||||||
|
initialLeases.add(retentionLeaseFromAllocationId.apply(AllocationId.newInitializing()));
|
||||||
|
}
|
||||||
|
tracker.updateRetentionLeasesOnReplica(new RetentionLeases(randomNonNegativeLong(), randomNonNegativeLong(), initialLeases));
|
||||||
|
|
||||||
|
IndexShardRoutingTable routingTable = routingTable(initializingAllocationIds, primaryId);
|
||||||
|
tracker.updateFromMaster(initialClusterStateVersion, ids(activeAllocationIds), routingTable);
|
||||||
|
tracker.activatePrimaryMode(NO_OPS_PERFORMED);
|
||||||
|
assertTrue("primary's retention lease should exist",
|
||||||
|
tracker.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(routingTable.primaryShard())));
|
||||||
|
|
||||||
|
final Consumer<Runnable> assertAsTimePasses = assertion -> {
|
||||||
|
final long startTime = currentTimeMillis.get();
|
||||||
|
while (currentTimeMillis.get() < startTime + retentionLeaseExpiryTimeMillis * 2) {
|
||||||
|
currentTimeMillis.addAndGet(randomLongBetween(0L, retentionLeaseExpiryTimeMillis * 2));
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
tracker.getRetentionLeases(true);
|
||||||
|
assertion.run();
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
assertAsTimePasses.accept(() -> {
|
||||||
|
// Leases for assigned replicas do not expire
|
||||||
|
final RetentionLeases retentionLeases = tracker.getRetentionLeases();
|
||||||
|
for (final AllocationId replicaId : initializingAllocationIds) {
|
||||||
|
final String leaseId = retentionLeaseFromAllocationId.apply(replicaId).id();
|
||||||
|
assertTrue("should not have removed lease for " + replicaId + " in " + retentionLeases,
|
||||||
|
initialLeases.stream().noneMatch(l -> l.id().equals(leaseId)) || retentionLeases.contains(leaseId));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// Leases that don't correspond to assigned replicas, however, are expired by this time.
|
||||||
|
final Set<String> expectedLeaseIds = Stream.concat(Stream.of(primaryId), initializingAllocationIds.stream())
|
||||||
|
.map(allocationId -> retentionLeaseFromAllocationId.apply(allocationId).id()).collect(Collectors.toSet());
|
||||||
|
for (final RetentionLease retentionLease : tracker.getRetentionLeases().leases()) {
|
||||||
|
assertThat(expectedLeaseIds, hasItem(retentionLease.id()));
|
||||||
|
}
|
||||||
|
|
||||||
|
for (AllocationId replicaId : initializingAllocationIds) {
|
||||||
|
markAsTrackingAndInSyncQuietly(tracker, replicaId.getId(), NO_OPS_PERFORMED);
|
||||||
|
}
|
||||||
|
|
||||||
|
assertThat(tracker.getRetentionLeases().leases().stream().map(RetentionLease::id).collect(Collectors.toSet()),
|
||||||
|
equalTo(expectedLeaseIds));
|
||||||
|
|
||||||
|
assertAsTimePasses.accept(() -> {
|
||||||
|
// Leases still don't expire
|
||||||
|
assertThat(tracker.getRetentionLeases().leases().stream().map(RetentionLease::id).collect(Collectors.toSet()),
|
||||||
|
equalTo(expectedLeaseIds));
|
||||||
|
|
||||||
|
// Also leases are renewed before reaching half the expiry time
|
||||||
|
//noinspection OptionalGetWithoutIsPresent
|
||||||
|
assertThat(tracker.getRetentionLeases() + " renewed before too long",
|
||||||
|
tracker.getRetentionLeases().leases().stream().mapToLong(RetentionLease::timestamp).min().getAsLong(),
|
||||||
|
greaterThanOrEqualTo(currentTimeMillis.get() - peerRecoveryRetentionLeaseRenewalTimeMillis));
|
||||||
|
});
|
||||||
|
|
||||||
|
IndexShardRoutingTable.Builder routingTableBuilder = new IndexShardRoutingTable.Builder(routingTable);
|
||||||
|
for (ShardRouting replicaShard : routingTable.replicaShards()) {
|
||||||
|
routingTableBuilder.removeShard(replicaShard);
|
||||||
|
routingTableBuilder.addShard(replicaShard.moveToStarted());
|
||||||
|
}
|
||||||
|
routingTable = routingTableBuilder.build();
|
||||||
|
activeAllocationIds.addAll(initializingAllocationIds);
|
||||||
|
|
||||||
|
tracker.updateFromMaster(initialClusterStateVersion + randomLongBetween(1, 10), ids(activeAllocationIds), routingTable);
|
||||||
|
|
||||||
|
assertAsTimePasses.accept(() -> {
|
||||||
|
// Leases still don't expire
|
||||||
|
assertThat(tracker.getRetentionLeases().leases().stream().map(RetentionLease::id).collect(Collectors.toSet()),
|
||||||
|
equalTo(expectedLeaseIds));
|
||||||
|
// ... and any extra peer recovery retention leases are expired immediately since the shard is fully active
|
||||||
|
tracker.addPeerRecoveryRetentionLease(randomAlphaOfLength(10), randomNonNegativeLong(), ActionListener.wrap(() -> {}));
|
||||||
|
});
|
||||||
|
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
assertTrue("expired extra lease", tracker.getRetentionLeases(true).v1());
|
||||||
|
|
||||||
|
final AllocationId advancingAllocationId
|
||||||
|
= initializingAllocationIds.isEmpty() || rarely() ? primaryId : randomFrom(initializingAllocationIds);
|
||||||
|
final String advancingLeaseId = retentionLeaseFromAllocationId.apply(advancingAllocationId).id();
|
||||||
|
|
||||||
|
final long initialGlobalCheckpoint
|
||||||
|
= Math.max(NO_OPS_PERFORMED, tracker.getTrackedLocalCheckpointForShard(advancingAllocationId.getId()).globalCheckpoint);
|
||||||
|
assertThat(tracker.getRetentionLeases().get(advancingLeaseId).retainingSequenceNumber(), equalTo(initialGlobalCheckpoint + 1));
|
||||||
|
final long newGlobalCheckpoint = initialGlobalCheckpoint + randomLongBetween(1, 1000);
|
||||||
|
tracker.updateGlobalCheckpointForShard(advancingAllocationId.getId(), newGlobalCheckpoint);
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
assertThat("lease was renewed because the shard advanced its global checkpoint",
|
||||||
|
tracker.getRetentionLeases().get(advancingLeaseId).retainingSequenceNumber(), equalTo(newGlobalCheckpoint + 1));
|
||||||
|
|
||||||
|
final long initialVersion = tracker.getRetentionLeases().version();
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
assertThat("immediate renewal is a no-op", tracker.getRetentionLeases().version(), equalTo(initialVersion));
|
||||||
|
|
||||||
|
//noinspection OptionalGetWithoutIsPresent
|
||||||
|
final long millisUntilFirstRenewal
|
||||||
|
= tracker.getRetentionLeases().leases().stream().mapToLong(RetentionLease::timestamp).min().getAsLong()
|
||||||
|
+ peerRecoveryRetentionLeaseRenewalTimeMillis
|
||||||
|
- currentTimeMillis.get();
|
||||||
|
|
||||||
|
if (millisUntilFirstRenewal != 0) {
|
||||||
|
final long shorterThanRenewalTime = randomLongBetween(0L, millisUntilFirstRenewal - 1);
|
||||||
|
currentTimeMillis.addAndGet(shorterThanRenewalTime);
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
assertThat("renewal is a no-op after a short time", tracker.getRetentionLeases().version(), equalTo(initialVersion));
|
||||||
|
currentTimeMillis.addAndGet(millisUntilFirstRenewal - shorterThanRenewalTime);
|
||||||
|
}
|
||||||
|
|
||||||
|
tracker.renewPeerRecoveryRetentionLeases();
|
||||||
|
assertThat("renewal happens after a sufficiently long time", tracker.getRetentionLeases().version(), greaterThan(initialVersion));
|
||||||
|
assertTrue("all leases were renewed",
|
||||||
|
tracker.getRetentionLeases().leases().stream().allMatch(l -> l.timestamp() == currentTimeMillis.get()));
|
||||||
|
|
||||||
|
assertThat("test ran for too long, potentially leading to overflow",
|
||||||
|
currentTimeMillis.get(), lessThanOrEqualTo(testStartTimeMillis + maximumTestTimeMillis));
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -73,11 +73,14 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(stats.getShards());
|
assertNotNull(stats.getShards());
|
||||||
assertThat(stats.getShards(), arrayWithSize(1));
|
assertThat(stats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
assertThat(retentionLease.id(), equalTo(id));
|
assertThat(retentionLease.id(), equalTo(id));
|
||||||
assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber));
|
assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber));
|
||||||
assertThat(retentionLease.source(), equalTo(source));
|
assertThat(retentionLease.source(), equalTo(source));
|
||||||
|
|
||||||
|
assertTrue(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(stats.getShards()[0].getShardRouting())));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testAddAlreadyExists() {
|
public void testAddAlreadyExists() {
|
||||||
|
@ -160,9 +163,11 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(initialStats.getShards());
|
assertNotNull(initialStats.getShards());
|
||||||
assertThat(initialStats.getShards(), arrayWithSize(1));
|
assertThat(initialStats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(initialStats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(initialStats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
|
assertTrue(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(initialStats.getShards()[0].getShardRouting())));
|
||||||
final RetentionLease initialRetentionLease =
|
final RetentionLease initialRetentionLease =
|
||||||
initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
|
|
||||||
final long nextRetainingSequenceNumber =
|
final long nextRetainingSequenceNumber =
|
||||||
retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL
|
retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL
|
||||||
|
@ -195,9 +200,11 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(renewedStats.getShards());
|
assertNotNull(renewedStats.getShards());
|
||||||
assertThat(renewedStats.getShards(), arrayWithSize(1));
|
assertThat(renewedStats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(renewedStats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(renewedStats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
|
assertTrue(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(initialStats.getShards()[0].getShardRouting())));
|
||||||
final RetentionLease renewedRetentionLease =
|
final RetentionLease renewedRetentionLease =
|
||||||
renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
assertThat(renewedRetentionLease.id(), equalTo(id));
|
assertThat(renewedRetentionLease.id(), equalTo(id));
|
||||||
assertThat(
|
assertThat(
|
||||||
renewedRetentionLease.retainingSequenceNumber(),
|
renewedRetentionLease.retainingSequenceNumber(),
|
||||||
|
@ -265,7 +272,9 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(stats.getShards());
|
assertNotNull(stats.getShards());
|
||||||
assertThat(stats.getShards(), arrayWithSize(1));
|
assertThat(stats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(0));
|
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
||||||
|
assertTrue(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(stats.getShards()[0].getShardRouting())));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testRemoveNotFound() {
|
public void testRemoveNotFound() {
|
||||||
|
@ -328,8 +337,10 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(stats.getShards());
|
assertNotNull(stats.getShards());
|
||||||
assertThat(stats.getShards(), arrayWithSize(1));
|
assertThat(stats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
assertTrue(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(stats.getShards()[0].getShardRouting())));
|
||||||
|
final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
assertThat(retentionLease.id(), equalTo(id));
|
assertThat(retentionLease.id(), equalTo(id));
|
||||||
assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber));
|
assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber));
|
||||||
assertThat(retentionLease.source(), equalTo(source));
|
assertThat(retentionLease.source(), equalTo(source));
|
||||||
|
@ -378,9 +389,10 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(initialStats.getShards());
|
assertNotNull(initialStats.getShards());
|
||||||
assertThat(initialStats.getShards(), arrayWithSize(1));
|
assertThat(initialStats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(initialStats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(initialStats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
final RetentionLease initialRetentionLease =
|
assertTrue(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(initialStats.getShards()[0].getShardRouting())));
|
||||||
|
final RetentionLease initialRetentionLease = initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
|
|
||||||
final long nextRetainingSequenceNumber =
|
final long nextRetainingSequenceNumber =
|
||||||
retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL
|
retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL
|
||||||
|
@ -427,9 +439,10 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(renewedStats.getShards());
|
assertNotNull(renewedStats.getShards());
|
||||||
assertThat(renewedStats.getShards(), arrayWithSize(1));
|
assertThat(renewedStats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(renewedStats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(renewedStats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2));
|
||||||
final RetentionLease renewedRetentionLease =
|
assertTrue(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next();
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(renewedStats.getShards()[0].getShardRouting())));
|
||||||
|
final RetentionLease renewedRetentionLease = renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id);
|
||||||
assertThat(renewedRetentionLease.id(), equalTo(id));
|
assertThat(renewedRetentionLease.id(), equalTo(id));
|
||||||
assertThat(
|
assertThat(
|
||||||
renewedRetentionLease.retainingSequenceNumber(),
|
renewedRetentionLease.retainingSequenceNumber(),
|
||||||
|
@ -484,7 +497,9 @@ public class RetentionLeaseActionsTests extends ESSingleNodeTestCase {
|
||||||
assertNotNull(stats.getShards());
|
assertNotNull(stats.getShards());
|
||||||
assertThat(stats.getShards(), arrayWithSize(1));
|
assertThat(stats.getShards(), arrayWithSize(1));
|
||||||
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
assertNotNull(stats.getShards()[0].getRetentionLeaseStats());
|
||||||
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(0));
|
assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
||||||
|
assertTrue(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(stats.getShards()[0].getShardRouting())));
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
|
@ -115,7 +115,8 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
retentionLock.close();
|
retentionLock.close();
|
||||||
|
|
||||||
// check retention leases have been written on the primary
|
// check retention leases have been written on the primary
|
||||||
assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(primary.loadRetentionLeases())));
|
assertThat(currentRetentionLeases,
|
||||||
|
equalTo(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases())));
|
||||||
|
|
||||||
// check current retention leases have been synced to all replicas
|
// check current retention leases have been synced to all replicas
|
||||||
for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) {
|
for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) {
|
||||||
|
@ -124,11 +125,13 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
final IndexShard replica = internalCluster()
|
final IndexShard replica = internalCluster()
|
||||||
.getInstance(IndicesService.class, replicaShardNodeName)
|
.getInstance(IndicesService.class, replicaShardNodeName)
|
||||||
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
||||||
final Map<String, RetentionLease> retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases());
|
final Map<String, RetentionLease> retentionLeasesOnReplica
|
||||||
|
= RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases());
|
||||||
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
||||||
|
|
||||||
// check retention leases have been written on the replica
|
// check retention leases have been written on the replica
|
||||||
assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases())));
|
assertThat(currentRetentionLeases,
|
||||||
|
equalTo(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases())));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -173,7 +176,8 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
retentionLock.close();
|
retentionLock.close();
|
||||||
|
|
||||||
// check retention leases have been written on the primary
|
// check retention leases have been written on the primary
|
||||||
assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(primary.loadRetentionLeases())));
|
assertThat(currentRetentionLeases,
|
||||||
|
equalTo(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases())));
|
||||||
|
|
||||||
// check current retention leases have been synced to all replicas
|
// check current retention leases have been synced to all replicas
|
||||||
for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) {
|
for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) {
|
||||||
|
@ -182,11 +186,13 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
final IndexShard replica = internalCluster()
|
final IndexShard replica = internalCluster()
|
||||||
.getInstance(IndicesService.class, replicaShardNodeName)
|
.getInstance(IndicesService.class, replicaShardNodeName)
|
||||||
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
||||||
final Map<String, RetentionLease> retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases());
|
final Map<String, RetentionLease> retentionLeasesOnReplica =
|
||||||
|
RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases());
|
||||||
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
||||||
|
|
||||||
// check retention leases have been written on the replica
|
// check retention leases have been written on the replica
|
||||||
assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases())));
|
assertThat(currentRetentionLeases,
|
||||||
|
equalTo(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases())));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -239,7 +245,8 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
final IndexShard replica = internalCluster()
|
final IndexShard replica = internalCluster()
|
||||||
.getInstance(IndicesService.class, replicaShardNodeName)
|
.getInstance(IndicesService.class, replicaShardNodeName)
|
||||||
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
||||||
assertThat(replica.getRetentionLeases().leases(), anyOf(empty(), contains(currentRetentionLease)));
|
assertThat(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()).values(),
|
||||||
|
anyOf(empty(), contains(currentRetentionLease)));
|
||||||
}
|
}
|
||||||
|
|
||||||
// update the index for retention leases to short a long time, to force expiration
|
// update the index for retention leases to short a long time, to force expiration
|
||||||
|
@ -256,7 +263,8 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
// sleep long enough that the current retention lease has expired
|
// sleep long enough that the current retention lease has expired
|
||||||
final long later = System.nanoTime();
|
final long later = System.nanoTime();
|
||||||
Thread.sleep(Math.max(0, retentionLeaseTimeToLive.millis() - TimeUnit.NANOSECONDS.toMillis(later - now)));
|
Thread.sleep(Math.max(0, retentionLeaseTimeToLive.millis() - TimeUnit.NANOSECONDS.toMillis(later - now)));
|
||||||
assertBusy(() -> assertThat(primary.getRetentionLeases().leases(), empty()));
|
assertBusy(() -> assertThat(
|
||||||
|
RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.getRetentionLeases()).entrySet(), empty()));
|
||||||
|
|
||||||
// now that all retention leases are expired should have been synced to all replicas
|
// now that all retention leases are expired should have been synced to all replicas
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
|
@ -266,7 +274,8 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
final IndexShard replica = internalCluster()
|
final IndexShard replica = internalCluster()
|
||||||
.getInstance(IndicesService.class, replicaShardNodeName)
|
.getInstance(IndicesService.class, replicaShardNodeName)
|
||||||
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
||||||
assertThat(replica.getRetentionLeases().leases(), empty());
|
assertThat(
|
||||||
|
RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()).entrySet(), empty());
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -432,11 +441,13 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
final IndexShard replica = internalCluster()
|
final IndexShard replica = internalCluster()
|
||||||
.getInstance(IndicesService.class, replicaShardNodeName)
|
.getInstance(IndicesService.class, replicaShardNodeName)
|
||||||
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
.getShardOrNull(new ShardId(resolveIndex("index"), 0));
|
||||||
final Map<String, RetentionLease> retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases());
|
final Map<String, RetentionLease> retentionLeasesOnReplica
|
||||||
|
= RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases());
|
||||||
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases));
|
||||||
|
|
||||||
// check retention leases have been written on the replica; see RecoveryTarget#finalizeRecovery
|
// check retention leases have been written on the replica; see RecoveryTarget#finalizeRecovery
|
||||||
assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases())));
|
assertThat(currentRetentionLeases,
|
||||||
|
equalTo(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases())));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -474,7 +485,9 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
* way for the current retention leases to end up written to disk so we assume that if they are written to disk, it
|
* way for the current retention leases to end up written to disk so we assume that if they are written to disk, it
|
||||||
* implies that the background sync was able to execute under a block.
|
* implies that the background sync was able to execute under a block.
|
||||||
*/
|
*/
|
||||||
assertBusy(() -> assertThat(primary.loadRetentionLeases().leases(), contains(retentionLease.get())));
|
assertBusy(() -> assertThat(
|
||||||
|
RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases()).values(),
|
||||||
|
contains(retentionLease.get())));
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
failWithException(e);
|
failWithException(e);
|
||||||
}
|
}
|
||||||
|
@ -593,7 +606,9 @@ public class RetentionLeaseIT extends ESIntegTestCase {
|
||||||
* way for the current retention leases to end up written to disk so we assume that if they are written to disk, it
|
* way for the current retention leases to end up written to disk so we assume that if they are written to disk, it
|
||||||
* implies that the background sync was able to execute despite wait for shards being set on the index.
|
* implies that the background sync was able to execute despite wait for shards being set on the index.
|
||||||
*/
|
*/
|
||||||
assertBusy(() -> assertThat(primary.loadRetentionLeases().leases(), contains(retentionLease.get())));
|
assertBusy(() -> assertThat(
|
||||||
|
RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases()).values(),
|
||||||
|
contains(retentionLease.get())));
|
||||||
} catch (final Exception e) {
|
} catch (final Exception e) {
|
||||||
failWithException(e);
|
failWithException(e);
|
||||||
}
|
}
|
||||||
|
|
|
@ -63,7 +63,8 @@ public class RetentionLeaseStatsTests extends ESSingleNodeTestCase {
|
||||||
final IndicesStatsResponse indicesStats = client().admin().indices().prepareStats("index").execute().actionGet();
|
final IndicesStatsResponse indicesStats = client().admin().indices().prepareStats("index").execute().actionGet();
|
||||||
assertThat(indicesStats.getShards(), arrayWithSize(1));
|
assertThat(indicesStats.getShards(), arrayWithSize(1));
|
||||||
final RetentionLeaseStats retentionLeaseStats = indicesStats.getShards()[0].getRetentionLeaseStats();
|
final RetentionLeaseStats retentionLeaseStats = indicesStats.getShards()[0].getRetentionLeaseStats();
|
||||||
assertThat(RetentionLeases.toMap(retentionLeaseStats.retentionLeases()), equalTo(currentRetentionLeases));
|
assertThat(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(retentionLeaseStats.retentionLeases()),
|
||||||
|
equalTo(currentRetentionLeases));
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,6 +27,7 @@ import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.engine.InternalEngineFactory;
|
import org.elasticsearch.index.engine.InternalEngineFactory;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.RetentionLease;
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseStats;
|
import org.elasticsearch.index.seqno.RetentionLeaseStats;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
|
@ -35,14 +36,13 @@ import org.elasticsearch.threadpool.TestThreadPool;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Collections;
|
import java.util.Arrays;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.contains;
|
import static org.hamcrest.Matchers.contains;
|
||||||
import static org.hamcrest.Matchers.empty;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.hasItem;
|
import static org.hamcrest.Matchers.hasItem;
|
||||||
import static org.hamcrest.Matchers.hasSize;
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
|
@ -73,7 +73,7 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
indexShard.addRetentionLease(
|
indexShard.addRetentionLease(
|
||||||
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
||||||
assertRetentionLeases(
|
assertRetentionLeases(
|
||||||
indexShard, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 1 + i, true, false);
|
indexShard, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 2 + i, true, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int i = 0; i < length; i++) {
|
for (int i = 0; i < length; i++) {
|
||||||
|
@ -84,7 +84,7 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
length,
|
length,
|
||||||
minimumRetainingSequenceNumbers,
|
minimumRetainingSequenceNumbers,
|
||||||
primaryTerm,
|
primaryTerm,
|
||||||
1 + length + i,
|
2 + length + i,
|
||||||
true,
|
true,
|
||||||
false);
|
false);
|
||||||
}
|
}
|
||||||
|
@ -105,7 +105,7 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
indexShard.addRetentionLease(
|
indexShard.addRetentionLease(
|
||||||
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {}));
|
||||||
assertRetentionLeases(
|
assertRetentionLeases(
|
||||||
indexShard, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 1 + i, true, false);
|
indexShard, i + 1, minimumRetainingSequenceNumbers, primaryTerm, 2 + i, true, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
for (int i = 0; i < length; i++) {
|
for (int i = 0; i < length; i++) {
|
||||||
|
@ -115,7 +115,7 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
length - i - 1,
|
length - i - 1,
|
||||||
minimumRetainingSequenceNumbers,
|
minimumRetainingSequenceNumbers,
|
||||||
primaryTerm,
|
primaryTerm,
|
||||||
1 + length + i,
|
2 + length + i,
|
||||||
true,
|
true,
|
||||||
false);
|
false);
|
||||||
}
|
}
|
||||||
|
@ -132,6 +132,12 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
runExpirationTest(false);
|
runExpirationTest(false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private RetentionLease peerRecoveryRetentionLease(IndexShard indexShard) {
|
||||||
|
return new RetentionLease(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(indexShard.routingEntry()), 0, currentTimeMillis.get(),
|
||||||
|
ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE);
|
||||||
|
}
|
||||||
|
|
||||||
private void runExpirationTest(final boolean primary) throws IOException {
|
private void runExpirationTest(final boolean primary) throws IOException {
|
||||||
final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis());
|
final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis());
|
||||||
final Settings settings = Settings
|
final Settings settings = Settings
|
||||||
|
@ -147,23 +153,28 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
try {
|
try {
|
||||||
final long[] retainingSequenceNumbers = new long[1];
|
final long[] retainingSequenceNumbers = new long[1];
|
||||||
retainingSequenceNumbers[0] = randomLongBetween(0, Long.MAX_VALUE);
|
retainingSequenceNumbers[0] = randomLongBetween(0, Long.MAX_VALUE);
|
||||||
|
final long initialVersion;
|
||||||
if (primary) {
|
if (primary) {
|
||||||
|
initialVersion = 2;
|
||||||
indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> {}));
|
indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> {}));
|
||||||
} else {
|
} else {
|
||||||
|
initialVersion = 3;
|
||||||
final RetentionLeases retentionLeases = new RetentionLeases(
|
final RetentionLeases retentionLeases = new RetentionLeases(
|
||||||
primaryTerm,
|
primaryTerm,
|
||||||
1,
|
initialVersion,
|
||||||
Collections.singleton(new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0")));
|
Arrays.asList(
|
||||||
|
peerRecoveryRetentionLease(indexShard),
|
||||||
|
new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0")));
|
||||||
indexShard.updateRetentionLeasesOnReplica(retentionLeases);
|
indexShard.updateRetentionLeasesOnReplica(retentionLeases);
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
||||||
assertThat(retentionLeases.version(), equalTo(1L));
|
assertThat(retentionLeases.version(), equalTo(initialVersion));
|
||||||
assertThat(retentionLeases.leases(), hasSize(1));
|
assertThat(retentionLeases.leases(), hasSize(2));
|
||||||
final RetentionLease retentionLease = retentionLeases.leases().iterator().next();
|
final RetentionLease retentionLease = retentionLeases.get("0");
|
||||||
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
||||||
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 1, primary, false);
|
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion, primary, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// renew the lease
|
// renew the lease
|
||||||
|
@ -174,28 +185,30 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
} else {
|
} else {
|
||||||
final RetentionLeases retentionLeases = new RetentionLeases(
|
final RetentionLeases retentionLeases = new RetentionLeases(
|
||||||
primaryTerm,
|
primaryTerm,
|
||||||
2,
|
initialVersion + 1,
|
||||||
Collections.singleton(new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0")));
|
Arrays.asList(
|
||||||
|
peerRecoveryRetentionLease(indexShard),
|
||||||
|
new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0")));
|
||||||
indexShard.updateRetentionLeasesOnReplica(retentionLeases);
|
indexShard.updateRetentionLeasesOnReplica(retentionLeases);
|
||||||
}
|
}
|
||||||
|
|
||||||
{
|
{
|
||||||
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
||||||
assertThat(retentionLeases.version(), equalTo(2L));
|
assertThat(retentionLeases.version(), equalTo(initialVersion + 1));
|
||||||
assertThat(retentionLeases.leases(), hasSize(1));
|
assertThat(retentionLeases.leases(), hasSize(2));
|
||||||
final RetentionLease retentionLease = retentionLeases.leases().iterator().next();
|
final RetentionLease retentionLease = retentionLeases.get("0");
|
||||||
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get()));
|
||||||
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, primary, false);
|
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion + 1, primary, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
// now force the lease to expire
|
// now force the lease to expire
|
||||||
currentTimeMillis.set(
|
currentTimeMillis.set(
|
||||||
currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get()));
|
currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get()));
|
||||||
if (primary) {
|
if (primary) {
|
||||||
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, true, false);
|
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion + 1, true, false);
|
||||||
assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, 3, true, true);
|
assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, initialVersion + 2, true, true);
|
||||||
} else {
|
} else {
|
||||||
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, false, false);
|
assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion + 1, false, false);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
closeShards(indexShard);
|
closeShards(indexShard);
|
||||||
|
@ -229,13 +242,8 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
// the written retention leases should equal our current retention leases
|
// the written retention leases should equal our current retention leases
|
||||||
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get();
|
||||||
final RetentionLeases writtenRetentionLeases = indexShard.loadRetentionLeases();
|
final RetentionLeases writtenRetentionLeases = indexShard.loadRetentionLeases();
|
||||||
if (retentionLeases.leases().isEmpty()) {
|
assertThat(writtenRetentionLeases.version(), equalTo(1L + length));
|
||||||
assertThat(writtenRetentionLeases.version(), equalTo(0L));
|
assertThat(writtenRetentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0])));
|
||||||
assertThat(writtenRetentionLeases.leases(), empty());
|
|
||||||
} else {
|
|
||||||
assertThat(writtenRetentionLeases.version(), equalTo((long) length));
|
|
||||||
assertThat(retentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0])));
|
|
||||||
}
|
|
||||||
|
|
||||||
// when we recover, we should recover the retention leases
|
// when we recover, we should recover the retention leases
|
||||||
final IndexShard recoveredShard = reinitShard(
|
final IndexShard recoveredShard = reinitShard(
|
||||||
|
@ -244,15 +252,10 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
try {
|
try {
|
||||||
recoverShardFromStore(recoveredShard);
|
recoverShardFromStore(recoveredShard);
|
||||||
final RetentionLeases recoveredRetentionLeases = recoveredShard.getEngine().config().retentionLeasesSupplier().get();
|
final RetentionLeases recoveredRetentionLeases = recoveredShard.getEngine().config().retentionLeasesSupplier().get();
|
||||||
if (retentionLeases.leases().isEmpty()) {
|
assertThat(recoveredRetentionLeases.version(), equalTo(1L + length));
|
||||||
assertThat(recoveredRetentionLeases.version(), equalTo(0L));
|
assertThat(
|
||||||
assertThat(recoveredRetentionLeases.leases(), empty());
|
recoveredRetentionLeases.leases(),
|
||||||
} else {
|
contains(retentionLeases.leases().toArray(new RetentionLease[0])));
|
||||||
assertThat(recoveredRetentionLeases.version(), equalTo((long) length));
|
|
||||||
assertThat(
|
|
||||||
recoveredRetentionLeases.leases(),
|
|
||||||
contains(retentionLeases.leases().toArray(new RetentionLease[0])));
|
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
closeShards(recoveredShard);
|
closeShards(recoveredShard);
|
||||||
}
|
}
|
||||||
|
@ -265,8 +268,10 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
try {
|
try {
|
||||||
recoverShardFromStore(forceRecoveredShard);
|
recoverShardFromStore(forceRecoveredShard);
|
||||||
final RetentionLeases recoveredRetentionLeases = forceRecoveredShard.getEngine().config().retentionLeasesSupplier().get();
|
final RetentionLeases recoveredRetentionLeases = forceRecoveredShard.getEngine().config().retentionLeasesSupplier().get();
|
||||||
assertThat(recoveredRetentionLeases.leases(), empty());
|
assertThat(recoveredRetentionLeases.leases(), hasSize(1));
|
||||||
assertThat(recoveredRetentionLeases.version(), equalTo(0L));
|
assertThat(recoveredRetentionLeases.leases().iterator().next().id(),
|
||||||
|
equalTo(ReplicationTracker.getPeerRecoveryRetentionLeaseId(indexShard.routingEntry())));
|
||||||
|
assertThat(recoveredRetentionLeases.version(), equalTo(1L));
|
||||||
} finally {
|
} finally {
|
||||||
closeShards(forceRecoveredShard);
|
closeShards(forceRecoveredShard);
|
||||||
}
|
}
|
||||||
|
@ -291,8 +296,8 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
stats.retentionLeases(),
|
stats.retentionLeases(),
|
||||||
length,
|
length,
|
||||||
minimumRetainingSequenceNumbers,
|
minimumRetainingSequenceNumbers,
|
||||||
length == 0 ? RetentionLeases.EMPTY.primaryTerm() : indexShard.getOperationPrimaryTerm(),
|
indexShard.getOperationPrimaryTerm(),
|
||||||
length);
|
length + 1);
|
||||||
} finally {
|
} finally {
|
||||||
closeShards(indexShard);
|
closeShards(indexShard);
|
||||||
}
|
}
|
||||||
|
@ -355,7 +360,9 @@ public class IndexShardRetentionLeaseTests extends IndexShardTestCase {
|
||||||
assertThat(retentionLeases.version(), equalTo(version));
|
assertThat(retentionLeases.version(), equalTo(version));
|
||||||
final Map<String, RetentionLease> idToRetentionLease = new HashMap<>();
|
final Map<String, RetentionLease> idToRetentionLease = new HashMap<>();
|
||||||
for (final RetentionLease retentionLease : retentionLeases.leases()) {
|
for (final RetentionLease retentionLease : retentionLeases.leases()) {
|
||||||
idToRetentionLease.put(retentionLease.id(), retentionLease);
|
if (ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()) == false) {
|
||||||
|
idToRetentionLease.put(retentionLease.id(), retentionLease);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
assertThat(idToRetentionLease.entrySet(), hasSize(size));
|
assertThat(idToRetentionLease.entrySet(), hasSize(size));
|
||||||
|
|
|
@ -103,6 +103,8 @@ import org.elasticsearch.index.mapper.SourceFieldMapper;
|
||||||
import org.elasticsearch.index.mapper.SourceToParse;
|
import org.elasticsearch.index.mapper.SourceToParse;
|
||||||
import org.elasticsearch.index.mapper.Uid;
|
import org.elasticsearch.index.mapper.Uid;
|
||||||
import org.elasticsearch.index.mapper.VersionFieldMapper;
|
import org.elasticsearch.index.mapper.VersionFieldMapper;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseSyncer;
|
import org.elasticsearch.index.seqno.RetentionLeaseSyncer;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.seqno.SeqNoStats;
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
|
@ -1563,14 +1565,17 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
|
|
||||||
public void testRefreshMetric() throws IOException {
|
public void testRefreshMetric() throws IOException {
|
||||||
IndexShard shard = newStartedShard();
|
IndexShard shard = newStartedShard();
|
||||||
assertThat(shard.refreshStats().getTotal(), equalTo(2L)); // refresh on: finalize and end of recovery
|
// refresh on: finalize and end of recovery
|
||||||
|
// finalizing a replica involves two refreshes with soft deletes because of estimateNumberOfHistoryOperations()
|
||||||
|
final long initialRefreshes = shard.routingEntry().primary() || shard.indexSettings().isSoftDeleteEnabled() == false ? 2L : 3L;
|
||||||
|
assertThat(shard.refreshStats().getTotal(), equalTo(initialRefreshes));
|
||||||
long initialTotalTime = shard.refreshStats().getTotalTimeInMillis();
|
long initialTotalTime = shard.refreshStats().getTotalTimeInMillis();
|
||||||
// check time advances
|
// check time advances
|
||||||
for (int i = 1; shard.refreshStats().getTotalTimeInMillis() == initialTotalTime; i++) {
|
for (int i = 1; shard.refreshStats().getTotalTimeInMillis() == initialTotalTime; i++) {
|
||||||
indexDoc(shard, "_doc", "test");
|
indexDoc(shard, "_doc", "test");
|
||||||
assertThat(shard.refreshStats().getTotal(), equalTo(2L + i - 1));
|
assertThat(shard.refreshStats().getTotal(), equalTo(initialRefreshes + i - 1));
|
||||||
shard.refresh("test");
|
shard.refresh("test");
|
||||||
assertThat(shard.refreshStats().getTotal(), equalTo(2L + i));
|
assertThat(shard.refreshStats().getTotal(), equalTo(initialRefreshes + i));
|
||||||
assertThat(shard.refreshStats().getTotalTimeInMillis(), greaterThanOrEqualTo(initialTotalTime));
|
assertThat(shard.refreshStats().getTotalTimeInMillis(), greaterThanOrEqualTo(initialTotalTime));
|
||||||
}
|
}
|
||||||
long refreshCount = shard.refreshStats().getTotal();
|
long refreshCount = shard.refreshStats().getTotal();
|
||||||
|
@ -1597,18 +1602,18 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
assertThat(shard.refreshStats().getExternalTotal(), equalTo(2L + i));
|
assertThat(shard.refreshStats().getExternalTotal(), equalTo(2L + i));
|
||||||
assertThat(shard.refreshStats().getExternalTotalTimeInMillis(), greaterThanOrEqualTo(initialTotalTime));
|
assertThat(shard.refreshStats().getExternalTotalTimeInMillis(), greaterThanOrEqualTo(initialTotalTime));
|
||||||
}
|
}
|
||||||
long externalRefreshCount = shard.refreshStats().getExternalTotal();
|
final long externalRefreshCount = shard.refreshStats().getExternalTotal();
|
||||||
|
final long extraInternalRefreshes = shard.routingEntry().primary() || shard.indexSettings().isSoftDeleteEnabled() == false ? 0 : 1;
|
||||||
indexDoc(shard, "_doc", "test");
|
indexDoc(shard, "_doc", "test");
|
||||||
try (Engine.GetResult ignored = shard.get(new Engine.Get(true, false, "_doc", "test",
|
try (Engine.GetResult ignored = shard.get(new Engine.Get(true, false, "_doc", "test",
|
||||||
new Term(IdFieldMapper.NAME, Uid.encodeId("test"))))) {
|
new Term(IdFieldMapper.NAME, Uid.encodeId("test"))))) {
|
||||||
assertThat(shard.refreshStats().getExternalTotal(), equalTo(externalRefreshCount));
|
assertThat(shard.refreshStats().getExternalTotal(), equalTo(externalRefreshCount));
|
||||||
assertThat(shard.refreshStats().getExternalTotal(), equalTo(shard.refreshStats().getTotal() - 1));
|
assertThat(shard.refreshStats().getExternalTotal(), equalTo(shard.refreshStats().getTotal() - 1 - extraInternalRefreshes));
|
||||||
}
|
}
|
||||||
indexDoc(shard, "_doc", "test");
|
indexDoc(shard, "_doc", "test");
|
||||||
shard.writeIndexingBuffer();
|
shard.writeIndexingBuffer();
|
||||||
assertThat(shard.refreshStats().getExternalTotal(), equalTo(externalRefreshCount));
|
assertThat(shard.refreshStats().getExternalTotal(), equalTo(externalRefreshCount));
|
||||||
assertThat(shard.refreshStats().getExternalTotal(), equalTo(shard.refreshStats().getTotal() - 2));
|
assertThat(shard.refreshStats().getExternalTotal(), equalTo(shard.refreshStats().getTotal() - 2 - extraInternalRefreshes));
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -2116,7 +2121,8 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
}
|
}
|
||||||
IndexShardTestCase.updateRoutingEntry(primarySource,
|
IndexShardTestCase.updateRoutingEntry(primarySource,
|
||||||
primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1));
|
primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1));
|
||||||
final IndexShard primaryTarget = newShard(primarySource.routingEntry().getTargetRelocatingShard());
|
final IndexShard primaryTarget = newShard(primarySource.routingEntry().getTargetRelocatingShard(), Settings.builder()
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), primarySource.indexSettings().isSoftDeleteEnabled()).build());
|
||||||
updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetaData());
|
updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetaData());
|
||||||
recoverReplica(primaryTarget, primarySource, true);
|
recoverReplica(primaryTarget, primarySource, true);
|
||||||
|
|
||||||
|
@ -2867,7 +2873,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
public void testDocStats() throws Exception {
|
public void testDocStats() throws Exception {
|
||||||
IndexShard indexShard = null;
|
IndexShard indexShard = null;
|
||||||
try {
|
try {
|
||||||
indexShard = newStartedShard(
|
indexShard = newStartedShard(false,
|
||||||
Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build());
|
Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), 0).build());
|
||||||
final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete
|
final long numDocs = randomIntBetween(2, 32); // at least two documents so we have docs to delete
|
||||||
final long numDocsToDelete = randomLongBetween(1, numDocs);
|
final long numDocsToDelete = randomLongBetween(1, numDocs);
|
||||||
|
@ -2905,15 +2911,23 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
deleteDoc(indexShard, "_doc", id);
|
deleteDoc(indexShard, "_doc", id);
|
||||||
indexDoc(indexShard, "_doc", id);
|
indexDoc(indexShard, "_doc", id);
|
||||||
}
|
}
|
||||||
// Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it.
|
// Need to update and sync the global checkpoint and the retention leases for the soft-deletes retention MergePolicy.
|
||||||
if (indexShard.indexSettings.isSoftDeleteEnabled()) {
|
if (indexShard.indexSettings.isSoftDeleteEnabled()) {
|
||||||
|
final long newGlobalCheckpoint = indexShard.getLocalCheckpoint();
|
||||||
if (indexShard.routingEntry().primary()) {
|
if (indexShard.routingEntry().primary()) {
|
||||||
indexShard.updateLocalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
indexShard.updateLocalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
||||||
indexShard.getLocalCheckpoint());
|
indexShard.getLocalCheckpoint());
|
||||||
indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
||||||
indexShard.getLocalCheckpoint());
|
indexShard.getLocalCheckpoint());
|
||||||
|
indexShard.syncRetentionLeases();
|
||||||
} else {
|
} else {
|
||||||
indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test");
|
indexShard.updateGlobalCheckpointOnReplica(newGlobalCheckpoint, "test");
|
||||||
|
|
||||||
|
final RetentionLeases retentionLeases = indexShard.getRetentionLeases();
|
||||||
|
indexShard.updateRetentionLeasesOnReplica(new RetentionLeases(
|
||||||
|
retentionLeases.primaryTerm(), retentionLeases.version() + 1,
|
||||||
|
retentionLeases.leases().stream().map(lease -> new RetentionLease(lease.id(), newGlobalCheckpoint + 1,
|
||||||
|
lease.timestamp(), ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE)).collect(Collectors.toList())));
|
||||||
}
|
}
|
||||||
indexShard.sync();
|
indexShard.sync();
|
||||||
}
|
}
|
||||||
|
@ -3501,6 +3515,10 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
// In order to instruct the merge policy not to keep a fully deleted segment,
|
// In order to instruct the merge policy not to keep a fully deleted segment,
|
||||||
// we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything.
|
// we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything.
|
||||||
if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) {
|
if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) {
|
||||||
|
primary.updateGlobalCheckpointForShard(
|
||||||
|
primary.routingEntry().allocationId().getId(),
|
||||||
|
primary.getLastSyncedGlobalCheckpoint());
|
||||||
|
primary.syncRetentionLeases();
|
||||||
primary.sync();
|
primary.sync();
|
||||||
flushShard(primary);
|
flushShard(primary);
|
||||||
}
|
}
|
||||||
|
@ -4019,10 +4037,10 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
assertTrue(indexResult.isCreated());
|
assertTrue(indexResult.isCreated());
|
||||||
|
|
||||||
DeleteResult deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "some_other_type", "id", VersionType.INTERNAL,
|
DeleteResult deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "some_other_type", "id", VersionType.INTERNAL,
|
||||||
UNASSIGNED_SEQ_NO, 0);
|
UNASSIGNED_SEQ_NO, 1);
|
||||||
assertFalse(deleteResult.isFound());
|
assertFalse(deleteResult.isFound());
|
||||||
|
|
||||||
deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "_doc", "id", VersionType.INTERNAL, UNASSIGNED_SEQ_NO, 0);
|
deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "_doc", "id", VersionType.INTERNAL, UNASSIGNED_SEQ_NO, 1);
|
||||||
assertTrue(deleteResult.isFound());
|
assertTrue(deleteResult.isFound());
|
||||||
|
|
||||||
closeShards(shard);
|
closeShards(shard);
|
||||||
|
@ -4098,7 +4116,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
||||||
final ShardRouting replicaRouting = shard.routingEntry();
|
final ShardRouting replicaRouting = shard.routingEntry();
|
||||||
ShardRouting readonlyShardRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true,
|
ShardRouting readonlyShardRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true,
|
||||||
ShardRoutingState.INITIALIZING, RecoverySource.ExistingStoreRecoverySource.INSTANCE);
|
ShardRoutingState.INITIALIZING, RecoverySource.ExistingStoreRecoverySource.INSTANCE);
|
||||||
final IndexShard readonlyShard = reinitShard(shard, readonlyShardRouting,
|
final IndexShard readonlyShard = reinitShard(shard, readonlyShardRouting, shard.indexSettings.getIndexMetaData(),
|
||||||
engineConfig -> new ReadOnlyEngine(engineConfig, null, null, true, Function.identity()) {
|
engineConfig -> new ReadOnlyEngine(engineConfig, null, null, true, Function.identity()) {
|
||||||
@Override
|
@Override
|
||||||
protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(SeqNoStats seqNoStats) {
|
protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(SeqNoStats seqNoStats) {
|
||||||
|
|
|
@ -124,10 +124,16 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase {
|
||||||
assertThat(resyncRequest.getMaxSeenAutoIdTimestampOnPrimary(), equalTo(shard.getMaxSeenAutoIdTimestamp()));
|
assertThat(resyncRequest.getMaxSeenAutoIdTimestampOnPrimary(), equalTo(shard.getMaxSeenAutoIdTimestamp()));
|
||||||
}
|
}
|
||||||
if (syncNeeded && globalCheckPoint < numDocs - 1) {
|
if (syncNeeded && globalCheckPoint < numDocs - 1) {
|
||||||
int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included
|
if (shard.indexSettings.isSoftDeleteEnabled()) {
|
||||||
assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps));
|
assertThat(resyncTask.getSkippedOperations(), equalTo(0));
|
||||||
assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps));
|
assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations()));
|
||||||
assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs));
|
assertThat(resyncTask.getTotalOperations(), equalTo(Math.toIntExact(numDocs - 1 - globalCheckPoint)));
|
||||||
|
} else {
|
||||||
|
int skippedOps = Math.toIntExact(globalCheckPoint + 1); // everything up to global checkpoint included
|
||||||
|
assertThat(resyncTask.getSkippedOperations(), equalTo(skippedOps));
|
||||||
|
assertThat(resyncTask.getResyncedOperations(), equalTo(numDocs - skippedOps));
|
||||||
|
assertThat(resyncTask.getTotalOperations(), equalTo(globalCheckPoint == numDocs - 1 ? 0 : numDocs));
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
assertThat(resyncTask.getSkippedOperations(), equalTo(0));
|
assertThat(resyncTask.getSkippedOperations(), equalTo(0));
|
||||||
assertThat(resyncTask.getResyncedOperations(), equalTo(0));
|
assertThat(resyncTask.getResyncedOperations(), equalTo(0));
|
||||||
|
|
|
@ -34,26 +34,41 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||||
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||||
import org.elasticsearch.action.index.IndexRequestBuilder;
|
import org.elasticsearch.action.index.IndexRequestBuilder;
|
||||||
import org.elasticsearch.action.search.SearchResponse;
|
import org.elasticsearch.action.search.SearchResponse;
|
||||||
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
|
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
|
||||||
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
|
import org.elasticsearch.cluster.ClusterState;
|
||||||
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
import org.elasticsearch.cluster.action.shard.ShardStateAction;
|
||||||
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.node.DiscoveryNodes;
|
||||||
|
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource;
|
import org.elasticsearch.cluster.routing.RecoverySource.PeerRecoverySource;
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
|
import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||||
|
import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand;
|
||||||
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
|
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
|
||||||
import org.elasticsearch.cluster.service.ClusterService;
|
import org.elasticsearch.cluster.service.ClusterService;
|
||||||
|
import org.elasticsearch.common.Priority;
|
||||||
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.common.unit.TimeValue;
|
import org.elasticsearch.common.unit.TimeValue;
|
||||||
import org.elasticsearch.common.xcontent.XContentType;
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
|
import org.elasticsearch.index.IndexService;
|
||||||
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.index.MockEngineFactoryPlugin;
|
||||||
import org.elasticsearch.index.analysis.AbstractTokenFilterFactory;
|
import org.elasticsearch.index.analysis.AbstractTokenFilterFactory;
|
||||||
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
import org.elasticsearch.index.analysis.TokenFilterFactory;
|
||||||
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.mapper.MapperParsingException;
|
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||||
import org.elasticsearch.index.recovery.RecoveryStats;
|
import org.elasticsearch.index.recovery.RecoveryStats;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.index.store.Store;
|
import org.elasticsearch.index.store.Store;
|
||||||
import org.elasticsearch.indices.IndicesService;
|
import org.elasticsearch.indices.IndicesService;
|
||||||
|
@ -70,7 +85,9 @@ import org.elasticsearch.test.BackgroundIndexer;
|
||||||
import org.elasticsearch.test.ESIntegTestCase;
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
|
||||||
import org.elasticsearch.test.ESIntegTestCase.Scope;
|
import org.elasticsearch.test.ESIntegTestCase.Scope;
|
||||||
|
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||||
import org.elasticsearch.test.InternalTestCluster;
|
import org.elasticsearch.test.InternalTestCluster;
|
||||||
|
import org.elasticsearch.test.engine.MockEngineSupport;
|
||||||
import org.elasticsearch.test.store.MockFSIndexStore;
|
import org.elasticsearch.test.store.MockFSIndexStore;
|
||||||
import org.elasticsearch.test.transport.MockTransportService;
|
import org.elasticsearch.test.transport.MockTransportService;
|
||||||
import org.elasticsearch.test.transport.StubbableTransport;
|
import org.elasticsearch.test.transport.StubbableTransport;
|
||||||
|
@ -79,7 +96,6 @@ import org.elasticsearch.transport.Transport;
|
||||||
import org.elasticsearch.transport.TransportRequest;
|
import org.elasticsearch.transport.TransportRequest;
|
||||||
import org.elasticsearch.transport.TransportRequestOptions;
|
import org.elasticsearch.transport.TransportRequestOptions;
|
||||||
import org.elasticsearch.transport.TransportService;
|
import org.elasticsearch.transport.TransportService;
|
||||||
import org.junit.After;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
|
@ -97,6 +113,7 @@ import java.util.function.Consumer;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
import java.util.stream.IntStream;
|
import java.util.stream.IntStream;
|
||||||
import java.util.stream.Stream;
|
import java.util.stream.Stream;
|
||||||
|
import java.util.stream.StreamSupport;
|
||||||
|
|
||||||
import static java.util.Collections.singletonMap;
|
import static java.util.Collections.singletonMap;
|
||||||
import static java.util.stream.Collectors.toList;
|
import static java.util.stream.Collectors.toList;
|
||||||
|
@ -126,13 +143,21 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
return Arrays.asList(MockTransportService.TestPlugin.class, MockFSIndexStore.TestPlugin.class,
|
return Arrays.asList(
|
||||||
RecoverySettingsChunkSizePlugin.class, TestAnalysisPlugin.class);
|
MockTransportService.TestPlugin.class,
|
||||||
|
MockFSIndexStore.TestPlugin.class,
|
||||||
|
RecoverySettingsChunkSizePlugin.class,
|
||||||
|
TestAnalysisPlugin.class,
|
||||||
|
InternalSettingsPlugin.class,
|
||||||
|
MockEngineFactoryPlugin.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@After
|
@Override
|
||||||
public void assertConsistentHistoryInLuceneIndex() throws Exception {
|
protected void beforeIndexDeletion() throws Exception {
|
||||||
|
super.beforeIndexDeletion();
|
||||||
internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex();
|
internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex();
|
||||||
|
internalCluster().assertSeqNos();
|
||||||
|
internalCluster().assertSameDocIdsOnShards();
|
||||||
}
|
}
|
||||||
|
|
||||||
private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary,
|
private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary,
|
||||||
|
@ -852,6 +877,7 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(secondNodeToStop));
|
internalCluster().stopRandomNode(InternalTestCluster.nameFilter(secondNodeToStop));
|
||||||
|
|
||||||
final long desyncNanoTime = System.nanoTime();
|
final long desyncNanoTime = System.nanoTime();
|
||||||
|
//noinspection StatementWithEmptyBody
|
||||||
while (System.nanoTime() <= desyncNanoTime) {
|
while (System.nanoTime() <= desyncNanoTime) {
|
||||||
// time passes
|
// time passes
|
||||||
}
|
}
|
||||||
|
@ -991,6 +1017,231 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
assertThat(syncIds, hasSize(1));
|
assertThat(syncIds, hasSize(1));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testRecoveryUsingSyncedFlushWithoutRetentionLease() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put("index.number_of_shards", 1)
|
||||||
|
.put("index.number_of_replicas", 1)
|
||||||
|
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "24h") // do not reallocate the lost shard
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING.getKey(), "100ms") // expire leases quickly
|
||||||
|
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "100ms") // sync frequently
|
||||||
|
.build());
|
||||||
|
int numDocs = randomIntBetween(0, 10);
|
||||||
|
indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, numDocs)
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
final ShardId shardId = new ShardId(resolveIndex(indexName), 0);
|
||||||
|
assertThat(SyncedFlushUtil.attemptSyncedFlush(logger, internalCluster(), shardId).successfulShards(), equalTo(2));
|
||||||
|
|
||||||
|
final ClusterState clusterState = client().admin().cluster().prepareState().get().getState();
|
||||||
|
final ShardRouting shardToResync = randomFrom(clusterState.routingTable().shardRoutingTable(shardId).activeShards());
|
||||||
|
internalCluster().restartNode(clusterState.nodes().get(shardToResync.currentNodeId()).getName(),
|
||||||
|
new InternalTestCluster.RestartCallback() {
|
||||||
|
@Override
|
||||||
|
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||||
|
assertBusy(() -> assertFalse(client().admin().indices().prepareStats(indexName).get()
|
||||||
|
.getShards()[0].getRetentionLeaseStats().retentionLeases().contains(
|
||||||
|
ReplicationTracker.getPeerRecoveryRetentionLeaseId(shardToResync))));
|
||||||
|
return super.onNodeStopped(nodeName);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
ensureGreen(indexName);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testRecoverLocallyUpToGlobalCheckpoint() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
List<String> nodes = randomSubsetOf(2, StreamSupport.stream(clusterService().state().nodes().getDataNodes().spliterator(), false)
|
||||||
|
.map(node -> node.value.getName()).collect(Collectors.toSet()));
|
||||||
|
String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put("index.number_of_shards", 1)
|
||||||
|
.put("index.number_of_replicas", 1)
|
||||||
|
// disable global checkpoint background sync so we can verify the start recovery request
|
||||||
|
.put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "12h")
|
||||||
|
.put("index.routing.allocation.include._name", String.join(",", nodes))
|
||||||
|
.build());
|
||||||
|
ensureGreen(indexName);
|
||||||
|
int numDocs = randomIntBetween(0, 100);
|
||||||
|
indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, numDocs)
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
client().admin().indices().prepareRefresh(indexName).get(); // avoid refresh when we are failing a shard
|
||||||
|
String failingNode = randomFrom(nodes);
|
||||||
|
PlainActionFuture<StartRecoveryRequest> startRecoveryRequestFuture = new PlainActionFuture<>();
|
||||||
|
for (String node : nodes) {
|
||||||
|
MockTransportService transportService = (MockTransportService) internalCluster().getInstance(TransportService.class, node);
|
||||||
|
transportService.addSendBehavior((connection, requestId, action, request, options) -> {
|
||||||
|
if (action.equals(PeerRecoverySourceService.Actions.START_RECOVERY)) {
|
||||||
|
startRecoveryRequestFuture.onResponse((StartRecoveryRequest) request);
|
||||||
|
}
|
||||||
|
connection.sendRequest(requestId, action, request, options);
|
||||||
|
});
|
||||||
|
}
|
||||||
|
IndexShard shard = internalCluster().getInstance(IndicesService.class, failingNode)
|
||||||
|
.getShardOrNull(new ShardId(resolveIndex(indexName), 0));
|
||||||
|
final long lastSyncedGlobalCheckpoint = shard.getLastSyncedGlobalCheckpoint();
|
||||||
|
final long localCheckpointOfSafeCommit;
|
||||||
|
try(Engine.IndexCommitRef safeCommitRef = shard.acquireSafeIndexCommit()){
|
||||||
|
localCheckpointOfSafeCommit =
|
||||||
|
SequenceNumbers.loadSeqNoInfoFromLuceneCommit(safeCommitRef.getIndexCommit().getUserData().entrySet()).localCheckpoint;
|
||||||
|
}
|
||||||
|
final long maxSeqNo = shard.seqNoStats().getMaxSeqNo();
|
||||||
|
shard.failShard("test", new IOException("simulated"));
|
||||||
|
StartRecoveryRequest startRecoveryRequest = startRecoveryRequestFuture.actionGet();
|
||||||
|
SequenceNumbers.CommitInfo commitInfoAfterLocalRecovery = SequenceNumbers.loadSeqNoInfoFromLuceneCommit(
|
||||||
|
startRecoveryRequest.metadataSnapshot().getCommitUserData().entrySet());
|
||||||
|
assertThat(commitInfoAfterLocalRecovery.localCheckpoint, equalTo(lastSyncedGlobalCheckpoint));
|
||||||
|
assertThat(commitInfoAfterLocalRecovery.maxSeqNo, equalTo(lastSyncedGlobalCheckpoint));
|
||||||
|
assertThat(startRecoveryRequest.startingSeqNo(), equalTo(lastSyncedGlobalCheckpoint + 1));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
for (RecoveryState recoveryState : client().admin().indices().prepareRecoveries().get().shardRecoveryStates().get(indexName)) {
|
||||||
|
if (startRecoveryRequest.targetNode().equals(recoveryState.getTargetNode())) {
|
||||||
|
assertThat("total recovered translog operations must include both local and remote recovery",
|
||||||
|
recoveryState.getTranslog().recoveredOperations(),
|
||||||
|
greaterThanOrEqualTo(Math.toIntExact(maxSeqNo - localCheckpointOfSafeCommit)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
for (String node : nodes) {
|
||||||
|
MockTransportService transportService = (MockTransportService) internalCluster().getInstance(TransportService.class, node);
|
||||||
|
transportService.clearAllRules();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUsesFileBasedRecoveryIfRetentionLeaseMissing() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
|
||||||
|
String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)
|
||||||
|
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "12h")
|
||||||
|
.build());
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, between(0, 100))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
final ShardId shardId = new ShardId(resolveIndex(indexName), 0);
|
||||||
|
final DiscoveryNodes discoveryNodes = clusterService().state().nodes();
|
||||||
|
final IndexShardRoutingTable indexShardRoutingTable = clusterService().state().routingTable().shardRoutingTable(shardId);
|
||||||
|
|
||||||
|
final IndexShard primary = internalCluster().getInstance(IndicesService.class,
|
||||||
|
discoveryNodes.get(indexShardRoutingTable.primaryShard().currentNodeId()).getName()).getShardOrNull(shardId);
|
||||||
|
|
||||||
|
final ShardRouting replicaShardRouting = indexShardRoutingTable.replicaShards().get(0);
|
||||||
|
internalCluster().restartNode(discoveryNodes.get(replicaShardRouting.currentNodeId()).getName(),
|
||||||
|
new InternalTestCluster.RestartCallback() {
|
||||||
|
@Override
|
||||||
|
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||||
|
assertFalse(client().admin().cluster().prepareHealth()
|
||||||
|
.setWaitForNodes(Integer.toString(discoveryNodes.getSize() - 1))
|
||||||
|
.setWaitForEvents(Priority.LANGUID).get().isTimedOut());
|
||||||
|
|
||||||
|
final PlainActionFuture<ReplicationResponse> future = new PlainActionFuture<>();
|
||||||
|
primary.removeRetentionLease(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replicaShardRouting), future);
|
||||||
|
future.get();
|
||||||
|
|
||||||
|
return super.onNodeStopped(nodeName);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
//noinspection OptionalGetWithoutIsPresent because it fails the test if absent
|
||||||
|
final RecoveryState recoveryState = client().admin().indices().prepareRecoveries(indexName).get()
|
||||||
|
.shardRecoveryStates().get(indexName).stream().filter(rs -> rs.getPrimary() == false).findFirst().get();
|
||||||
|
assertThat(recoveryState.getIndex().totalFileCount(), greaterThan(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testUsesFileBasedRecoveryIfRetentionLeaseAheadOfGlobalCheckpoint() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
|
||||||
|
String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)
|
||||||
|
.put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "12h")
|
||||||
|
.build());
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, between(0, 100))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
final ShardId shardId = new ShardId(resolveIndex(indexName), 0);
|
||||||
|
final DiscoveryNodes discoveryNodes = clusterService().state().nodes();
|
||||||
|
final IndexShardRoutingTable indexShardRoutingTable = clusterService().state().routingTable().shardRoutingTable(shardId);
|
||||||
|
|
||||||
|
final IndexShard primary = internalCluster().getInstance(IndicesService.class,
|
||||||
|
discoveryNodes.get(indexShardRoutingTable.primaryShard().currentNodeId()).getName()).getShardOrNull(shardId);
|
||||||
|
|
||||||
|
final ShardRouting replicaShardRouting = indexShardRoutingTable.replicaShards().get(0);
|
||||||
|
internalCluster().restartNode(discoveryNodes.get(replicaShardRouting.currentNodeId()).getName(),
|
||||||
|
new InternalTestCluster.RestartCallback() {
|
||||||
|
@Override
|
||||||
|
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||||
|
assertFalse(client().admin().cluster().prepareHealth()
|
||||||
|
.setWaitForNodes(Integer.toString(discoveryNodes.getSize() - 1))
|
||||||
|
.setWaitForEvents(Priority.LANGUID).get().isTimedOut());
|
||||||
|
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, between(1, 100))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
|
||||||
|
// We do not guarantee that the replica can recover locally all the way to its own global checkpoint before starting
|
||||||
|
// to recover from the primary, so we must be careful not to perform an operations-based recovery if this would require
|
||||||
|
// some operations that are not being retained. Emulate this by advancing the lease ahead of the replica's GCP:
|
||||||
|
primary.renewRetentionLease(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replicaShardRouting),
|
||||||
|
primary.seqNoStats().getMaxSeqNo() + 1, ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE);
|
||||||
|
|
||||||
|
return super.onNodeStopped(nodeName);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
//noinspection OptionalGetWithoutIsPresent because it fails the test if absent
|
||||||
|
final RecoveryState recoveryState = client().admin().indices().prepareRecoveries(indexName).get()
|
||||||
|
.shardRecoveryStates().get(indexName).stream().filter(rs -> rs.getPrimary() == false).findFirst().get();
|
||||||
|
assertThat(recoveryState.getIndex().totalFileCount(), greaterThan(0));
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testDoesNotCopyOperationsInSafeCommit() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
|
||||||
|
String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true).build());
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, between(0, 100))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
|
||||||
|
final ShardId shardId = new ShardId(resolveIndex(indexName), 0);
|
||||||
|
final DiscoveryNodes discoveryNodes = clusterService().state().nodes();
|
||||||
|
final IndexShardRoutingTable indexShardRoutingTable = clusterService().state().routingTable().shardRoutingTable(shardId);
|
||||||
|
|
||||||
|
final IndexShard primary = internalCluster().getInstance(IndicesService.class,
|
||||||
|
discoveryNodes.get(indexShardRoutingTable.primaryShard().currentNodeId()).getName()).getShardOrNull(shardId);
|
||||||
|
final long maxSeqNoBeforeRecovery = primary.seqNoStats().getMaxSeqNo();
|
||||||
|
assertBusy(() -> assertThat(primary.getLastSyncedGlobalCheckpoint(), equalTo(maxSeqNoBeforeRecovery)));
|
||||||
|
assertThat(client().admin().indices().prepareFlush(indexName).get().getFailedShards(), is(0)); // makes a safe commit
|
||||||
|
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, between(0, 100))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
|
||||||
|
assertAcked(client().admin().indices().prepareUpdateSettings(indexName)
|
||||||
|
.setSettings(Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
final long maxSeqNoAfterRecovery = primary.seqNoStats().getMaxSeqNo();
|
||||||
|
|
||||||
|
//noinspection OptionalGetWithoutIsPresent because it fails the test if absent
|
||||||
|
final RecoveryState recoveryState = client().admin().indices().prepareRecoveries(indexName).get()
|
||||||
|
.shardRecoveryStates().get(indexName).stream().filter(rs -> rs.getPrimary() == false).findFirst().get();
|
||||||
|
assertThat((long)recoveryState.getTranslog().recoveredOperations(),
|
||||||
|
lessThanOrEqualTo(maxSeqNoAfterRecovery - maxSeqNoBeforeRecovery));
|
||||||
|
}
|
||||||
|
|
||||||
public static final class TestAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
public static final class TestAnalysisPlugin extends Plugin implements AnalysisPlugin {
|
||||||
final AtomicBoolean throwParsingError = new AtomicBoolean();
|
final AtomicBoolean throwParsingError = new AtomicBoolean();
|
||||||
@Override
|
@Override
|
||||||
|
@ -1008,4 +1259,71 @@ public class IndexRecoveryIT extends ESIntegTestCase {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testRepeatedRecovery() throws Exception {
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
|
||||||
|
// Ensures that you can remove a replica and then add it back again without any ill effects, even if it's allocated back to the
|
||||||
|
// node that held it previously, in case that node hasn't completely cleared it up.
|
||||||
|
|
||||||
|
final String indexName = "test-index";
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 6))
|
||||||
|
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "200ms")
|
||||||
|
.build());
|
||||||
|
indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, randomIntBetween(0, 10))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
|
||||||
|
assertThat(client().admin().indices().prepareFlush(indexName).get().getFailedShards(), equalTo(0));
|
||||||
|
|
||||||
|
assertBusy(() -> {
|
||||||
|
final ShardStats[] shardsStats = client().admin().indices().prepareStats(indexName).get().getIndex(indexName).getShards();
|
||||||
|
for (final ShardStats shardStats : shardsStats) {
|
||||||
|
final long maxSeqNo = shardStats.getSeqNoStats().getMaxSeqNo();
|
||||||
|
assertTrue(shardStats.getRetentionLeaseStats().retentionLeases().leases().stream()
|
||||||
|
.allMatch(retentionLease -> retentionLease.retainingSequenceNumber() == maxSeqNo + 1));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
logger.info("--> remove replicas");
|
||||||
|
assertAcked(client().admin().indices().prepareUpdateSettings(indexName)
|
||||||
|
.setSettings(Settings.builder().put("index.number_of_replicas", 0)));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
|
||||||
|
logger.info("--> index more documents");
|
||||||
|
indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, randomIntBetween(0, 10))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
|
||||||
|
logger.info("--> add replicas again");
|
||||||
|
assertAcked(client().admin().indices().prepareUpdateSettings(indexName)
|
||||||
|
.setSettings(Settings.builder().put("index.number_of_replicas", 1)));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testAllocateEmptyPrimaryResetsGlobalCheckpoint() throws Exception {
|
||||||
|
internalCluster().startMasterOnlyNode(Settings.EMPTY);
|
||||||
|
final List<String> dataNodes = internalCluster().startDataOnlyNodes(2);
|
||||||
|
final Settings randomNodeDataPathSettings = internalCluster().dataPathSettings(randomFrom(dataNodes));
|
||||||
|
final String indexName = "test";
|
||||||
|
assertAcked(client().admin().indices().prepareCreate(indexName).setSettings(Settings.builder()
|
||||||
|
.put("index.number_of_shards", 1).put("index.number_of_replicas", 1)
|
||||||
|
.put(MockEngineSupport.DISABLE_FLUSH_ON_CLOSE.getKey(), randomBoolean())).get());
|
||||||
|
final List<IndexRequestBuilder> indexRequests = IntStream.range(0, between(10, 500))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "type").setSource("foo", "bar"))
|
||||||
|
.collect(Collectors.toList());
|
||||||
|
indexRandom(randomBoolean(), true, true, indexRequests);
|
||||||
|
ensureGreen();
|
||||||
|
internalCluster().stopRandomDataNode();
|
||||||
|
internalCluster().stopRandomDataNode();
|
||||||
|
final String nodeWithoutData = internalCluster().startDataOnlyNode();
|
||||||
|
assertAcked(client().admin().cluster().prepareReroute()
|
||||||
|
.add(new AllocateEmptyPrimaryAllocationCommand(indexName, 0, nodeWithoutData, true)).get());
|
||||||
|
internalCluster().startDataOnlyNode(randomNodeDataPathSettings);
|
||||||
|
ensureGreen();
|
||||||
|
for (ShardStats shardStats : client().admin().indices().prepareStats(indexName).get().getIndex(indexName).getShards()) {
|
||||||
|
assertThat(shardStats.getSeqNoStats().getMaxSeqNo(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||||
|
assertThat(shardStats.getSeqNoStats().getLocalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||||
|
assertThat(shardStats.getSeqNoStats().getGlobalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,21 +19,25 @@
|
||||||
|
|
||||||
package org.elasticsearch.indices.recovery;
|
package org.elasticsearch.indices.recovery;
|
||||||
|
|
||||||
import org.apache.lucene.index.DirectoryReader;
|
|
||||||
import org.apache.lucene.index.IndexCommit;
|
|
||||||
import org.apache.lucene.index.IndexWriter;
|
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
|
||||||
import org.apache.lucene.index.NoMergePolicy;
|
|
||||||
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.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;
|
||||||
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
import org.elasticsearch.action.support.PlainActionFuture;
|
import org.elasticsearch.action.support.PlainActionFuture;
|
||||||
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
|
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||||
import org.elasticsearch.common.Randomness;
|
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.lucene.Lucene;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.common.xcontent.XContentType;
|
||||||
|
import org.elasticsearch.index.engine.NoOpEngine;
|
||||||
|
import org.elasticsearch.index.mapper.SourceToParse;
|
||||||
|
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;
|
||||||
|
@ -41,93 +45,23 @@ 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 java.io.IOException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
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.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.LongStream;
|
||||||
|
|
||||||
|
import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||||
import static org.hamcrest.Matchers.empty;
|
import static org.hamcrest.Matchers.empty;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
|
|
||||||
public void testGetStartingSeqNo() throws Exception {
|
|
||||||
final IndexShard replica = newShard(false);
|
|
||||||
try {
|
|
||||||
// Empty store
|
|
||||||
{
|
|
||||||
recoveryEmptyReplica(replica, true);
|
|
||||||
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null);
|
|
||||||
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(0L));
|
|
||||||
recoveryTarget.decRef();
|
|
||||||
}
|
|
||||||
// Last commit is good - use it.
|
|
||||||
final long initDocs = scaledRandomIntBetween(1, 10);
|
|
||||||
{
|
|
||||||
for (int i = 0; i < initDocs; i++) {
|
|
||||||
indexDoc(replica, "_doc", Integer.toString(i));
|
|
||||||
if (randomBoolean()) {
|
|
||||||
flushShard(replica);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
flushShard(replica);
|
|
||||||
replica.updateGlobalCheckpointOnReplica(initDocs - 1, "test");
|
|
||||||
replica.sync();
|
|
||||||
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null);
|
|
||||||
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(initDocs));
|
|
||||||
recoveryTarget.decRef();
|
|
||||||
}
|
|
||||||
// Global checkpoint does not advance, last commit is not good - use the previous commit
|
|
||||||
final int moreDocs = randomIntBetween(1, 10);
|
|
||||||
{
|
|
||||||
for (int i = 0; i < moreDocs; i++) {
|
|
||||||
indexDoc(replica, "_doc", Long.toString(i));
|
|
||||||
if (randomBoolean()) {
|
|
||||||
flushShard(replica);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
flushShard(replica);
|
|
||||||
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null);
|
|
||||||
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(initDocs));
|
|
||||||
recoveryTarget.decRef();
|
|
||||||
}
|
|
||||||
// Advances the global checkpoint, a safe commit also advances
|
|
||||||
{
|
|
||||||
replica.updateGlobalCheckpointOnReplica(initDocs + moreDocs - 1, "test");
|
|
||||||
replica.sync();
|
|
||||||
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null);
|
|
||||||
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(initDocs + moreDocs));
|
|
||||||
recoveryTarget.decRef();
|
|
||||||
}
|
|
||||||
// Different translogUUID, fallback to file-based
|
|
||||||
{
|
|
||||||
replica.close("test", false);
|
|
||||||
final List<IndexCommit> commits = DirectoryReader.listCommits(replica.store().directory());
|
|
||||||
IndexWriterConfig iwc = new IndexWriterConfig(null)
|
|
||||||
.setSoftDeletesField(Lucene.SOFT_DELETES_FIELD)
|
|
||||||
.setCommitOnClose(false)
|
|
||||||
.setMergePolicy(NoMergePolicy.INSTANCE)
|
|
||||||
.setOpenMode(IndexWriterConfig.OpenMode.APPEND);
|
|
||||||
try (IndexWriter writer = new IndexWriter(replica.store().directory(), iwc)) {
|
|
||||||
final Map<String, String> userData = new HashMap<>(commits.get(commits.size() - 1).getUserData());
|
|
||||||
userData.put(Translog.TRANSLOG_UUID_KEY, UUIDs.randomBase64UUID());
|
|
||||||
writer.setLiveCommitData(userData.entrySet());
|
|
||||||
writer.commit();
|
|
||||||
}
|
|
||||||
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null);
|
|
||||||
assertThat(PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO));
|
|
||||||
recoveryTarget.decRef();
|
|
||||||
}
|
|
||||||
} finally {
|
|
||||||
closeShards(replica);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
public void testWriteFileChunksConcurrently() throws Exception {
|
public void testWriteFileChunksConcurrently() throws Exception {
|
||||||
IndexShard sourceShard = newStartedShard(true);
|
IndexShard sourceShard = newStartedShard(true);
|
||||||
int numDocs = between(20, 100);
|
int numDocs = between(20, 100);
|
||||||
|
@ -202,4 +136,146 @@ public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
|
||||||
assertThat(diff.different, empty());
|
assertThat(diff.different, empty());
|
||||||
closeShards(sourceShard, targetShard);
|
closeShards(sourceShard, targetShard);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private SeqNoStats populateRandomData(IndexShard shard) throws IOException {
|
||||||
|
List<Long> seqNos = LongStream.range(0, 100).boxed().collect(Collectors.toList());
|
||||||
|
Randomness.shuffle(seqNos);
|
||||||
|
for (long seqNo : seqNos) {
|
||||||
|
shard.applyIndexOperationOnReplica(seqNo, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, new SourceToParse(
|
||||||
|
shard.shardId().getIndexName(), "_doc", UUIDs.randomBase64UUID(), new BytesArray("{}"), XContentType.JSON));
|
||||||
|
if (randomInt(100) < 5) {
|
||||||
|
shard.flush(new FlushRequest().waitIfOngoing(true));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
shard.sync();
|
||||||
|
long globalCheckpoint = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, shard.getLocalCheckpoint());
|
||||||
|
shard.updateGlobalCheckpointOnReplica(globalCheckpoint, "test");
|
||||||
|
shard.sync();
|
||||||
|
return shard.seqNoStats();
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testPrepareIndexForPeerRecovery() throws Exception {
|
||||||
|
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(),
|
||||||
|
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
||||||
|
|
||||||
|
// empty copy
|
||||||
|
IndexShard shard = newShard(false);
|
||||||
|
shard.markAsRecovering("for testing", new RecoveryState(shard.routingEntry(), localNode, localNode));
|
||||||
|
shard.prepareForIndexRecovery();
|
||||||
|
assertThat(shard.recoverLocallyUpToGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
assertThat(shard.recoveryState().getTranslog().totalLocal(), equalTo(RecoveryState.Translog.UNKNOWN));
|
||||||
|
assertThat(shard.recoveryState().getTranslog().recoveredOperations(), equalTo(0));
|
||||||
|
assertThat(shard.getLastKnownGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
closeShards(shard);
|
||||||
|
|
||||||
|
// good copy
|
||||||
|
shard = newStartedShard(false);
|
||||||
|
long globalCheckpoint = populateRandomData(shard).getGlobalCheckpoint();
|
||||||
|
Optional<SequenceNumbers.CommitInfo> safeCommit = shard.store().findSafeIndexCommit(globalCheckpoint);
|
||||||
|
assertTrue(safeCommit.isPresent());
|
||||||
|
int expectedTotalLocal = 0;
|
||||||
|
try (Translog.Snapshot snapshot = getTranslog(shard).newSnapshotFromMinSeqNo(safeCommit.get().localCheckpoint + 1)) {
|
||||||
|
Translog.Operation op;
|
||||||
|
while ((op = snapshot.next()) != null) {
|
||||||
|
if (op.seqNo() <= globalCheckpoint) {
|
||||||
|
expectedTotalLocal++;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
IndexShard replica = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(),
|
||||||
|
RecoverySource.PeerRecoverySource.INSTANCE));
|
||||||
|
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
||||||
|
replica.prepareForIndexRecovery();
|
||||||
|
assertThat(replica.recoverLocallyUpToGlobalCheckpoint(), equalTo(globalCheckpoint + 1));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().totalLocal(), equalTo(expectedTotalLocal));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().recoveredOperations(), equalTo(expectedTotalLocal));
|
||||||
|
assertThat(replica.getLastKnownGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
closeShards(replica);
|
||||||
|
|
||||||
|
// corrupted copy
|
||||||
|
shard = newStartedShard(false);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
populateRandomData(shard);
|
||||||
|
}
|
||||||
|
shard.store().markStoreCorrupted(new IOException("test"));
|
||||||
|
replica = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(),
|
||||||
|
RecoverySource.PeerRecoverySource.INSTANCE));
|
||||||
|
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
||||||
|
replica.prepareForIndexRecovery();
|
||||||
|
assertThat(replica.recoverLocallyUpToGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().totalLocal(), equalTo(RecoveryState.Translog.UNKNOWN));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().recoveredOperations(), equalTo(0));
|
||||||
|
assertThat(replica.getLastKnownGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
closeShards(replica);
|
||||||
|
|
||||||
|
// copy with truncated translog
|
||||||
|
shard = newStartedShard(false);
|
||||||
|
globalCheckpoint = populateRandomData(shard).getGlobalCheckpoint();
|
||||||
|
replica = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(),
|
||||||
|
RecoverySource.PeerRecoverySource.INSTANCE));
|
||||||
|
String translogUUID = Translog.createEmptyTranslog(replica.shardPath().resolveTranslog(), globalCheckpoint,
|
||||||
|
replica.shardId(), replica.getPendingPrimaryTerm());
|
||||||
|
replica.store().associateIndexWithNewTranslog(translogUUID);
|
||||||
|
safeCommit = replica.store().findSafeIndexCommit(globalCheckpoint);
|
||||||
|
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
||||||
|
replica.prepareForIndexRecovery();
|
||||||
|
if (safeCommit.isPresent()) {
|
||||||
|
assertThat(replica.recoverLocallyUpToGlobalCheckpoint(), equalTo(safeCommit.get().localCheckpoint + 1));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().totalLocal(), equalTo(0));
|
||||||
|
} else {
|
||||||
|
assertThat(replica.recoverLocallyUpToGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().totalLocal(), equalTo(RecoveryState.Translog.UNKNOWN));
|
||||||
|
}
|
||||||
|
assertThat(replica.recoveryState().getTranslog().recoveredOperations(), equalTo(0));
|
||||||
|
assertThat(replica.getLastKnownGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
closeShards(replica);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testClosedIndexSkipsLocalRecovery() throws Exception {
|
||||||
|
DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(),
|
||||||
|
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
||||||
|
IndexShard shard = newStartedShard(false);
|
||||||
|
long globalCheckpoint = populateRandomData(shard).getGlobalCheckpoint();
|
||||||
|
Optional<SequenceNumbers.CommitInfo> safeCommit = shard.store().findSafeIndexCommit(globalCheckpoint);
|
||||||
|
assertTrue(safeCommit.isPresent());
|
||||||
|
final IndexMetaData indexMetaData;
|
||||||
|
if (randomBoolean()) {
|
||||||
|
indexMetaData = IndexMetaData.builder(shard.indexSettings().getIndexMetaData())
|
||||||
|
.settings(shard.indexSettings().getSettings())
|
||||||
|
.state(IndexMetaData.State.CLOSE).build();
|
||||||
|
} else {
|
||||||
|
indexMetaData = IndexMetaData.builder(shard.indexSettings().getIndexMetaData())
|
||||||
|
.settings(Settings.builder().put(shard.indexSettings().getSettings())
|
||||||
|
.put(IndexMetaData.SETTING_BLOCKS_WRITE, true)).build();
|
||||||
|
}
|
||||||
|
IndexShard replica = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(),
|
||||||
|
RecoverySource.PeerRecoverySource.INSTANCE), indexMetaData, NoOpEngine::new);
|
||||||
|
replica.markAsRecovering("for testing", new RecoveryState(replica.routingEntry(), localNode, localNode));
|
||||||
|
replica.prepareForIndexRecovery();
|
||||||
|
assertThat(replica.recoverLocallyUpToGlobalCheckpoint(), equalTo(safeCommit.get().localCheckpoint + 1));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().totalLocal(), equalTo(0));
|
||||||
|
assertThat(replica.recoveryState().getTranslog().recoveredOperations(), equalTo(0));
|
||||||
|
assertThat(replica.getLastKnownGlobalCheckpoint(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
closeShards(replica);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testResetStartingSeqNoIfLastCommitCorrupted() throws Exception {
|
||||||
|
IndexShard shard = newStartedShard(false);
|
||||||
|
populateRandomData(shard);
|
||||||
|
DiscoveryNode pNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(),
|
||||||
|
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
||||||
|
DiscoveryNode rNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(),
|
||||||
|
Collections.emptyMap(), Collections.emptySet(), Version.CURRENT);
|
||||||
|
shard = reinitShard(shard, ShardRoutingHelper.initWithSameId(shard.routingEntry(), RecoverySource.PeerRecoverySource.INSTANCE));
|
||||||
|
shard.markAsRecovering("peer recovery", new RecoveryState(shard.routingEntry(), pNode, rNode));
|
||||||
|
shard.prepareForIndexRecovery();
|
||||||
|
long startingSeqNo = shard.recoverLocallyUpToGlobalCheckpoint();
|
||||||
|
shard.store().markStoreCorrupted(new IOException("simulated"));
|
||||||
|
RecoveryTarget recoveryTarget = new RecoveryTarget(shard, null, null);
|
||||||
|
StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(logger, rNode, recoveryTarget, startingSeqNo);
|
||||||
|
assertThat(request.startingSeqNo(), equalTo(UNASSIGNED_SEQ_NO));
|
||||||
|
assertThat(request.metadataSnapshot().size(), equalTo(0));
|
||||||
|
recoveryTarget.decRef();
|
||||||
|
closeShards(shard);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -64,6 +64,7 @@ import org.elasticsearch.index.mapper.ParsedDocument;
|
||||||
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
|
import org.elasticsearch.index.mapper.SeqNoFieldMapper;
|
||||||
import org.elasticsearch.index.mapper.Uid;
|
import org.elasticsearch.index.mapper.Uid;
|
||||||
import org.elasticsearch.index.seqno.ReplicationTracker;
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeases;
|
||||||
import org.elasticsearch.index.seqno.SeqNoStats;
|
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||||
|
@ -78,6 +79,7 @@ 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;
|
||||||
import org.elasticsearch.test.IndexSettingsModule;
|
import org.elasticsearch.test.IndexSettingsModule;
|
||||||
|
import org.elasticsearch.test.VersionUtils;
|
||||||
import org.elasticsearch.threadpool.FixedExecutorBuilder;
|
import org.elasticsearch.threadpool.FixedExecutorBuilder;
|
||||||
import org.elasticsearch.threadpool.TestThreadPool;
|
import org.elasticsearch.threadpool.TestThreadPool;
|
||||||
import org.elasticsearch.threadpool.ThreadPool;
|
import org.elasticsearch.threadpool.ThreadPool;
|
||||||
|
@ -100,6 +102,7 @@ import java.util.concurrent.Executor;
|
||||||
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.concurrent.atomic.AtomicLong;
|
import java.util.concurrent.atomic.AtomicLong;
|
||||||
|
import java.util.function.Consumer;
|
||||||
import java.util.function.IntSupplier;
|
import java.util.function.IntSupplier;
|
||||||
import java.util.zip.CRC32;
|
import java.util.zip.CRC32;
|
||||||
|
|
||||||
|
@ -441,6 +444,18 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
((ActionListener<Releasable>)invocation.getArguments()[0]).onResponse(() -> {});
|
((ActionListener<Releasable>)invocation.getArguments()[0]).onResponse(() -> {});
|
||||||
return null;
|
return null;
|
||||||
}).when(shard).acquirePrimaryOperationPermit(any(), anyString(), anyObject());
|
}).when(shard).acquirePrimaryOperationPermit(any(), anyString(), anyObject());
|
||||||
|
|
||||||
|
final IndexMetaData.Builder indexMetaData = IndexMetaData.builder("test").settings(Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, between(0,5))
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, between(1,5))
|
||||||
|
.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), randomBoolean())
|
||||||
|
.put(IndexMetaData.SETTING_VERSION_CREATED, VersionUtils.randomVersion(random()))
|
||||||
|
.put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID(random())));
|
||||||
|
if (randomBoolean()) {
|
||||||
|
indexMetaData.state(IndexMetaData.State.CLOSE);
|
||||||
|
}
|
||||||
|
when(shard.indexSettings()).thenReturn(new IndexSettings(indexMetaData.build(), Settings.EMPTY));
|
||||||
|
|
||||||
final AtomicBoolean phase1Called = new AtomicBoolean();
|
final AtomicBoolean phase1Called = new AtomicBoolean();
|
||||||
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean();
|
||||||
final AtomicBoolean phase2Called = new AtomicBoolean();
|
final AtomicBoolean phase2Called = new AtomicBoolean();
|
||||||
|
@ -453,9 +468,10 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
between(1, 8)) {
|
between(1, 8)) {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
void phase1(IndexCommit snapshot, Consumer<ActionListener<RetentionLease>> createRetentionLease,
|
||||||
|
IntSupplier translogOps, ActionListener<SendFileResult> listener) {
|
||||||
phase1Called.set(true);
|
phase1Called.set(true);
|
||||||
super.phase1(snapshot, globalCheckpoint, translogOps, listener);
|
super.phase1(snapshot, createRetentionLease, translogOps, listener);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -670,7 +686,9 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
||||||
final StepListener<RecoverySourceHandler.SendFileResult> phase1Listener = new StepListener<>();
|
final StepListener<RecoverySourceHandler.SendFileResult> phase1Listener = new StepListener<>();
|
||||||
try {
|
try {
|
||||||
final CountDownLatch latch = new CountDownLatch(1);
|
final CountDownLatch latch = new CountDownLatch(1);
|
||||||
handler.phase1(DirectoryReader.listCommits(dir).get(0), randomNonNegativeLong(), () -> 0,
|
handler.phase1(DirectoryReader.listCommits(dir).get(0),
|
||||||
|
l -> recoveryExecutor.execute(() -> l.onResponse(null)),
|
||||||
|
() -> 0,
|
||||||
new LatchedActionListener<>(phase1Listener, latch));
|
new LatchedActionListener<>(phase1Listener, latch));
|
||||||
latch.await();
|
latch.await();
|
||||||
phase1Listener.result();
|
phase1Listener.result();
|
||||||
|
|
|
@ -79,7 +79,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
||||||
shards.addReplica();
|
shards.addReplica();
|
||||||
shards.startAll();
|
shards.startAll();
|
||||||
final IndexShard replica = shards.getReplicas().get(0);
|
final IndexShard replica = shards.getReplicas().get(0);
|
||||||
assertThat(getTranslog(replica).totalOperations(), equalTo(docs + moreDocs));
|
boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled();
|
||||||
|
assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? moreDocs : docs + moreDocs));
|
||||||
shards.assertAllEqual(docs + moreDocs);
|
shards.assertAllEqual(docs + moreDocs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -294,7 +295,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
||||||
shards.recoverReplica(newReplica);
|
shards.recoverReplica(newReplica);
|
||||||
// file based recovery should be made
|
// file based recovery should be made
|
||||||
assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
|
||||||
assertThat(getTranslog(newReplica).totalOperations(), equalTo(numDocs));
|
boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled();
|
||||||
|
assertThat(getTranslog(newReplica).totalOperations(), equalTo(softDeletesEnabled ? nonFlushedDocs : numDocs));
|
||||||
|
|
||||||
// history uuid was restored
|
// history uuid was restored
|
||||||
assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID));
|
assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID));
|
||||||
|
@ -410,7 +412,8 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
||||||
shards.recoverReplica(replica);
|
shards.recoverReplica(replica);
|
||||||
// Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false)
|
// Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is false)
|
||||||
assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false)));
|
assertBusy(() -> assertThat(getEngine(replica).shouldPeriodicallyFlush(), equalTo(false)));
|
||||||
assertThat(getTranslog(replica).totalOperations(), equalTo(numDocs));
|
boolean softDeletesEnabled = replica.indexSettings().isSoftDeleteEnabled();
|
||||||
|
assertThat(getTranslog(replica).totalOperations(), equalTo(softDeletesEnabled ? 0 : numDocs));
|
||||||
shards.assertAllEqual(numDocs);
|
shards.assertAllEqual(numDocs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -1008,7 +1008,10 @@ public class IndexStatsIT extends ESIntegTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
public void testFilterCacheStats() throws Exception {
|
public void testFilterCacheStats() throws Exception {
|
||||||
Settings settings = Settings.builder().put(indexSettings()).put("number_of_replicas", 0).build();
|
Settings settings = Settings.builder().put(indexSettings())
|
||||||
|
.put("number_of_replicas", 0)
|
||||||
|
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "200ms")
|
||||||
|
.build();
|
||||||
assertAcked(prepareCreate("index").setSettings(settings).get());
|
assertAcked(prepareCreate("index").setSettings(settings).get());
|
||||||
indexRandom(false, true,
|
indexRandom(false, true,
|
||||||
client().prepareIndex("index", "type", "1").setSource("foo", "bar"),
|
client().prepareIndex("index", "type", "1").setSource("foo", "bar"),
|
||||||
|
@ -1052,6 +1055,13 @@ public class IndexStatsIT extends ESIntegTestCase {
|
||||||
// we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything.
|
// we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything.
|
||||||
if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) {
|
if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) {
|
||||||
persistGlobalCheckpoint("index");
|
persistGlobalCheckpoint("index");
|
||||||
|
assertBusy(() -> {
|
||||||
|
for (final ShardStats shardStats : client().admin().indices().prepareStats("index").get().getIndex("index").getShards()) {
|
||||||
|
final long maxSeqNo = shardStats.getSeqNoStats().getMaxSeqNo();
|
||||||
|
assertTrue(shardStats.getRetentionLeaseStats().retentionLeases().leases().stream()
|
||||||
|
.allMatch(retentionLease -> retentionLease.retainingSequenceNumber() == maxSeqNo + 1));
|
||||||
|
}
|
||||||
|
});
|
||||||
flush("index");
|
flush("index");
|
||||||
}
|
}
|
||||||
ForceMergeResponse forceMergeResponse =
|
ForceMergeResponse forceMergeResponse =
|
||||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.elasticsearch.Version;
|
import org.elasticsearch.Version;
|
||||||
import org.elasticsearch.action.index.IndexRequest;
|
import org.elasticsearch.action.index.IndexRequest;
|
||||||
|
import org.elasticsearch.action.support.replication.ReplicationResponse;
|
||||||
import org.elasticsearch.cluster.ClusterModule;
|
import org.elasticsearch.cluster.ClusterModule;
|
||||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.AllocationId;
|
import org.elasticsearch.cluster.routing.AllocationId;
|
||||||
|
@ -673,7 +674,7 @@ public abstract class EngineTestCase extends ESTestCase {
|
||||||
SequenceNumbers.NO_OPS_PERFORMED,
|
SequenceNumbers.NO_OPS_PERFORMED,
|
||||||
update -> {},
|
update -> {},
|
||||||
() -> 0L,
|
() -> 0L,
|
||||||
(leases, listener) -> {});
|
(leases, listener) -> listener.onResponse(new ReplicationResponse()));
|
||||||
globalCheckpointSupplier = replicationTracker;
|
globalCheckpointSupplier = replicationTracker;
|
||||||
retentionLeasesSupplier = replicationTracker::getRetentionLeases;
|
retentionLeasesSupplier = replicationTracker::getRetentionLeases;
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -169,7 +169,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
private final AtomicInteger replicaId = new AtomicInteger();
|
private final AtomicInteger replicaId = new AtomicInteger();
|
||||||
private final AtomicInteger docId = new AtomicInteger();
|
private final AtomicInteger docId = new AtomicInteger();
|
||||||
boolean closed = false;
|
boolean closed = false;
|
||||||
private ReplicationTargets replicationTargets;
|
private volatile ReplicationTargets replicationTargets;
|
||||||
|
|
||||||
private final PrimaryReplicaSyncer primaryReplicaSyncer = new PrimaryReplicaSyncer(
|
private final PrimaryReplicaSyncer primaryReplicaSyncer = new PrimaryReplicaSyncer(
|
||||||
new TaskManager(Settings.EMPTY, threadPool, Collections.emptySet()),
|
new TaskManager(Settings.EMPTY, threadPool, Collections.emptySet()),
|
||||||
|
@ -288,6 +288,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
|
|
||||||
public void startPrimary() throws IOException {
|
public void startPrimary() throws IOException {
|
||||||
recoverPrimary(primary);
|
recoverPrimary(primary);
|
||||||
|
computeReplicationTargets();
|
||||||
HashSet<String> activeIds = new HashSet<>();
|
HashSet<String> activeIds = new HashSet<>();
|
||||||
activeIds.addAll(activeIds());
|
activeIds.addAll(activeIds());
|
||||||
activeIds.add(primary.routingEntry().allocationId().getId());
|
activeIds.add(primary.routingEntry().allocationId().getId());
|
||||||
|
@ -528,7 +529,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
||||||
this.replicationTargets = new ReplicationTargets(this.primary, new ArrayList<>(this.replicas));
|
this.replicationTargets = new ReplicationTargets(this.primary, new ArrayList<>(this.replicas));
|
||||||
}
|
}
|
||||||
|
|
||||||
private synchronized ReplicationTargets getReplicationTargets() {
|
private ReplicationTargets getReplicationTargets() {
|
||||||
return replicationTargets;
|
return replicationTargets;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,99 @@
|
||||||
|
/*
|
||||||
|
* Licensed to Elasticsearch under one or more contributor
|
||||||
|
* license agreements. See the NOTICE file distributed with
|
||||||
|
* this work for additional information regarding copyright
|
||||||
|
* ownership. Elasticsearch licenses this file to you under
|
||||||
|
* the Apache License, Version 2.0 (the "License"); you may
|
||||||
|
* not use this file except in compliance with the License.
|
||||||
|
* You may obtain a copy of the License at
|
||||||
|
*
|
||||||
|
* http://www.apache.org/licenses/LICENSE-2.0
|
||||||
|
*
|
||||||
|
* Unless required by applicable law or agreed to in writing,
|
||||||
|
* software distributed under the License is distributed on an
|
||||||
|
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||||
|
* KIND, either express or implied. See the License for the
|
||||||
|
* specific language governing permissions and limitations
|
||||||
|
* under the License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.index.seqno;
|
||||||
|
|
||||||
|
import org.elasticsearch.client.Request;
|
||||||
|
import org.elasticsearch.client.RestClient;
|
||||||
|
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.cluster.routing.UnassignedInfo;
|
||||||
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
|
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
||||||
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.function.Function;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.hasItems;
|
||||||
|
|
||||||
|
public class RetentionLeaseUtils {
|
||||||
|
|
||||||
|
private RetentionLeaseUtils() {
|
||||||
|
// only static methods
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A utility method to convert a retention lease collection to a map from retention lease ID to retention lease and exclude
|
||||||
|
* the automatically-added peer-recovery retention leases
|
||||||
|
*
|
||||||
|
* @param retentionLeases the retention lease collection
|
||||||
|
* @return the map from retention lease ID to retention lease
|
||||||
|
*/
|
||||||
|
public static Map<String, RetentionLease> toMapExcludingPeerRecoveryRetentionLeases(final RetentionLeases retentionLeases) {
|
||||||
|
return retentionLeases.leases().stream()
|
||||||
|
.filter(l -> ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(l.source()) == false)
|
||||||
|
.collect(Collectors.toMap(RetentionLease::id, Function.identity(),
|
||||||
|
(o1, o2) -> {
|
||||||
|
throw new AssertionError("unexpectedly merging " + o1 + " and " + o2);
|
||||||
|
},
|
||||||
|
LinkedHashMap::new));
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Asserts that every copy of every shard of the given index has a peer recovery retention lease according to the stats exposed by the
|
||||||
|
* REST API
|
||||||
|
*/
|
||||||
|
public static void assertAllCopiesHavePeerRecoveryRetentionLeases(RestClient restClient, String index) throws IOException {
|
||||||
|
final Request statsRequest = new Request("GET", "/" + index + "/_stats");
|
||||||
|
statsRequest.addParameter("level", "shards");
|
||||||
|
final Map<?, ?> shardsStats = ObjectPath.createFromResponse(restClient.performRequest(statsRequest))
|
||||||
|
.evaluate("indices." + index + ".shards");
|
||||||
|
for (Map.Entry<?, ?> shardCopiesEntry : shardsStats.entrySet()) {
|
||||||
|
final List<?> shardCopiesList = (List<?>) shardCopiesEntry.getValue();
|
||||||
|
|
||||||
|
final Set<String> expectedLeaseIds = new HashSet<>();
|
||||||
|
for (Object shardCopyStats : shardCopiesList) {
|
||||||
|
final String nodeId
|
||||||
|
= Objects.requireNonNull((String) ((Map<?, ?>) (((Map<?, ?>) shardCopyStats).get("routing"))).get("node"));
|
||||||
|
expectedLeaseIds.add(ReplicationTracker.getPeerRecoveryRetentionLeaseId(
|
||||||
|
ShardRouting.newUnassigned(new ShardId("_na_", "test", 0), false, RecoverySource.PeerRecoverySource.INSTANCE,
|
||||||
|
new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "test")).initialize(nodeId, null, 0L)));
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<String> actualLeaseIds = new HashSet<>();
|
||||||
|
for (Object shardCopyStats : shardCopiesList) {
|
||||||
|
final List<?> leases
|
||||||
|
= (List<?>) ((Map<?, ?>) (((Map<?, ?>) shardCopyStats).get("retention_leases"))).get("leases");
|
||||||
|
for (Object lease : leases) {
|
||||||
|
actualLeaseIds.add(Objects.requireNonNull((String) (((Map<?, ?>) lease).get("id"))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Assert.assertThat("[" + index + "][" + shardCopiesEntry.getKey() + "] has leases " + actualLeaseIds
|
||||||
|
+ " but expected " + expectedLeaseIds,
|
||||||
|
actualLeaseIds, hasItems(expectedLeaseIds.toArray(new String[0])));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -224,7 +224,12 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException {
|
protected IndexShard newShard(ShardRouting shardRouting, final IndexingOperationListener... listeners) throws IOException {
|
||||||
return newShard(shardRouting, Settings.EMPTY, new InternalEngineFactory(), listeners);
|
return newShard(shardRouting, Settings.EMPTY, listeners);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected IndexShard newShard(ShardRouting shardRouting, final Settings settings, final IndexingOperationListener... listeners)
|
||||||
|
throws IOException {
|
||||||
|
return newShard(shardRouting, settings, new InternalEngineFactory(), listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -422,23 +427,24 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
* @param listeners new listerns to use for the newly created shard
|
* @param listeners new listerns to use for the newly created shard
|
||||||
*/
|
*/
|
||||||
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexingOperationListener... listeners) throws IOException {
|
||||||
return reinitShard(current, routing, current.engineFactory, listeners);
|
return reinitShard(current, routing, current.indexSettings.getIndexMetaData(), current.engineFactory, listeners);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Takes an existing shard, closes it and starts a new initialing shard at the same location
|
* Takes an existing shard, closes it and starts a new initialing shard at the same location
|
||||||
*
|
*
|
||||||
* @param routing the shard routing to use for the newly created shard.
|
* @param routing the shard routing to use for the newly created shard.
|
||||||
* @param listeners new listerns to use for the newly created shard
|
* @param listeners new listerns to use for the newly created shard
|
||||||
|
* @param indexMetaData the index metadata to use for the newly created shard
|
||||||
* @param engineFactory the engine factory for the new shard
|
* @param engineFactory the engine factory for the new shard
|
||||||
*/
|
*/
|
||||||
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, EngineFactory engineFactory,
|
protected IndexShard reinitShard(IndexShard current, ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory,
|
||||||
IndexingOperationListener... listeners) throws IOException {
|
IndexingOperationListener... listeners) throws IOException {
|
||||||
closeShards(current);
|
closeShards(current);
|
||||||
return newShard(
|
return newShard(
|
||||||
routing,
|
routing,
|
||||||
current.shardPath(),
|
current.shardPath(),
|
||||||
current.indexSettings().getIndexMetaData(),
|
indexMetaData,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
engineFactory,
|
engineFactory,
|
||||||
|
@ -623,18 +629,9 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
||||||
}
|
}
|
||||||
replica.prepareForIndexRecovery();
|
replica.prepareForIndexRecovery();
|
||||||
final RecoveryTarget recoveryTarget = targetSupplier.apply(replica, pNode);
|
final RecoveryTarget recoveryTarget = targetSupplier.apply(replica, pNode);
|
||||||
final String targetAllocationId = recoveryTarget.indexShard().routingEntry().allocationId().getId();
|
final long startingSeqNo = recoveryTarget.indexShard().recoverLocallyUpToGlobalCheckpoint();
|
||||||
|
final StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest(
|
||||||
final Store.MetadataSnapshot snapshot = getMetadataSnapshotOrEmpty(replica);
|
logger, rNode, recoveryTarget, startingSeqNo);
|
||||||
final long startingSeqNo;
|
|
||||||
if (snapshot.size() > 0) {
|
|
||||||
startingSeqNo = PeerRecoveryTargetService.getStartingSeqNo(logger, recoveryTarget);
|
|
||||||
} else {
|
|
||||||
startingSeqNo = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
|
||||||
}
|
|
||||||
|
|
||||||
final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId,
|
|
||||||
pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo);
|
|
||||||
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, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8));
|
request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8));
|
||||||
|
|
|
@ -50,6 +50,7 @@ public final class InternalSettingsPlugin extends Plugin {
|
||||||
PROVIDED_NAME_SETTING,
|
PROVIDED_NAME_SETTING,
|
||||||
TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING,
|
TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING,
|
||||||
IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING,
|
IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING,
|
||||||
|
IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING,
|
||||||
IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING
|
IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -14,6 +14,7 @@ import org.elasticsearch.common.settings.SecureString;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
import org.elasticsearch.common.util.concurrent.ThreadContext;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
import org.elasticsearch.test.rest.yaml.ObjectPath;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
@ -24,7 +25,6 @@ import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
||||||
import static org.hamcrest.Matchers.containsString;
|
import static org.hamcrest.Matchers.containsString;
|
||||||
import static org.hamcrest.Matchers.empty;
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
import static org.hamcrest.Matchers.hasSize;
|
import static org.hamcrest.Matchers.hasSize;
|
||||||
import static org.hamcrest.Matchers.is;
|
import static org.hamcrest.Matchers.is;
|
||||||
|
@ -228,7 +228,9 @@ public class FollowIndexSecurityIT extends ESCCRRestTestCase {
|
||||||
final Map<?, ?> shardStatsAsMap = (Map<?, ?>) shardsStats.get(0);
|
final Map<?, ?> shardStatsAsMap = (Map<?, ?>) shardsStats.get(0);
|
||||||
final Map<?, ?> retentionLeasesStats = (Map<?, ?>) shardStatsAsMap.get("retention_leases");
|
final Map<?, ?> retentionLeasesStats = (Map<?, ?>) shardStatsAsMap.get("retention_leases");
|
||||||
final List<?> leases = (List<?>) retentionLeasesStats.get("leases");
|
final List<?> leases = (List<?>) retentionLeasesStats.get("leases");
|
||||||
assertThat(leases, empty());
|
for (final Object lease : leases) {
|
||||||
|
assertThat(((Map<?, ?>) lease).get("source"), equalTo(ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,7 +36,7 @@ import org.elasticsearch.index.IndexSettings;
|
||||||
import org.elasticsearch.index.seqno.RetentionLease;
|
import org.elasticsearch.index.seqno.RetentionLease;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException;
|
import org.elasticsearch.index.seqno.RetentionLeaseNotFoundException;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeases;
|
import org.elasticsearch.index.seqno.RetentionLeaseUtils;
|
||||||
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.indices.IndicesService;
|
import org.elasticsearch.indices.IndicesService;
|
||||||
|
@ -97,20 +97,11 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static final class RetentionLeaseSyncIntervalSettingPlugin extends Plugin {
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<Setting<?>> getSettings() {
|
|
||||||
return Collections.singletonList(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
return Stream.concat(
|
return Stream.concat(
|
||||||
super.nodePlugins().stream(),
|
super.nodePlugins().stream(),
|
||||||
Stream.of(RetentionLeaseRenewIntervalSettingPlugin.class, RetentionLeaseSyncIntervalSettingPlugin.class))
|
Stream.of(RetentionLeaseRenewIntervalSettingPlugin.class))
|
||||||
.collect(Collectors.toList());
|
.collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -189,10 +180,10 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
final RetentionLease retentionLease =
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
currentRetentionLeases.leases().iterator().next();
|
final RetentionLease retentionLease = currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
@ -310,7 +301,7 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
*/
|
*/
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
// sample the leases after recovery
|
// sample the leases after recovery
|
||||||
final List<RetentionLeases> retentionLeases = new ArrayList<>();
|
final List< Map<String, RetentionLease>> retentionLeases = new ArrayList<>();
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
retentionLeases.clear();
|
retentionLeases.clear();
|
||||||
final IndicesStatsResponse stats =
|
final IndicesStatsResponse stats =
|
||||||
|
@ -320,13 +311,15 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
= RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final ClusterStateResponse followerIndexClusterState =
|
final ClusterStateResponse followerIndexClusterState =
|
||||||
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
||||||
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
final String expectedRetentionLeaseId = retentionLeaseId(
|
final String expectedRetentionLeaseId = retentionLeaseId(
|
||||||
getFollowerCluster().getClusterName(),
|
getFollowerCluster().getClusterName(),
|
||||||
new Index(followerIndex, followerUUID),
|
new Index(followerIndex, followerUUID),
|
||||||
|
@ -353,16 +346,17 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final ClusterStateResponse followerIndexClusterState =
|
final ClusterStateResponse followerIndexClusterState =
|
||||||
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
||||||
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID)));
|
||||||
// we assert that retention leases are being renewed by an increase in the timestamp
|
// we assert that retention leases are being renewed by an increase in the timestamp
|
||||||
assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).leases().iterator().next().timestamp()));
|
assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).values().iterator().next().timestamp()));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
|
|
||||||
|
@ -392,10 +386,10 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet();
|
leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet();
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (final ShardStats shardStats : shardsStats) {
|
for (final ShardStats shardStats : shardsStats) {
|
||||||
assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), hasSize(1));
|
final Map<String, RetentionLease> retentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(
|
shardStats.getRetentionLeaseStats().retentionLeases());
|
||||||
shardStats.getRetentionLeaseStats().retentionLeases().leases().iterator().next().id(),
|
assertThat(Strings.toString(shardStats), retentionLeases.values(), hasSize(1));
|
||||||
equalTo(retentionLeaseId));
|
assertThat(retentionLeases.values().iterator().next().id(), equalTo(retentionLeaseId));
|
||||||
}
|
}
|
||||||
|
|
||||||
// we will sometimes fake that some of the retention leases are already removed on the leader shard
|
// we will sometimes fake that some of the retention leases are already removed on the leader shard
|
||||||
|
@ -454,7 +448,8 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet();
|
leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet();
|
||||||
final List<ShardStats> afterUnfollowShardsStats = getShardsStats(afterUnfollowStats);
|
final List<ShardStats> afterUnfollowShardsStats = getShardsStats(afterUnfollowStats);
|
||||||
for (final ShardStats shardStats : afterUnfollowShardsStats) {
|
for (final ShardStats shardStats : afterUnfollowShardsStats) {
|
||||||
assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty());
|
assertThat(Strings.toString(shardStats), RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
|
shardStats.getRetentionLeaseStats().retentionLeases()).values(), empty());
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) {
|
for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) {
|
||||||
|
@ -605,10 +600,11 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
||||||
// we assert that retention leases are being advanced
|
// we assert that retention leases are being advanced
|
||||||
assertThat(
|
assertThat(
|
||||||
|
@ -665,7 +661,7 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
*/
|
*/
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
// sample the leases after pausing
|
// sample the leases after pausing
|
||||||
final List<RetentionLeases> retentionLeases = new ArrayList<>();
|
final List<Map<String, RetentionLease>> retentionLeases = new ArrayList<>();
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
retentionLeases.clear();
|
retentionLeases.clear();
|
||||||
final IndicesStatsResponse stats =
|
final IndicesStatsResponse stats =
|
||||||
|
@ -675,13 +671,15 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
= RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final ClusterStateResponse followerIndexClusterState =
|
final ClusterStateResponse followerIndexClusterState =
|
||||||
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
||||||
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
final String expectedRetentionLeaseId = retentionLeaseId(
|
final String expectedRetentionLeaseId = retentionLeaseId(
|
||||||
getFollowerCluster().getClusterName(),
|
getFollowerCluster().getClusterName(),
|
||||||
new Index(followerIndex, followerUUID),
|
new Index(followerIndex, followerUUID),
|
||||||
|
@ -708,16 +706,17 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final ClusterStateResponse followerIndexClusterState =
|
final ClusterStateResponse followerIndexClusterState =
|
||||||
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get();
|
||||||
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID();
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID)));
|
||||||
// we assert that retention leases are not being renewed by an unchanged timestamp
|
// we assert that retention leases are not being renewed by an unchanged timestamp
|
||||||
assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).leases().iterator().next().timestamp()));
|
assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).values().iterator().next().timestamp()));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
@ -924,7 +923,8 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> afterUnfollowShardsStats = getShardsStats(afterUnfollowStats);
|
final List<ShardStats> afterUnfollowShardsStats = getShardsStats(afterUnfollowStats);
|
||||||
for (final ShardStats shardStats : afterUnfollowShardsStats) {
|
for (final ShardStats shardStats : afterUnfollowShardsStats) {
|
||||||
assertNotNull(shardStats.getRetentionLeaseStats());
|
assertNotNull(shardStats.getRetentionLeaseStats());
|
||||||
assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty());
|
assertThat(Strings.toString(shardStats), RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
|
shardStats.getRetentionLeaseStats().retentionLeases()).values(), empty());
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) {
|
for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) {
|
||||||
|
@ -975,7 +975,8 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> afterForgetFollowerShardsStats = getShardsStats(afterForgetFollowerStats);
|
final List<ShardStats> afterForgetFollowerShardsStats = getShardsStats(afterForgetFollowerStats);
|
||||||
for (final ShardStats shardStats : afterForgetFollowerShardsStats) {
|
for (final ShardStats shardStats : afterForgetFollowerShardsStats) {
|
||||||
assertNotNull(shardStats.getRetentionLeaseStats());
|
assertNotNull(shardStats.getRetentionLeaseStats());
|
||||||
assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty());
|
assertThat(Strings.toString(shardStats), RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
|
shardStats.getRetentionLeaseStats().retentionLeases()).values(), empty());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -985,7 +986,7 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final String followerIndex,
|
final String followerIndex,
|
||||||
final String leaderIndex) throws Exception {
|
final String leaderIndex) throws Exception {
|
||||||
// ensure that a retention lease has been put in place on each shard, and grab a copy of them
|
// ensure that a retention lease has been put in place on each shard, and grab a copy of them
|
||||||
final List<RetentionLeases> retentionLeases = new ArrayList<>();
|
final List<Map<String, RetentionLease>> retentionLeases = new ArrayList<>();
|
||||||
assertBusy(() -> {
|
assertBusy(() -> {
|
||||||
retentionLeases.clear();
|
retentionLeases.clear();
|
||||||
final IndicesStatsResponse stats =
|
final IndicesStatsResponse stats =
|
||||||
|
@ -995,10 +996,11 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
||||||
retentionLeases.add(currentRetentionLeases);
|
retentionLeases.add(currentRetentionLeases);
|
||||||
}
|
}
|
||||||
|
@ -1013,13 +1015,14 @@ public class CcrRetentionLeaseIT extends CcrIntegTestCase {
|
||||||
final List<ShardStats> shardsStats = getShardsStats(stats);
|
final List<ShardStats> shardsStats = getShardsStats(stats);
|
||||||
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) {
|
||||||
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
assertNotNull(shardsStats.get(i).getRetentionLeaseStats());
|
||||||
final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases();
|
final Map<String, RetentionLease> currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(
|
||||||
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1));
|
shardsStats.get(i).getRetentionLeaseStats().retentionLeases());
|
||||||
|
assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1));
|
||||||
final RetentionLease retentionLease =
|
final RetentionLease retentionLease =
|
||||||
currentRetentionLeases.leases().iterator().next();
|
currentRetentionLeases.values().iterator().next();
|
||||||
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex)));
|
||||||
// we assert that retention leases are being renewed by an increase in the timestamp
|
// we assert that retention leases are being renewed by an increase in the timestamp
|
||||||
assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).leases().iterator().next().timestamp()));
|
assertThat(retentionLease.timestamp(), greaterThan(retentionLeases.get(i).values().iterator().next().timestamp()));
|
||||||
}
|
}
|
||||||
});
|
});
|
||||||
}
|
}
|
||||||
|
|
|
@ -67,7 +67,9 @@ import org.elasticsearch.common.xcontent.XContentType;
|
||||||
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
import org.elasticsearch.common.xcontent.support.XContentMapValues;
|
||||||
import org.elasticsearch.index.Index;
|
import org.elasticsearch.index.Index;
|
||||||
import org.elasticsearch.index.IndexNotFoundException;
|
import org.elasticsearch.index.IndexNotFoundException;
|
||||||
|
import org.elasticsearch.index.IndexService;
|
||||||
import org.elasticsearch.index.IndexSettings;
|
import org.elasticsearch.index.IndexSettings;
|
||||||
|
import org.elasticsearch.index.seqno.ReplicationTracker;
|
||||||
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
import org.elasticsearch.index.seqno.RetentionLeaseActions;
|
||||||
import org.elasticsearch.index.shard.ShardId;
|
import org.elasticsearch.index.shard.ShardId;
|
||||||
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
import org.elasticsearch.persistent.PersistentTasksCustomMetaData;
|
||||||
|
@ -1190,8 +1192,10 @@ public class IndexFollowingIT extends CcrIntegTestCase {
|
||||||
final CheckedRunnable<Exception> afterPausingFollower,
|
final CheckedRunnable<Exception> afterPausingFollower,
|
||||||
final Consumer<Collection<ResourceNotFoundException>> exceptionConsumer) throws Exception {
|
final Consumer<Collection<ResourceNotFoundException>> exceptionConsumer) throws Exception {
|
||||||
final int numberOfPrimaryShards = randomIntBetween(1, 3);
|
final int numberOfPrimaryShards = randomIntBetween(1, 3);
|
||||||
final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1),
|
final Map<String, String> extraSettingsMap = new HashMap<>(2);
|
||||||
singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"));
|
extraSettingsMap.put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true");
|
||||||
|
extraSettingsMap.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "200ms");
|
||||||
|
final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), extraSettingsMap);
|
||||||
assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON));
|
assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON));
|
||||||
ensureLeaderYellow("index1");
|
ensureLeaderYellow("index1");
|
||||||
|
|
||||||
|
@ -1224,6 +1228,15 @@ public class IndexFollowingIT extends CcrIntegTestCase {
|
||||||
leaderClient().prepareDelete("index1", "doc", "1").get();
|
leaderClient().prepareDelete("index1", "doc", "1").get();
|
||||||
leaderClient().admin().indices().refresh(new RefreshRequest("index1")).actionGet();
|
leaderClient().admin().indices().refresh(new RefreshRequest("index1")).actionGet();
|
||||||
leaderClient().admin().indices().flush(new FlushRequest("index1").force(true)).actionGet();
|
leaderClient().admin().indices().flush(new FlushRequest("index1").force(true)).actionGet();
|
||||||
|
assertBusy(() -> {
|
||||||
|
final ShardStats[] shardsStats = leaderClient().admin().indices().prepareStats("index1").get().getIndex("index1").getShards();
|
||||||
|
for (final ShardStats shardStats : shardsStats) {
|
||||||
|
final long maxSeqNo = shardStats.getSeqNoStats().getMaxSeqNo();
|
||||||
|
assertTrue(shardStats.getRetentionLeaseStats().retentionLeases().leases().stream()
|
||||||
|
.filter(retentionLease -> ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()))
|
||||||
|
.allMatch(retentionLease -> retentionLease.retainingSequenceNumber() == maxSeqNo + 1));
|
||||||
|
}
|
||||||
|
});
|
||||||
ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index1");
|
ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index1");
|
||||||
forceMergeRequest.maxNumSegments(1);
|
forceMergeRequest.maxNumSegments(1);
|
||||||
leaderClient().admin().indices().forceMerge(forceMergeRequest).actionGet();
|
leaderClient().admin().indices().forceMerge(forceMergeRequest).actionGet();
|
||||||
|
|
|
@ -15,17 +15,20 @@ import org.elasticsearch.action.admin.indices.refresh.RefreshRequest;
|
||||||
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||||
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.index.IndexService;
|
||||||
import org.elasticsearch.index.engine.Engine;
|
import org.elasticsearch.index.engine.Engine;
|
||||||
import org.elasticsearch.index.translog.Translog;
|
import org.elasticsearch.index.translog.Translog;
|
||||||
import org.elasticsearch.plugins.Plugin;
|
import org.elasticsearch.plugins.Plugin;
|
||||||
import org.elasticsearch.test.ESSingleNodeTestCase;
|
import org.elasticsearch.test.ESSingleNodeTestCase;
|
||||||
|
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||||
import org.elasticsearch.xpack.ccr.Ccr;
|
import org.elasticsearch.xpack.ccr.Ccr;
|
||||||
import org.elasticsearch.xpack.ccr.LocalStateCcr;
|
import org.elasticsearch.xpack.ccr.LocalStateCcr;
|
||||||
|
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
|
||||||
import java.util.concurrent.CountDownLatch;
|
import java.util.concurrent.CountDownLatch;
|
||||||
import java.util.concurrent.atomic.AtomicReference;
|
import java.util.concurrent.atomic.AtomicReference;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.Stream;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.contains;
|
import static org.hamcrest.Matchers.contains;
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
@ -35,7 +38,7 @@ public class ShardChangesTests extends ESSingleNodeTestCase {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected Collection<Class<? extends Plugin>> getPlugins() {
|
protected Collection<Class<? extends Plugin>> getPlugins() {
|
||||||
return Collections.singleton(LocalStateCcr.class);
|
return Stream.of(LocalStateCcr.class, InternalSettingsPlugin.class).collect(Collectors.toList());
|
||||||
}
|
}
|
||||||
|
|
||||||
// this emulates what the CCR persistent task will do for pulling
|
// this emulates what the CCR persistent task will do for pulling
|
||||||
|
@ -102,7 +105,8 @@ public class ShardChangesTests extends ESSingleNodeTestCase {
|
||||||
.put("index.soft_deletes.enabled", true)
|
.put("index.soft_deletes.enabled", true)
|
||||||
.put("index.soft_deletes.retention.operations", 0)
|
.put("index.soft_deletes.retention.operations", 0)
|
||||||
.put("index.number_of_shards", 1)
|
.put("index.number_of_shards", 1)
|
||||||
.put("index.number_of_replicas", 0))
|
.put("index.number_of_replicas", 0)
|
||||||
|
.put(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING.getKey(), "200ms"))
|
||||||
.get();
|
.get();
|
||||||
|
|
||||||
for (int i = 0; i < 32; i++) {
|
for (int i = 0; i < 32; i++) {
|
||||||
|
@ -111,6 +115,15 @@ public class ShardChangesTests extends ESSingleNodeTestCase {
|
||||||
client().admin().indices().flush(new FlushRequest("index").force(true)).actionGet();
|
client().admin().indices().flush(new FlushRequest("index").force(true)).actionGet();
|
||||||
}
|
}
|
||||||
client().admin().indices().refresh(new RefreshRequest("index")).actionGet();
|
client().admin().indices().refresh(new RefreshRequest("index")).actionGet();
|
||||||
|
assertBusy(() -> {
|
||||||
|
final ShardStats[] shardsStats = client().admin().indices().prepareStats("index").get().getIndex("index").getShards();
|
||||||
|
for (final ShardStats shardStats : shardsStats) {
|
||||||
|
final long maxSeqNo = shardStats.getSeqNoStats().getMaxSeqNo();
|
||||||
|
assertTrue(shardStats.getRetentionLeaseStats().retentionLeases().leases().stream()
|
||||||
|
.allMatch(retentionLease -> retentionLease.retainingSequenceNumber() == maxSeqNo + 1));
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index");
|
ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index");
|
||||||
forceMergeRequest.maxNumSegments(1);
|
forceMergeRequest.maxNumSegments(1);
|
||||||
client().admin().indices().forceMerge(forceMergeRequest).actionGet();
|
client().admin().indices().forceMerge(forceMergeRequest).actionGet();
|
||||||
|
|
|
@ -3,41 +3,99 @@
|
||||||
* or more contributor license agreements. Licensed under the Elastic License;
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
* you may not use this file except in compliance with the Elastic License.
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package org.elasticsearch.index.engine;
|
package org.elasticsearch.index.engine;
|
||||||
|
|
||||||
import org.elasticsearch.cluster.routing.RecoverySource;
|
import org.elasticsearch.client.Client;
|
||||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||||
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||||
import org.elasticsearch.common.settings.Settings;
|
import org.elasticsearch.common.settings.Settings;
|
||||||
import org.elasticsearch.index.shard.IndexShard;
|
import org.elasticsearch.common.util.set.Sets;
|
||||||
import org.elasticsearch.index.shard.IndexShardTestCase;
|
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||||
|
import org.elasticsearch.plugins.Plugin;
|
||||||
|
import org.elasticsearch.protocol.xpack.frozen.FreezeRequest;
|
||||||
|
import org.elasticsearch.test.ESIntegTestCase;
|
||||||
|
import org.elasticsearch.test.InternalTestCluster;
|
||||||
|
import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction;
|
||||||
|
import org.elasticsearch.xpack.frozen.FrozenIndices;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.stream.Collectors;
|
||||||
|
import java.util.stream.IntStream;
|
||||||
|
|
||||||
import static org.hamcrest.Matchers.equalTo;
|
import static java.util.stream.Collectors.toList;
|
||||||
|
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||||
|
import static org.hamcrest.Matchers.empty;
|
||||||
|
import static org.hamcrest.Matchers.not;
|
||||||
|
|
||||||
public class FrozenIndexRecoveryTests extends IndexShardTestCase {
|
public class FrozenIndexRecoveryTests extends ESIntegTestCase {
|
||||||
|
|
||||||
/**
|
@Override
|
||||||
* Make sure we can recover from a frozen engine
|
protected boolean addMockInternalEngine() {
|
||||||
*/
|
return false;
|
||||||
public void testRecoverFromFrozenPrimary() throws IOException {
|
}
|
||||||
IndexShard indexShard = newStartedShard(true);
|
|
||||||
indexDoc(indexShard, "_doc", "1");
|
|
||||||
indexDoc(indexShard, "_doc", "2");
|
|
||||||
indexDoc(indexShard, "_doc", "3");
|
|
||||||
indexShard.close("test", true);
|
|
||||||
final ShardRouting shardRouting = indexShard.routingEntry();
|
|
||||||
IndexShard frozenShard = reinitShard(indexShard, ShardRoutingHelper.initWithSameId(shardRouting,
|
|
||||||
shardRouting.primary() ? RecoverySource.ExistingStoreRecoverySource.INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE
|
|
||||||
), FrozenEngine::new);
|
|
||||||
recoverShardFromStore(frozenShard);
|
|
||||||
assertThat(frozenShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(frozenShard.seqNoStats().getMaxSeqNo()));
|
|
||||||
assertDocCount(frozenShard, 3);
|
|
||||||
|
|
||||||
IndexShard replica = newShard(false, Settings.EMPTY, FrozenEngine::new);
|
@Override
|
||||||
recoverReplica(replica, frozenShard, true);
|
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||||
assertDocCount(replica, 3);
|
List<Class<? extends Plugin>> plugins = new ArrayList<>(super.nodePlugins());
|
||||||
closeShards(frozenShard, replica);
|
plugins.add(FrozenIndices.class);
|
||||||
|
return plugins;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
|
||||||
|
List<Class<? extends Plugin>> plugins = new ArrayList<>(super.nodePlugins());
|
||||||
|
plugins.add(FrozenIndices.class);
|
||||||
|
return plugins;
|
||||||
|
}
|
||||||
|
|
||||||
|
public void testRecoverExistingReplica() throws Exception {
|
||||||
|
final String indexName = "test-recover-existing-replica";
|
||||||
|
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||||
|
List<String> dataNodes = randomSubsetOf(2, Sets.newHashSet(
|
||||||
|
clusterService().state().nodes().getDataNodes().valuesIt()).stream().map(DiscoveryNode::getName).collect(Collectors.toSet()));
|
||||||
|
createIndex(indexName, Settings.builder()
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
|
||||||
|
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
|
||||||
|
.put("index.routing.allocation.include._name", String.join(",", dataNodes))
|
||||||
|
.build());
|
||||||
|
indexRandom(randomBoolean(), randomBoolean(), randomBoolean(), IntStream.range(0, randomIntBetween(0, 50))
|
||||||
|
.mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)).collect(toList()));
|
||||||
|
ensureGreen(indexName);
|
||||||
|
if (randomBoolean()) {
|
||||||
|
client().admin().indices().prepareFlush(indexName).get();
|
||||||
|
} else {
|
||||||
|
client().admin().indices().prepareSyncedFlush(indexName).get();
|
||||||
|
}
|
||||||
|
// index more documents while one shard copy is offline
|
||||||
|
internalCluster().restartNode(dataNodes.get(1), new InternalTestCluster.RestartCallback() {
|
||||||
|
@Override
|
||||||
|
public Settings onNodeStopped(String nodeName) throws Exception {
|
||||||
|
Client client = client(dataNodes.get(0));
|
||||||
|
int moreDocs = randomIntBetween(1, 50);
|
||||||
|
for (int i = 0; i < moreDocs; i++) {
|
||||||
|
client.prepareIndex(indexName, "_doc").setSource("num", i).get();
|
||||||
|
}
|
||||||
|
assertAcked(client().execute(FreezeIndexAction.INSTANCE, new FreezeRequest(indexName)).actionGet());
|
||||||
|
return super.onNodeStopped(nodeName);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
ensureGreen(indexName);
|
||||||
|
internalCluster().assertSameDocIdsOnShards();
|
||||||
|
for (RecoveryState recovery : client().admin().indices().prepareRecoveries(indexName).get().shardRecoveryStates().get(indexName)) {
|
||||||
|
if (recovery.getPrimary() == false) {
|
||||||
|
assertThat(recovery.getIndex().fileDetails(), not(empty()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
internalCluster().fullRestart();
|
||||||
|
ensureGreen(indexName);
|
||||||
|
internalCluster().assertSameDocIdsOnShards();
|
||||||
|
for (RecoveryState recovery : client().admin().indices().prepareRecoveries(indexName).get().shardRecoveryStates().get(indexName)) {
|
||||||
|
if (recovery.getPrimary() == false) {
|
||||||
|
assertThat(recovery.getIndex().fileDetails(), empty());
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,43 @@
|
||||||
|
/*
|
||||||
|
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
|
||||||
|
* or more contributor license agreements. Licensed under the Elastic License;
|
||||||
|
* you may not use this file except in compliance with the Elastic License.
|
||||||
|
*/
|
||||||
|
package org.elasticsearch.index.engine;
|
||||||
|
|
||||||
|
import org.elasticsearch.cluster.routing.RecoverySource;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||||
|
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
||||||
|
import org.elasticsearch.common.settings.Settings;
|
||||||
|
import org.elasticsearch.index.shard.IndexShard;
|
||||||
|
import org.elasticsearch.index.shard.IndexShardTestCase;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import static org.hamcrest.Matchers.equalTo;
|
||||||
|
|
||||||
|
public class FrozenIndexShardTests extends IndexShardTestCase {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Make sure we can recover from a frozen engine
|
||||||
|
*/
|
||||||
|
public void testRecoverFromFrozenPrimary() throws IOException {
|
||||||
|
IndexShard indexShard = newStartedShard(true);
|
||||||
|
indexDoc(indexShard, "_doc", "1");
|
||||||
|
indexDoc(indexShard, "_doc", "2");
|
||||||
|
indexDoc(indexShard, "_doc", "3");
|
||||||
|
indexShard.close("test", true);
|
||||||
|
final ShardRouting shardRouting = indexShard.routingEntry();
|
||||||
|
IndexShard frozenShard = reinitShard(indexShard, ShardRoutingHelper.initWithSameId(shardRouting,
|
||||||
|
shardRouting.primary() ? RecoverySource.ExistingStoreRecoverySource.INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE
|
||||||
|
), indexShard.indexSettings().getIndexMetaData(), FrozenEngine::new);
|
||||||
|
recoverShardFromStore(frozenShard);
|
||||||
|
assertThat(frozenShard.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(frozenShard.seqNoStats().getMaxSeqNo()));
|
||||||
|
assertDocCount(frozenShard, 3);
|
||||||
|
|
||||||
|
IndexShard replica = newShard(false, Settings.EMPTY, FrozenEngine::new);
|
||||||
|
recoverReplica(replica, frozenShard, true);
|
||||||
|
assertDocCount(replica, 3);
|
||||||
|
closeShards(frozenShard, replica);
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue