diff --git a/build.gradle b/build.gradle index 6f6f33a8890df..1ced202266826 100644 --- a/build.gradle +++ b/build.gradle @@ -173,8 +173,8 @@ task verifyVersions { * after the backport of the backcompat code is complete. */ -boolean bwc_tests_enabled = true -final String bwc_tests_disabled_issue = "" /* place a PR link here when committing bwc changes */ +boolean bwc_tests_enabled = false +final String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/45137" /* place a PR link here when committing bwc changes */ if (bwc_tests_enabled == false) { if (bwc_tests_disabled_issue.isEmpty()) { throw new GradleException("bwc_tests_disabled_issue must be set when bwc_tests_enabled == false") diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/CCRIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/CCRIT.java index 790c3c9a955b5..11650ca040635 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/CCRIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/CCRIT.java @@ -54,6 +54,7 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.test.rest.yaml.ObjectPath; import org.junit.Before; @@ -260,7 +261,9 @@ public void testForgetFollower() throws IOException { final Map shardStatsAsMap = (Map) shardStats.get(0); final Map retentionLeasesStats = (Map) shardStatsAsMap.get("retention_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)); + } } } diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 350ef19ccc6c5..13be323aee888 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -34,7 +34,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; - +import org.elasticsearch.index.seqno.RetentionLeaseUtils; import org.elasticsearch.test.NotEqualMessageBuilder; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.yaml.ObjectPath; @@ -80,7 +80,7 @@ public class FullClusterRestartIT extends AbstractFullClusterRestartTestCase { private String index; @Before - public void setIndex() throws IOException { + public void setIndex() { index = getTestName().toLowerCase(Locale.ROOT); } @@ -1230,4 +1230,26 @@ protected void ensureGreenLongWait(String index) throws IOException { assertEquals("green", healthRsp.get("status")); assertFalse((Boolean) healthRsp.get("timed_out")); } + + public void testPeerRecoveryRetentionLeases() throws IOException { + 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)); + 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); + } + } } diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java index ac7f38d7ac934..c0d192d38ec00 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/RecoveryIT.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.RetentionLeaseUtils; import org.elasticsearch.rest.action.document.RestIndexAction; import org.elasticsearch.test.rest.yaml.ObjectPath; import org.hamcrest.Matcher; @@ -382,6 +383,80 @@ public void testRecoveryWithSoftDeletes() throws Exception { 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 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 * is effectively closed and potentially replicated (if the version the index was created on supports diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 32d1dc2ef659e..7a50d3471a335 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -747,7 +747,7 @@ public abstract int estimateNumberOfHistoryOperations(String source, 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; diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index a6479c486f653..119f52ea1736e 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -514,10 +514,15 @@ public void syncTranslog() throws IOException { } /** - * 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 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); } @@ -525,7 +530,14 @@ public Translog.Snapshot readHistoryOperations(String source, MapperService mapp * Returns the estimated number of history operations whose seq# at least the provided seq# in this engine. */ @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); } @@ -2571,6 +2583,10 @@ public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperS @Override public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return getMinRetainedSeqNo() <= startingSeqNo; + } + final long currentLocalCheckpoint = localCheckpointTracker.getProcessedCheckpoint(); // avoid scanning translog if not necessary if (startingSeqNo > currentLocalCheckpoint) { @@ -2600,15 +2616,7 @@ public final long getMinRetainedSeqNo() { @Override public Closeable acquireRetentionLock() { if (softDeleteEnabled) { - final Releasable softDeletesRetentionLock = softDeletesPolicy.acquireRetentionLock(); - final Closeable translogRetentionLock; - try { - translogRetentionLock = translog.acquireRetentionLock(); - } catch (Exception e) { - softDeletesRetentionLock.close(); - throw e; - } - return () -> IOUtils.close(translogRetentionLock, softDeletesRetentionLock); + return softDeletesPolicy.acquireRetentionLock(); } else { return translog.acquireRetentionLock(); } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 20e998b1e12a6..7185fd4319af4 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -23,7 +23,9 @@ import com.carrotsearch.hppc.ObjectLongMap; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.GroupedActionListener; import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -59,6 +61,7 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; 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). @@ -199,6 +202,14 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L */ 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. * @@ -224,10 +235,22 @@ public synchronized Tuple getRetentionLeases(final boo // the primary calculates the non-expired retention leases and syncs them to replicas final long currentTimeMillis = currentTimeMillisSupplier.getAsLong(); final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis(); + final Set leaseIdsForCurrentPeers + = routingTable.assignedShards().stream().map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet()); final Map> partitionByExpiration = retentionLeases .leases() .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 expiredLeases = partitionByExpiration.get(true); if (expiredLeases == null) { // early out as no retention leases have expired @@ -249,7 +272,7 @@ public synchronized Tuple getRetentionLeases(final boo * @param source the source of the retention lease * @param listener the callback when the retention lease is successfully added and synced to replicas * @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( final String id, @@ -259,23 +282,69 @@ public RetentionLease addRetentionLease( Objects.requireNonNull(listener); final RetentionLease retentionLease; final RetentionLeases currentRetentionLeases; + synchronized (this) { + retentionLease = innerAddRetentionLease(id, retainingSequenceNumber, source); + currentRetentionLeases = retentionLeases; + } + onSyncRetentionLeases.accept(currentRetentionLeases, listener); + 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 listener) { + Objects.requireNonNull(listener); + final RetentionLease retentionLease; + final RetentionLeases currentRetentionLeases; synchronized (this) { assert primaryMode; - if (retentionLeases.contains(id)) { - throw new RetentionLeaseAlreadyExistsException(id); + if (getRetentionLeases().contains(sourceLeaseId) == false) { + throw new RetentionLeaseNotFoundException(sourceLeaseId); } - 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())); + 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. * @@ -283,7 +352,7 @@ public RetentionLease addRetentionLease( * @param retainingSequenceNumber the retaining sequence number * @param source the source of the 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) { assert primaryMode; @@ -397,6 +466,108 @@ public boolean assertRetentionLeasesPersisted(final Path path) throws IOExceptio 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 listener) { + return addRetentionLease(getPeerRecoveryRetentionLeaseId(nodeId), globalCheckpoint + 1, + PEER_RECOVERY_RETENTION_LEASE_SOURCE, listener); + } + + public RetentionLease cloneLocalPeerRecoveryRetentionLease(String nodeId, ActionListener listener) { + return cloneRetentionLease( + getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()), + getPeerRecoveryRetentionLeaseId(nodeId), listener); + } + + public void removePeerRecoveryRetentionLease(String nodeId, ActionListener 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 { /** @@ -614,6 +785,23 @@ private boolean invariant() { 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; } @@ -667,6 +855,8 @@ public ReplicationTracker( this.pendingInSync = new HashSet<>(); this.routingTable = null; this.replicationGroup = null; + this.hasAllPeerRecoveryRetentionLeases = indexSettings.getIndexVersionCreated().onOrAfter(Version.V_7_4_0); + assert Version.V_EMPTY.equals(indexSettings.getIndexVersionCreated()) == false; assert invariant(); } @@ -758,9 +948,55 @@ public synchronized void activatePrimaryMode(final long localCheckpoint) { primaryMode = true; updateLocalCheckpoint(shardAllocationId, checkpoints.get(shardAllocationId), localCheckpoint); updateGlobalCheckpointOnPrimary(); + + if (indexSettings.isSoftDeleteEnabled()) { + addPeerRecoveryRetentionLeaseForSolePrimary(); + } + 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. * @@ -1057,9 +1293,57 @@ public synchronized void activateWithPrimaryContext(PrimaryContext primaryContex // note that if there was no cluster state update between start of the engine of this shard and the call to // initializeWithPrimaryContext, we might still have missed a cluster state update. This is best effort. runAfter.run(); + + if (indexSettings.isSoftDeleteEnabled()) { + addPeerRecoveryRetentionLeaseForSolePrimary(); + } + 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 listener) { + if (indexSettings().isSoftDeleteEnabled() + && indexSettings().getIndexMetaData().getState() == IndexMetaData.State.OPEN + && hasAllPeerRecoveryRetentionLeases == false) { + + final List shardRoutings = routingTable.assignedShards(); + final GroupedActionListener 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() { assert primaryMode == false; final long lastAppliedClusterStateVersion = appliedClusterStateVersion; diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncer.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncer.java index 927d2ec499960..7de6bad3f1102 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncer.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncer.java @@ -44,7 +44,7 @@ public interface RetentionLeaseSyncer { RetentionLeaseSyncer EMPTY = new RetentionLeaseSyncer() { @Override public void sync(final ShardId shardId, final RetentionLeases retentionLeases, final ActionListener listener) { - + listener.onResponse(new ReplicationResponse()); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java index 81fd7e2fce047..8c5c282a72d08 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java @@ -274,14 +274,5 @@ private static Map toMap(final Collection toMap(final RetentionLeases retentionLeases) { - return retentionLeases.leases; - } - } + diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 209ad7d1a9837..c62cec46faa26 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -40,6 +40,7 @@ import org.elasticsearch.Assertions; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; @@ -159,6 +160,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; @@ -170,6 +172,7 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; +import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -484,6 +487,7 @@ public void updateShardState(final ShardRouting newRouting, if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) { // the master started a recovering primary, activate primary mode. replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + ensurePeerRecoveryRetentionLeasesExist(); } } else { assert currentRouting.primary() == false : "term is only increased as part of primary promotion"; @@ -526,6 +530,7 @@ public void updateShardState(final ShardRouting newRouting, assert getOperationPrimaryTerm() == newPrimaryTerm; try { replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + ensurePeerRecoveryRetentionLeasesExist(); /* * 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 @@ -1357,6 +1362,88 @@ public void prepareForIndexRecovery() { 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 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 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) { getEngine().trimOperationsFromTranslog(getOperationPrimaryTerm(), aboveSeqNo); } @@ -1456,11 +1543,23 @@ int runTranslogRecovery(Engine engine, Translog.Snapshot snapshot, Engine.Operat 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. * Operations from the translog will be replayed to bring lucene up to date. **/ 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 Engine.TranslogRecoveryRunner translogRecoveryRunner = (engine, snapshot) -> { translogRecoveryStats.totalOperations(snapshot.totalOperations()); @@ -1468,7 +1567,8 @@ public void openEngineAndRecoverFromTranslog() throws IOException { return runTranslogRecovery(engine, snapshot, Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY, translogRecoveryStats::incrementRecoveredOperations); }; - innerOpenEngineAndTranslog(); + loadGlobalCheckpointToReplicationTracker(); + innerOpenEngineAndTranslog(replicationTracker); getEngine().recoverFromTranslog(translogRecoveryRunner, Long.MAX_VALUE); } @@ -1477,36 +1577,22 @@ public void openEngineAndRecoverFromTranslog() throws IOException { * The translog is kept but its operations won't be replayed. */ 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(); } - private void innerOpenEngineAndTranslog() throws IOException { + private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) throws IOException { if (state != IndexShardState.RECOVERING) { throw new IndexShardNotRecoveringException(shardId, state); } - recoveryState.setStage(RecoveryState.Stage.VERIFY_INDEX); - // 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(); + final EngineConfig config = newEngineConfig(globalCheckpointSupplier); // 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 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()); assert recoveryState.getRecoverySource().expectEmptyRetentionLeases() == false || getRetentionLeases().leases().isEmpty() : "expected empty set of retention leases with recovery source [" + recoveryState.getRecoverySource() @@ -1551,14 +1637,22 @@ private void onNewEngine(Engine newEngine) { */ public void performRecoveryRestart() throws IOException { synchronized (mutex) { - if (state != IndexShardState.RECOVERING) { - throw new IndexShardNotRecoveringException(shardId, state); - } assert refreshListeners.pendingCount() == 0 : "we can't restart with pending listeners"; - final Engine engine = this.currentEngineReference.getAndSet(null); - IOUtils.close(engine); - recoveryState().setStage(RecoveryState.Stage.INIT); + IOUtils.close(currentEngineReference.getAndSet(null)); + resetRecoveryStage(); + } + } + + /** + * 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); } /** @@ -2099,6 +2193,7 @@ public void syncRetentionLeases() { assert assertPrimaryMode(); verifyNotClosed(); ensureSoftDeletesEnabled("retention leases"); + replicationTracker.renewPeerRecoveryRetentionLeases(); final Tuple retentionLeases = getRetentionLeases(true); if (retentionLeases.v1()) { logger.trace("syncing retention leases [{}] after expiration check", retentionLeases.v2()); @@ -2266,6 +2361,13 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p synchronized (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 void noopUpdate(String 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 { if (store.tryIncRef()) { try { @@ -2485,6 +2598,24 @@ public boolean isRelocatedPrimary() { return replicationTracker.isRelocated(); } + public RetentionLease addPeerRecoveryRetentionLease(String nodeId, long globalCheckpoint, + ActionListener 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 listener) { + assert assertPrimaryMode(); + return replicationTracker.cloneLocalPeerRecoveryRetentionLease(nodeId, listener); + } + + public void removePeerRecoveryRetentionLease(String nodeId, ActionListener listener) { + assert assertPrimaryMode(); + replicationTracker.removePeerRecoveryRetentionLease(nodeId, listener); + } + class ShardEventListener implements Engine.EventListener { private final CopyOnWriteArrayList> delegates = new CopyOnWriteArrayList<>(); @@ -2538,7 +2669,7 @@ private DocumentMapperForType docMapper(String type) { mapperService.resolveDocumentType(type)); } - private EngineConfig newEngineConfig() { + private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) { Sort indexSort = indexSortSupplier.get(); return new EngineConfig(shardId, shardRouting.allocationId().getId(), threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(), @@ -2548,7 +2679,7 @@ private EngineConfig newEngineConfig() { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, circuitBreakerService, replicationTracker, replicationTracker::getRetentionLeases, + indexSort, circuitBreakerService, globalCheckpointSupplier, replicationTracker::getRetentionLeases, () -> getOperationPrimaryTerm(), tombstoneDocSupplier()); } @@ -3185,7 +3316,7 @@ assert getActiveOperationsCount() == OPERATIONS_BLOCKED // we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata, // acquireXXXCommit and close works. final Engine readOnlyEngine = - new ReadOnlyEngine(newEngineConfig(), seqNoStats, translogStats, false, Function.identity()) { + new ReadOnlyEngine(newEngineConfig(replicationTracker), seqNoStats, translogStats, false, Function.identity()) { @Override public IndexCommitRef acquireLastIndexCommit(boolean flushFirst) { synchronized (mutex) { @@ -3214,7 +3345,7 @@ public void close() throws IOException { } }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); - newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig())); + newEngineReference.set(engineFactory.newReadWriteEngine(newEngineConfig(replicationTracker))); onNewEngine(newEngineReference.get()); } final Engine.TranslogRecoveryRunner translogRunner = (engine, snapshot) -> runTranslogRecovery( diff --git a/server/src/main/java/org/elasticsearch/index/store/Store.java b/server/src/main/java/org/elasticsearch/index/store/Store.java index 410774114bd78..eae9ca3a8bda6 100644 --- a/server/src/main/java/org/elasticsearch/index/store/Store.java +++ b/server/src/main/java/org/elasticsearch/index/store/Store.java @@ -96,6 +96,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -1560,6 +1561,22 @@ public void trimUnsafeCommits(final long lastSyncedGlobalCheckpoint, final long } } + /** + * Returns a {@link org.elasticsearch.index.seqno.SequenceNumbers.CommitInfo} of the safe commit if exists. + */ + public Optional findSafeIndexCommit(long globalCheckpoint) throws IOException { + final List 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 keysToUpdate) throws IOException { final Map userData = getUserData(writer); userData.putAll(keysToUpdate); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 8a11cdf5ec961..5874fd423a541 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -22,8 +22,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; 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.RateLimiter; import org.elasticsearch.ElasticsearchException; @@ -44,18 +42,14 @@ import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.index.engine.CombinedDeletionPolicy; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardNotFoundException; 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.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; @@ -68,12 +62,11 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.List; -import java.util.StringJoiner; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; 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. @@ -178,9 +171,12 @@ private void doRecovery(final long recoveryId) { cancellableThreads = recoveryTarget.cancellableThreads(); try { 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()); 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) { // this will be logged as warning later on... logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e); @@ -314,59 +310,44 @@ public RecoveryResponse read(StreamInput in) throws IOException { } /** - * Obtains a snapshot of the store metadata for the recovery target. + * 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 - * @return a snapshot of the store metadata + * @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 */ - private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) { + public static StartRecoveryRequest getStartRecoveryRequest(Logger logger, DiscoveryNode localNode, + RecoveryTarget recoveryTarget, long startingSeqNo) { + final StartRecoveryRequest request; + logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + + Store.MetadataSnapshot metadataSnapshot; try { - return recoveryTarget.indexShard().snapshotStoreMetadata(); + 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); - return Store.MetadataSnapshot.EMPTY; + metadataSnapshot = 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; + 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; } - } - - /** - * Prepare the start recovery request. - * - * @param recoveryTarget the target of the recovery - * @return a start recovery request - */ - private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recoveryTarget) { - final StartRecoveryRequest request; - logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); - - final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget); 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( recoveryTarget.shardId(), recoveryTarget.indexShard().routingEntry().allocationId().getId(), recoveryTarget.sourceNode(), - clusterService.localNode(), + localNode, metadataSnapshot, recoveryTarget.state().getPrimary(), recoveryTarget.recoveryId(), @@ -374,50 +355,6 @@ private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recove 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 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 { void onRecoveryDone(RecoveryState state); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index e1353ecb52fac..77c86c6e02946 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -29,9 +29,14 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; 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.ShardRouting; import org.elasticsearch.common.CheckedSupplier; @@ -49,6 +54,9 @@ import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; 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.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -139,8 +147,14 @@ public void recoverToTarget(ActionListener listener) { IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); throw e; }); - final Consumer onFailure = e -> + final Consumer onFailure = e -> { + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[onFailure]"); IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); + }; + + final boolean useRetentionLeases = shard.indexSettings().isSoftDeleteEnabled() + && shard.indexSettings().getIndexMetaData().getState() != IndexMetaData.State.CLOSE; + final SetOnce retentionLeaseRef = new SetOnce<>(); runUnderPrimaryPermit(() -> { final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); @@ -151,13 +165,35 @@ public void recoverToTarget(ActionListener listener) { 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; + retentionLeaseRef.set(useRetentionLeases ? shard.getRetentionLeases().get( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(targetShardRouting)) : null); }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); final Closeable retentionLock = shard.acquireRetentionLock(); resources.add(retentionLock); final long startingSeqNo; - final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); + final boolean isSequenceNumberBasedRecovery + = 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 sendFileStep = new StepListener<>(); final StepListener prepareEngineStep = new StepListener<>(); @@ -176,9 +212,22 @@ public void recoverToTarget(ActionListener listener) { } catch (final Exception 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. - startingSeqNo = 0; + + // Try and copy enough operations to the recovering peer so that if it is promoted to primary then it has a chance of being + // 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 { final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); shard.store().incRef(); @@ -191,7 +240,40 @@ public void recoverToTarget(ActionListener listener) { logger.warn("releasing snapshot caused exception", ex); } }); - phase1(safeCommitRef.getIndexCommit(), shard.getLastKnownGlobalCheckpoint(), () -> estimateNumOps, sendFileStep); + + final StepListener 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> 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) { throw new RecoveryEngineException(shard.shardId(), 1, "sendFileStep failed", e); } @@ -199,12 +281,14 @@ public void recoverToTarget(ActionListener listener) { assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo; sendFileStep.whenComplete(r -> { + assert Transports.assertNotTransportThread(RecoverySourceHandler.this + "[prepareTargetForTranslog]"); // For a sequence based recovery, the target can keep its local translog prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep); }, onFailure); 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. * This means that any document indexed into the primary after this will be replicated to this replica as well @@ -215,14 +299,16 @@ public void recoverToTarget(ActionListener listener) { shardId + " initiating tracking of " + request.targetAllocationId(), shard, cancellableThreads, logger); final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - if (logger.isTraceEnabled()) { - logger.trace("snapshot translog for recovery; current size is [{}]", - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); - } + logger.trace("snapshot translog for recovery; current size is [{}]", + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); final Translog.Snapshot phase2Snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo); 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 // 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(); @@ -341,16 +427,9 @@ static final class SendFileResult { * segments that are missing. Only segments that have the same size and * checksum can be reused */ - void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps, ActionListener listener) { + void phase1(IndexCommit snapshot, Consumer> createRetentionLease, + IntSupplier translogOps, ActionListener listener) { 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 phase1FileNames = new ArrayList<>(); - final List phase1FileSizes = new ArrayList<>(); - final List phase1ExistingFileNames = new ArrayList<>(); - final List phase1ExistingFileSizes = new ArrayList<>(); final Store store = shard.store(); try { StopWatch stopWatch = new StopWatch().start(); @@ -370,6 +449,16 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps } } if (canSkipPhase1(recoverySourceMetadata, request.metadataSnapshot()) == false) { + final List phase1FileNames = new ArrayList<>(); + final List phase1FileSizes = new ArrayList<>(); + final List phase1ExistingFileNames = new ArrayList<>(); + final List 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 // segment files on the target node, using the existing files on // the source node @@ -404,6 +493,7 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSizeInBytes)); final StepListener sendFileInfoStep = new StepListener<>(); final StepListener sendFilesStep = new StepListener<>(); + final StepListener createRetentionLeaseStep = new StepListener<>(); final StepListener cleanFilesStep = new StepListener<>(); cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo(phase1FileNames, phase1FileSizes, phase1ExistingFileNames, @@ -412,8 +502,20 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps sendFileInfoStep.whenComplete(r -> sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps, sendFilesStep), listener::onFailure); - sendFilesStep.whenComplete(r -> - cleanFiles(store, recoverySourceMetadata, translogOps, globalCheckpoint, cleanFilesStep), listener::onFailure); + sendFilesStep.whenComplete(r -> createRetentionLease.accept(createRetentionLeaseStep), 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 existingTotalSize = existingTotalSizeInBytes; @@ -424,18 +526,59 @@ void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps phase1ExistingFileSizes, existingTotalSize, took)); }, listener::onFailure); } else { - logger.trace("skipping [phase1]- identical sync id [{}] found on both source and target", - recoverySourceMetadata.getSyncId()); - final TimeValue took = stopWatch.totalTime(); - logger.trace("recovery [phase1]: took [{}]", took); - listener.onResponse(new SendFileResult(phase1FileNames, phase1FileSizes, totalSizeInBytes, phase1ExistingFileNames, - phase1ExistingFileSizes, existingTotalSizeInBytes, took)); + logger.trace("skipping [phase1] since source and target have identical sync id [{}]", recoverySourceMetadata.getSyncId()); + + // but we must still create a retention lease + final StepListener createRetentionLeaseStep = new StepListener<>(); + createRetentionLease.accept(createRetentionLeaseStep); + 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) { - 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 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 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 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) { if (source.getSyncId() == null || source.getSyncId().equals(target.getSyncId()) == false) { return false; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 4808b65e7eab9..fec3f1508ff99 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices.recovery; +import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; @@ -33,6 +34,7 @@ import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import java.io.IOException; @@ -459,6 +461,7 @@ public static class Translog extends Timer implements ToXContentFragment, Writea private int recovered; private int total = UNKNOWN; private int totalOnStart = UNKNOWN; + private int totalLocal = UNKNOWN; public Translog() { } @@ -468,6 +471,9 @@ public Translog(StreamInput in) throws IOException { recovered = in.readVInt(); total = in.readVInt(); totalOnStart = in.readVInt(); + if (in.getVersion().onOrAfter(Version.V_7_4_0)) { + totalLocal = in.readVInt(); + } } @Override @@ -476,6 +482,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVInt(recovered); out.writeVInt(total); out.writeVInt(totalOnStart); + if (out.getVersion().onOrAfter(Version.V_7_4_0)) { + out.writeVInt(totalLocal); + } } public synchronized void reset() { @@ -483,6 +492,7 @@ public synchronized void reset() { recovered = 0; total = UNKNOWN; totalOnStart = UNKNOWN; + totalLocal = UNKNOWN; } public synchronized void incrementRecoveredOperations() { @@ -524,8 +534,8 @@ public synchronized int totalOperations() { } public synchronized void totalOperations(int total) { - this.total = total; - assert total == UNKNOWN || total >= recovered : "total, if known, should be > recovered. total [" + total + + this.total = totalLocal == UNKNOWN ? total : totalLocal + total; + assert total == UNKNOWN || this.total >= recovered : "total, if known, should be > recovered. total [" + total + "], recovered [" + recovered + "]"; } @@ -540,7 +550,20 @@ public synchronized int totalOperationsOnStart() { } 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() { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index cc14b0af72000..2a3133122d6b7 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -376,6 +376,8 @@ public void receiveFileInfo(List phase1FileNames, int totalTranslogOps, ActionListener listener) { ActionListener.completeWith(listener, () -> { + indexShard.resetRecoveryStage(); + indexShard.prepareForIndexRecovery(); final RecoveryState.Index index = state().getIndex(); for (int i = 0; i < phase1ExistingFileNames.size(); i++) { index.addFileDetail(phase1ExistingFileNames.get(i), phase1ExistingFileSizes.get(i), true); @@ -413,7 +415,8 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada } else { assert indexShard.assertRetentionLeasesPersisted(); } - + indexShard.maybeCheckIndex(); + state().setStage(RecoveryState.Stage.TRANSLOG); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. // this means we transferred files from the remote that have not be checksummed and they are diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 962788f09d23b..4656c2da54155 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.engine.Engine; @@ -427,8 +428,12 @@ public void testReuseInFileBasedPeerRecovery() throws Exception { .setSettings(Settings.builder() .put("number_of_shards", 1) .put("number_of_replicas", 1) + // 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(); logger.info("--> indexing docs"); @@ -472,10 +477,13 @@ public Settings onNodeStopped(String nodeName) throws Exception { .put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_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_LEASE_PERIOD_SETTING.getKey(), "0s") ).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 - client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + client().admin().indices().prepareFlush("test").setForce(true).get(); } return super.onNodeStopped(nodeName); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index af322fba3f85b..68af2d6438f6c 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -73,6 +73,7 @@ import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -2357,7 +2358,7 @@ public void testIndexWriterInfoStream() throws IllegalAccessException, IOExcepti } } - public void testSeqNoAndCheckpoints() throws IOException { + public void testSeqNoAndCheckpoints() throws IOException, InterruptedException { final int opCount = randomIntBetween(1, 256); long primarySeqNo = SequenceNumbers.NO_OPS_PERFORMED; final String[] ids = new String[]{"1", "2", "3"}; @@ -2375,13 +2376,27 @@ public void testSeqNoAndCheckpoints() throws IOException { final ShardRouting primary = TestShardRouting.newShardRouting("test", shardId.id(), "node1", null, true, ShardRoutingState.STARTED, allocationId); - final ShardRouting replica = - TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED); + final ShardRouting initializingReplica = + TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.INITIALIZING); + ReplicationTracker gcpTracker = (ReplicationTracker) initialEngine.config().getGlobalCheckpointSupplier(); - gcpTracker.updateFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), - replica.allocationId().getId())), - new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build()); + gcpTracker.updateFromMaster(1L, new HashSet<>(Collections.singletonList(primary.allocationId().getId())), + new IndexShardRoutingTable.Builder(shardId).addShard(primary).build()); 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++) { final String id; // mostly index, sometimes delete diff --git a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java index 7e8f18dd005fc..f4949804e3079 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/NoOpEngineRecoveryTests.java @@ -40,7 +40,8 @@ public void testRecoverFromNoOp() throws IOException { indexShard.close("test", true); 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); assertEquals(primary.seqNoStats().getMaxSeqNo(), primary.getMaxSeqNoOfUpdatesOrDeletes()); assertEquals(nbDocs, primary.docStats().getCount()); diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index c60f32132c646..f05ddce567a8a 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -49,6 +49,7 @@ import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.InternalEngineTests; import org.elasticsearch.index.mapper.SourceToParse; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -66,6 +67,7 @@ import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @@ -78,6 +80,7 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.everyItem; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.isIn; @@ -114,31 +117,26 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { int docs = shards.indexDocs(randomInt(50)); shards.flush(); final IndexShard originalReplica = shards.getReplicas().get(0); - long replicaCommittedLocalCheckpoint = docs - 1; - boolean replicaHasDocsSinceLastFlushedCheckpoint = false; for (int i = 0; i < randomInt(2); i++) { final int indexedDocs = shards.indexDocs(randomInt(5)); docs += indexedDocs; - if (indexedDocs > 0) { - replicaHasDocsSinceLastFlushedCheckpoint = true; - } final boolean flush = randomBoolean(); if (flush) { 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 shards.syncGlobalCheckpoint(); - + long globalCheckpointOnReplica = originalReplica.getLastSyncedGlobalCheckpoint(); + Optional safeCommitOnReplica = + originalReplica.store().findSafeIndexCommit(globalCheckpointOnReplica); + assertTrue(safeCommitOnReplica.isPresent()); shards.removeReplica(originalReplica); final int missingOnReplica = shards.indexDocs(randomInt(5)); docs += missingOnReplica; - replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0; final boolean translogTrimmed; if (randomBoolean()) { @@ -157,14 +155,15 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { final IndexShard recoveredReplica = shards.addReplicaWithExistingPath(originalReplica.shardPath(), originalReplica.routingEntry().currentNodeId()); 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 assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty())); } else { assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty()); - assertThat( - recoveredReplica.recoveryState().getTranslog().recoveredOperations(), - equalTo(Math.toIntExact(docs - (replicaCommittedLocalCheckpoint + 1)))); + assertThat(recoveredReplica.recoveryState().getTranslog().recoveredOperations(), + equalTo(Math.toIntExact(docs - 1 - safeCommitOnReplica.get().localCheckpoint))); + assertThat(recoveredReplica.recoveryState().getTranslog().totalLocal(), + equalTo(Math.toIntExact(globalCheckpointOnReplica - safeCommitOnReplica.get().localCheckpoint))); } docs += shards.indexDocs(randomInt(5)); @@ -231,10 +230,9 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { try (ReplicationGroup shards = createGroup(2)) { shards.startAll(); int totalDocs = shards.indexDocs(randomInt(10)); - int committedDocs = 0; + shards.syncGlobalCheckpoint(); if (randomBoolean()) { shards.flush(); - committedDocs = totalDocs; } final IndexShard oldPrimary = shards.getPrimary(); @@ -254,7 +252,10 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { oldPrimary.flush(new FlushRequest(index.getName())); } } - + long globalCheckpointOnOldPrimary = oldPrimary.getLastSyncedGlobalCheckpoint(); + Optional safeCommitOnOldPrimary = + oldPrimary.store().findSafeIndexCommit(globalCheckpointOnOldPrimary); + assertTrue(safeCommitOnOldPrimary.isPresent()); shards.promoteReplicaToPrimary(newPrimary).get(); // check that local checkpoint of new primary is properly tracked after primary promotion @@ -291,6 +292,15 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { // 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. 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)); totalDocs += uncommittedOpsOnPrimary; @@ -310,7 +320,10 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { if (expectSeqNoRecovery) { 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 { assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedOpsOnPrimary)); diff --git a/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java index ce3986f0a2517..75de0bb677296 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; +import org.elasticsearch.index.seqno.RetentionLeaseUtils; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; @@ -61,9 +62,10 @@ public void testSimpleSyncRetentionLeases() throws Exception { } } 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.leases(), containsInAnyOrder(leases.toArray(new RetentionLease[0]))); + assertThat(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(leasesOnPrimary).values(), + containsInAnyOrder(leases.toArray(new RetentionLease[0]))); latch.await(); for (IndexShard replica : group.getReplicas()) { assertThat(replica.getRetentionLeases(), equalTo(leasesOnPrimary)); @@ -109,9 +111,12 @@ protected void syncRetentionLeases(ShardId shardId, RetentionLeases leases, Acti } }) { group.startAll(); + for (IndexShard replica : group.getReplicas()) { + replica.updateRetentionLeasesOnReplica(group.getPrimary().getRetentionLeases()); + } int numLeases = between(1, 100); IndexShard newPrimary = randomFrom(group.getReplicas()); - RetentionLeases latestRetentionLeasesOnNewPrimary = RetentionLeases.EMPTY; + RetentionLeases latestRetentionLeasesOnNewPrimary = newPrimary.getRetentionLeases(); for (int i = 0; i < numLeases; i++) { PlainActionFuture addLeaseFuture = new PlainActionFuture<>(); group.addRetentionLease(Integer.toString(i), randomNonNegativeLong(), "test-" + i, addLeaseFuture); @@ -130,7 +135,7 @@ protected void syncRetentionLeases(ShardId shardId, RetentionLeases leases, Acti group.addRetentionLease("new-lease-after-promotion", randomNonNegativeLong(), "test", newLeaseFuture); RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases(); 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)); RetentionLeaseSyncAction.Request request = ((SyncRetentionLeasesResponse) newLeaseFuture.actionGet()).syncRequest; for (IndexShard replica : group.getReplicas()) { diff --git a/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseCreationIT.java b/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseCreationIT.java new file mode 100644 index 0000000000000..03842d11b1a40 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseCreationIT.java @@ -0,0 +1,85 @@ +/* + * 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, + VersionUtils.randomVersionBetween(random(), Version.CURRENT.minimumIndexCompatibilityVersion(), 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))); + } + +} diff --git a/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseExpiryTests.java b/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseExpiryTests.java new file mode 100644 index 0000000000000..22d4f5e86f964 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseExpiryTests.java @@ -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 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 retentionLeases = replicationTracker.getRetentionLeases(true); + assertFalse(retentionLeases.v1()); + + final Set 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 retentionLeases = replicationTracker.getRetentionLeases(true); + assertFalse("should not have expired anything", retentionLeases.v1()); + + final Set 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 retentionLeases = replicationTracker.getRetentionLeases(true); + assertTrue("should have expired something", retentionLeases.v1()); + + final Set 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 retentionLeases = replicationTracker.getRetentionLeases(true); + assertTrue(retentionLeases.v1()); + + final Set 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()))); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java index 2334cb4330887..7611fad5a7e43 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -20,6 +20,8 @@ package org.elasticsearch.index.seqno; 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.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; @@ -43,6 +45,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import java.util.stream.Collectors; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; @@ -83,7 +86,7 @@ public void testAddOrRenewRetentionLease() { minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); replicationTracker.addRetentionLease( 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++) { @@ -93,7 +96,7 @@ public void testAddOrRenewRetentionLease() { } minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE); 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 void testAddRetentionLeaseCausesRetentionLeaseSync() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId)); replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + retainingSequenceNumbers.put(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { @@ -239,12 +243,109 @@ public void testRemoveRetentionLease() { length - i - 1, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } } + public void testCloneRetentionLease() { + final AllocationId allocationId = AllocationId.newInitializing(); + final AtomicReference 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 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 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() { final AllocationId allocationId = AllocationId.newInitializing(); long primaryTerm = randomLongBetween(1, Long.MAX_VALUE); @@ -298,6 +399,7 @@ public void testRemoveRetentionLeaseCausesRetentionLeaseSync() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId)); replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + retainingSequenceNumbers.put(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { @@ -365,11 +467,12 @@ private void runExpirationTest(final boolean primaryMode) { { final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases(); - assertThat(retentionLeases.version(), equalTo(1L)); - assertThat(retentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); + final long expectedVersion = primaryMode ? 2L : 1L; + assertThat(retentionLeases.version(), equalTo(expectedVersion)); + assertThat(retentionLeases.leases(), hasSize(primaryMode ? 2 : 1)); + final RetentionLease retentionLease = retentionLeases.get("0"); 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 @@ -387,18 +490,19 @@ private void runExpirationTest(final boolean primaryMode) { { final RetentionLeases retentionLeases = replicationTracker.getRetentionLeases(); - assertThat(retentionLeases.version(), equalTo(2L)); - assertThat(retentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); + final long expectedVersion = primaryMode ? 3L : 2L; + assertThat(retentionLeases.version(), equalTo(expectedVersion)); + assertThat(retentionLeases.leases(), hasSize(primaryMode ? 2 : 1)); + final RetentionLease retentionLease = retentionLeases.get("0"); 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 currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); if (primaryMode) { - assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, true, false); - assertRetentionLeases(replicationTracker, 0, new long[0], primaryTerm, 3, true, true); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 3, true, false); + assertRetentionLeases(replicationTracker, 0, new long[0], primaryTerm, 4, true, true); } else { // leases do not expire on replicas until synced from the primary assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, primaryTerm, 2, false, false); @@ -625,10 +729,9 @@ private void assertRetentionLeases( } assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm)); assertThat(retentionLeases.version(), equalTo(version)); - final Map idToRetentionLease = new HashMap<>(); - for (final RetentionLease retentionLease : retentionLeases.leases()) { - idToRetentionLease.put(retentionLease.id(), retentionLease); - } + final Map idToRetentionLease = retentionLeases.leases().stream() + .filter(retentionLease -> ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()) == false) + .collect(Collectors.toMap(RetentionLease::id, Function.identity())); assertThat(idToRetentionLease.entrySet(), hasSize(size)); for (int i = 0; i < size; i++) { diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java index 5165f2e8dc9e4..5f035a3604f41 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java @@ -52,10 +52,14 @@ ReplicationTracker newTracker( (leases, listener) -> {}); } + static String nodeIdFromAllocationId(final AllocationId allocationId) { + return "n-" + allocationId.getId().substring(0, 8); + } + static IndexShardRoutingTable routingTable(final Set initializingIds, final AllocationId primaryId) { final ShardId shardId = new ShardId("test", "_na_", 0); - final ShardRouting primaryShard = - TestShardRouting.newShardRouting(shardId, randomAlphaOfLength(10), null, true, ShardRoutingState.STARTED, primaryId); + final ShardRouting primaryShard = TestShardRouting.newShardRouting( + shardId, nodeIdFromAllocationId(primaryId), null, true, ShardRoutingState.STARTED, primaryId); return routingTable(initializingIds, primaryShard); } @@ -65,7 +69,7 @@ static IndexShardRoutingTable routingTable(final Set initializingI final IndexShardRoutingTable.Builder builder = new IndexShardRoutingTable.Builder(shardId); for (final AllocationId initializingId : initializingIds) { builder.addShard(TestShardRouting.newShardRouting( - shardId, randomAlphaOfLength(10), null, false, ShardRoutingState.INITIALIZING, initializingId)); + shardId, nodeIdFromAllocationId(initializingId), null, false, ShardRoutingState.INITIALIZING, initializingId)); } builder.addShard(primaryShard); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java index 70b728a4a2022..f571193cec405 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.test.IndexSettingsModule; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -50,6 +51,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongConsumer; import java.util.stream.Collectors; @@ -61,6 +63,9 @@ import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; 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; public class ReplicationTrackerTests extends ReplicationTrackerTestCase { @@ -149,6 +154,7 @@ public void testGlobalCheckpointUpdate() { newInitializing.add(extraId); tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId)); + addPeerRecoveryRetentionLease(tracker, extraId); tracker.initiateTracking(extraId.getId()); // now notify for the new id @@ -190,6 +196,7 @@ public void testMarkAllocationIdAsInSync() throws BrokenBarrierException, Interr tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId)); final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1); tracker.activatePrimaryMode(localCheckpoint); + addPeerRecoveryRetentionLease(tracker, replicaId); tracker.initiateTracking(replicaId.getId()); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { @@ -357,6 +364,7 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId)); tracker.activatePrimaryMode(globalCheckpoint); + addPeerRecoveryRetentionLease(tracker, trackingAllocationId); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -421,6 +429,7 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()), routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId)); tracker.activatePrimaryMode(globalCheckpoint); + addPeerRecoveryRetentionLease(tracker, trackingAllocationId); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -563,6 +572,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { initialClusterStateVersion + 3, ids(newActiveAllocationIds), routingTable(newInitializingAllocationIds, primaryId)); + addPeerRecoveryRetentionLease(tracker, newSyncingAllocationId); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { try { @@ -610,7 +620,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { * 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 * 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 * call is added after notifying all waiters in {@link ReplicationTracker#updateLocalCheckpoint(String, long)}). * @@ -630,6 +640,7 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok Collections.singleton(active.getId()), routingTable(Collections.singleton(initializing), active)); tracker.activatePrimaryMode(activeLocalCheckpoint); + addPeerRecoveryRetentionLease(tracker, initializing); final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE); final Thread activeThread = new Thread(() -> { try { @@ -693,7 +704,9 @@ public void testPrimaryContextHandoff() throws IOException { clusterState.apply(oldPrimary); 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); for (int i = 0; i < numUpdates; i++) { @@ -706,7 +719,7 @@ public void testPrimaryContextHandoff() throws IOException { randomLocalCheckpointUpdate(oldPrimary); } if (randomBoolean()) { - randomMarkInSync(oldPrimary); + randomMarkInSync(oldPrimary, newPrimary); } } @@ -738,7 +751,7 @@ public void testPrimaryContextHandoff() throws IOException { randomLocalCheckpointUpdate(oldPrimary); } if (randomBoolean()) { - randomMarkInSync(oldPrimary); + randomMarkInSync(oldPrimary, newPrimary); } // do another handoff @@ -876,7 +889,10 @@ private static FakeClusterState initialState() { final ShardId shardId = new ShardId("test", "_na_", 0); final ShardRouting primaryShard = 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( initialClusterStateVersion, @@ -884,20 +900,17 @@ private static FakeClusterState initialState() { routingTable(initializingAllocationIds, primaryShard)); } - private static void activatePrimary(ReplicationTracker gcp) { - gcp.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10)); - } - private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) { String allocationId = randomFrom(gcp.checkpoints.keySet()); long currentLocalCheckpoint = gcp.checkpoints.get(allocationId).getLocalCheckpoint(); gcp.updateLocalCheckpoint(allocationId, Math.max(SequenceNumbers.NO_OPS_PERFORMED, currentLocalCheckpoint + randomInt(5))); } - private static void randomMarkInSync(ReplicationTracker gcp) { - String allocationId = randomFrom(gcp.checkpoints.keySet()); - long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, gcp.getGlobalCheckpoint() + randomInt(5)); - markAsTrackingAndInSyncQuietly(gcp, allocationId, newLocalCheckpoint); + private static void randomMarkInSync(ReplicationTracker oldPrimary, ReplicationTracker newPrimary) { + final String allocationId = randomFrom(oldPrimary.checkpoints.keySet()); + final long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, oldPrimary.getGlobalCheckpoint() + randomInt(5)); + markAsTrackingAndInSyncQuietly(oldPrimary, allocationId, newLocalCheckpoint); + newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); } private static FakeClusterState randomUpdateClusterState(Set allocationIds, FakeClusterState clusterState) { @@ -908,11 +921,14 @@ private static FakeClusterState randomUpdateClusterState(Set allocationI final Set inSyncIdsToRemove = new HashSet<>( exclude(randomSubsetOf(randomInt(clusterState.inSyncIds.size()), clusterState.inSyncIds), allocationIds)); final Set remainingInSyncIds = Sets.difference(clusterState.inSyncIds, inSyncIdsToRemove); + final Set initializingIdsExceptRelocationTargets = exclude(clusterState.initializingIds(), + clusterState.routingTable.activeShards().stream().filter(ShardRouting::relocating) + .map(s -> s.allocationId().getRelocationId()).collect(Collectors.toSet())); return new FakeClusterState( clusterState.version + randomIntBetween(1, 5), remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds, routingTable( - Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove), + Sets.difference(Sets.union(initializingIdsExceptRelocationTargets, initializingIdsToAdd), initializingIdsToRemove), clusterState.routingTable.primaryShard())); } @@ -945,6 +961,7 @@ private static Set randomAllocationIdsExcludingExistingIds(final S private static void markAsTrackingAndInSyncQuietly( final ReplicationTracker tracker, final String allocationId, final long localCheckpoint) { try { + addPeerRecoveryRetentionLease(tracker, allocationId); tracker.initiateTracking(allocationId); tracker.markAllocationIdAsInSync(allocationId, localCheckpoint); } catch (final InterruptedException e) { @@ -952,4 +969,169 @@ private static void markAsTrackingAndInSyncQuietly( } } + 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> activeAndInitializingAllocationIds = + randomActiveAndInitializingAllocationIds(numberOfActiveAllocationsIds, numberOfInitializingIds); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set 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 retentionLeaseFromAllocationId = allocationId + -> new RetentionLease(ReplicationTracker.getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), + 0L, currentTimeMillis.get(), ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE); + + final List 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 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 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)); + } + } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java index bff4493321289..511a93e8268d1 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java @@ -73,11 +73,14 @@ public void testAddAction() { assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(1)); assertNotNull(stats.getShards()[0].getRetentionLeaseStats()); - assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); - final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2)); + final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id); assertThat(retentionLease.id(), equalTo(id)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber)); assertThat(retentionLease.source(), equalTo(source)); + + assertTrue(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(stats.getShards()[0].getShardRouting()))); } public void testAddAlreadyExists() { @@ -160,9 +163,11 @@ public void testRenewAction() throws InterruptedException { assertNotNull(initialStats.getShards()); assertThat(initialStats.getShards(), arrayWithSize(1)); 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 = - initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id); final long nextRetainingSequenceNumber = retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL @@ -195,9 +200,11 @@ public void testRenewAction() throws InterruptedException { assertNotNull(renewedStats.getShards()); assertThat(renewedStats.getShards(), arrayWithSize(1)); 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 = - renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id); assertThat(renewedRetentionLease.id(), equalTo(id)); assertThat( renewedRetentionLease.retainingSequenceNumber(), @@ -265,7 +272,9 @@ public void testRemoveAction() { assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(1)); 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() { @@ -328,8 +337,10 @@ public void onFailure(final Exception e) { assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(1)); assertNotNull(stats.getShards()[0].getRetentionLeaseStats()); - assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); - final RetentionLease retentionLease = stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + assertThat(stats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2)); + 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.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber)); assertThat(retentionLease.source(), equalTo(source)); @@ -378,9 +389,10 @@ public void testRenewUnderBlock() throws InterruptedException { assertNotNull(initialStats.getShards()); assertThat(initialStats.getShards(), arrayWithSize(1)); assertNotNull(initialStats.getShards()[0].getRetentionLeaseStats()); - assertThat(initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); - final RetentionLease initialRetentionLease = - initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + 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 = initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id); final long nextRetainingSequenceNumber = retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL @@ -427,9 +439,10 @@ public void onFailure(final Exception e) { assertNotNull(renewedStats.getShards()); assertThat(renewedStats.getShards(), arrayWithSize(1)); assertNotNull(renewedStats.getShards()[0].getRetentionLeaseStats()); - assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); - final RetentionLease renewedRetentionLease = - renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + assertThat(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases(), hasSize(2)); + assertTrue(renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().contains( + ReplicationTracker.getPeerRecoveryRetentionLeaseId(renewedStats.getShards()[0].getShardRouting()))); + final RetentionLease renewedRetentionLease = renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().get(id); assertThat(renewedRetentionLease.id(), equalTo(id)); assertThat( renewedRetentionLease.retainingSequenceNumber(), @@ -484,7 +497,9 @@ public void onFailure(final Exception e) { assertNotNull(stats.getShards()); assertThat(stats.getShards(), arrayWithSize(1)); 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()))); } /* diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java index debb6d219a5f1..22edba58c9446 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java @@ -115,7 +115,8 @@ public void testRetentionLeasesSyncedOnAdd() throws Exception { retentionLock.close(); // 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 for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) { @@ -124,11 +125,13 @@ public void testRetentionLeasesSyncedOnAdd() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - final Map retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases()); + final Map retentionLeasesOnReplica + = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()); assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); // 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 void testRetentionLeaseSyncedOnRemove() throws Exception { retentionLock.close(); // 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 for (final ShardRouting replicaShard : clusterService().state().routingTable().index("index").shard(0).replicaShards()) { @@ -182,11 +186,13 @@ public void testRetentionLeaseSyncedOnRemove() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - final Map retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases()); + final Map retentionLeasesOnReplica = + RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()); assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); // 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 void testRetentionLeasesSyncOnExpiration() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .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 @@ -256,7 +263,8 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { // sleep long enough that the current retention lease has expired final long later = System.nanoTime(); 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 assertBusy(() -> { @@ -266,7 +274,8 @@ public void testRetentionLeasesSyncOnExpiration() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - assertThat(replica.getRetentionLeases().leases(), empty()); + assertThat( + RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()).entrySet(), empty()); } }); } @@ -432,11 +441,13 @@ public void testRetentionLeasesSyncOnRecovery() throws Exception { final IndexShard replica = internalCluster() .getInstance(IndicesService.class, replicaShardNodeName) .getShardOrNull(new ShardId(resolveIndex("index"), 0)); - final Map retentionLeasesOnReplica = RetentionLeases.toMap(replica.getRetentionLeases()); + final Map retentionLeasesOnReplica + = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()); assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); // 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 void testCanRenewRetentionLeaseUnderBlock() throws InterruptedException { * 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. */ - assertBusy(() -> assertThat(primary.loadRetentionLeases().leases(), contains(retentionLease.get()))); + assertBusy(() -> assertThat( + RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases()).values(), + contains(retentionLease.get()))); } catch (final Exception e) { failWithException(e); } @@ -593,7 +606,9 @@ public void testCanRenewRetentionLeaseWithoutWaitingForShards() throws Interrupt * 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. */ - assertBusy(() -> assertThat(primary.loadRetentionLeases().leases(), contains(retentionLease.get()))); + assertBusy(() -> assertThat( + RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(primary.loadRetentionLeases()).values(), + contains(retentionLease.get()))); } catch (final Exception e) { failWithException(e); } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java index adacf6539a80e..a568bd728418f 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java @@ -63,7 +63,8 @@ public void testRetentionLeaseStats() throws InterruptedException { final IndicesStatsResponse indicesStats = client().admin().indices().prepareStats("index").execute().actionGet(); assertThat(indicesStats.getShards(), arrayWithSize(1)); final RetentionLeaseStats retentionLeaseStats = indicesStats.getShards()[0].getRetentionLeaseStats(); - assertThat(RetentionLeases.toMap(retentionLeaseStats.retentionLeases()), equalTo(currentRetentionLeases)); + assertThat(RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases(retentionLeaseStats.retentionLeases()), + equalTo(currentRetentionLeases)); } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index 974e060bf2520..ed429bb680d7d 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseStats; import org.elasticsearch.index.seqno.RetentionLeases; @@ -35,14 +36,13 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; -import java.util.Collections; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; @@ -73,7 +73,7 @@ public void testAddOrRenewRetentionLease() throws IOException { indexShard.addRetentionLease( Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {})); 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++) { @@ -84,7 +84,7 @@ public void testAddOrRenewRetentionLease() throws IOException { length, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } @@ -105,7 +105,7 @@ public void testRemoveRetentionLease() throws IOException { indexShard.addRetentionLease( Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i, ActionListener.wrap(() -> {})); 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++) { @@ -115,7 +115,7 @@ public void testRemoveRetentionLease() throws IOException { length - i - 1, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } @@ -132,6 +132,12 @@ public void testExpirationOnReplica() throws IOException { 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 { final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis()); final Settings settings = Settings @@ -147,23 +153,28 @@ private void runExpirationTest(final boolean primary) throws IOException { try { final long[] retainingSequenceNumbers = new long[1]; retainingSequenceNumbers[0] = randomLongBetween(0, Long.MAX_VALUE); + final long initialVersion; if (primary) { + initialVersion = 2; indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> {})); } else { + initialVersion = 3; final RetentionLeases retentionLeases = new RetentionLeases( primaryTerm, - 1, - Collections.singleton(new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0"))); + initialVersion, + Arrays.asList( + peerRecoveryRetentionLease(indexShard), + new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0"))); indexShard.updateRetentionLeasesOnReplica(retentionLeases); } { final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); - assertThat(retentionLeases.version(), equalTo(1L)); - assertThat(retentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); + assertThat(retentionLeases.version(), equalTo(initialVersion)); + assertThat(retentionLeases.leases(), hasSize(2)); + final RetentionLease retentionLease = retentionLeases.get("0"); 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 @@ -174,28 +185,30 @@ private void runExpirationTest(final boolean primary) throws IOException { } else { final RetentionLeases retentionLeases = new RetentionLeases( primaryTerm, - 2, - Collections.singleton(new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0"))); + initialVersion + 1, + Arrays.asList( + peerRecoveryRetentionLease(indexShard), + new RetentionLease("0", retainingSequenceNumbers[0], currentTimeMillis.get(), "test-0"))); indexShard.updateRetentionLeasesOnReplica(retentionLeases); } { final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); - assertThat(retentionLeases.version(), equalTo(2L)); - assertThat(retentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = retentionLeases.leases().iterator().next(); + assertThat(retentionLeases.version(), equalTo(initialVersion + 1)); + assertThat(retentionLeases.leases(), hasSize(2)); + final RetentionLease retentionLease = retentionLeases.get("0"); 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 currentTimeMillis.set( currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); if (primary) { - assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, true, false); - assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, 3, true, true); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion + 1, true, false); + assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, initialVersion + 2, true, true); } else { - assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, 2, false, false); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, primaryTerm, initialVersion + 1, false, false); } } finally { closeShards(indexShard); @@ -229,13 +242,8 @@ public void testPersistence() throws IOException { // the written retention leases should equal our current retention leases final RetentionLeases retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); final RetentionLeases writtenRetentionLeases = indexShard.loadRetentionLeases(); - if (retentionLeases.leases().isEmpty()) { - assertThat(writtenRetentionLeases.version(), equalTo(0L)); - assertThat(writtenRetentionLeases.leases(), empty()); - } else { - assertThat(writtenRetentionLeases.version(), equalTo((long) length)); - assertThat(retentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0]))); - } + assertThat(writtenRetentionLeases.version(), equalTo(1L + length)); + assertThat(writtenRetentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0]))); // when we recover, we should recover the retention leases final IndexShard recoveredShard = reinitShard( @@ -244,15 +252,10 @@ public void testPersistence() throws IOException { try { recoverShardFromStore(recoveredShard); final RetentionLeases recoveredRetentionLeases = recoveredShard.getEngine().config().retentionLeasesSupplier().get(); - if (retentionLeases.leases().isEmpty()) { - assertThat(recoveredRetentionLeases.version(), equalTo(0L)); - assertThat(recoveredRetentionLeases.leases(), empty()); - } else { - assertThat(recoveredRetentionLeases.version(), equalTo((long) length)); - assertThat( - recoveredRetentionLeases.leases(), - contains(retentionLeases.leases().toArray(new RetentionLease[0]))); - } + assertThat(recoveredRetentionLeases.version(), equalTo(1L + length)); + assertThat( + recoveredRetentionLeases.leases(), + contains(retentionLeases.leases().toArray(new RetentionLease[0]))); } finally { closeShards(recoveredShard); } @@ -265,8 +268,10 @@ public void testPersistence() throws IOException { try { recoverShardFromStore(forceRecoveredShard); final RetentionLeases recoveredRetentionLeases = forceRecoveredShard.getEngine().config().retentionLeasesSupplier().get(); - assertThat(recoveredRetentionLeases.leases(), empty()); - assertThat(recoveredRetentionLeases.version(), equalTo(0L)); + assertThat(recoveredRetentionLeases.leases(), hasSize(1)); + assertThat(recoveredRetentionLeases.leases().iterator().next().id(), + equalTo(ReplicationTracker.getPeerRecoveryRetentionLeaseId(indexShard.routingEntry()))); + assertThat(recoveredRetentionLeases.version(), equalTo(1L)); } finally { closeShards(forceRecoveredShard); } @@ -291,8 +296,8 @@ public void testRetentionLeaseStats() throws IOException { stats.retentionLeases(), length, minimumRetainingSequenceNumbers, - length == 0 ? RetentionLeases.EMPTY.primaryTerm() : indexShard.getOperationPrimaryTerm(), - length); + indexShard.getOperationPrimaryTerm(), + length + 1); } finally { closeShards(indexShard); } @@ -355,7 +360,9 @@ private void assertRetentionLeases( assertThat(retentionLeases.version(), equalTo(version)); final Map idToRetentionLease = new HashMap<>(); 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)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 4e1a9a2ba396b..9482054486dc2 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -103,6 +103,8 @@ import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; 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.RetentionLeases; import org.elasticsearch.index.seqno.SeqNoStats; @@ -1556,14 +1558,17 @@ public String[] listAll() throws IOException { public void testRefreshMetric() throws IOException { 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(); // check time advances for (int i = 1; shard.refreshStats().getTotalTimeInMillis() == initialTotalTime; i++) { indexDoc(shard, "_doc", "test"); - assertThat(shard.refreshStats().getTotal(), equalTo(2L + i - 1)); + assertThat(shard.refreshStats().getTotal(), equalTo(initialRefreshes + i - 1)); shard.refresh("test"); - assertThat(shard.refreshStats().getTotal(), equalTo(2L + i)); + assertThat(shard.refreshStats().getTotal(), equalTo(initialRefreshes + i)); assertThat(shard.refreshStats().getTotalTimeInMillis(), greaterThanOrEqualTo(initialTotalTime)); } long refreshCount = shard.refreshStats().getTotal(); @@ -1590,18 +1595,18 @@ public void testExternalRefreshMetric() throws IOException { assertThat(shard.refreshStats().getExternalTotal(), equalTo(2L + i)); 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"); try (Engine.GetResult ignored = shard.get(new Engine.Get(true, false, "_doc", "test", new Term(IdFieldMapper.NAME, Uid.encodeId("test"))))) { 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"); shard.writeIndexingBuffer(); 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); } @@ -2109,7 +2114,8 @@ public void testPrimaryHandOffUpdatesLocalCheckpoint() throws IOException { } IndexShardTestCase.updateRoutingEntry(primarySource, 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()); recoverReplica(primaryTarget, primarySource, true); @@ -2860,7 +2866,7 @@ public void testCompletionStatsMarksSearcherAccessed() throws Exception { public void testDocStats() throws Exception { IndexShard indexShard = null; try { - indexShard = newStartedShard( + indexShard = newStartedShard(false, 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 numDocsToDelete = randomLongBetween(1, numDocs); @@ -2898,15 +2904,23 @@ public void testDocStats() throws Exception { deleteDoc(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()) { + final long newGlobalCheckpoint = indexShard.getLocalCheckpoint(); if (indexShard.routingEntry().primary()) { indexShard.updateLocalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), indexShard.getLocalCheckpoint()); indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(), indexShard.getLocalCheckpoint()); + indexShard.syncRetentionLeases(); } 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(); } @@ -3494,6 +3508,10 @@ public void testSegmentMemoryTrackedInBreaker() throws Exception { // 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. if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { + primary.updateGlobalCheckpointForShard( + primary.routingEntry().allocationId().getId(), + primary.getLastSyncedGlobalCheckpoint()); + primary.syncRetentionLeases(); primary.sync(); flushShard(primary); } @@ -4012,10 +4030,10 @@ public void testTypelessDelete() throws IOException { assertTrue(indexResult.isCreated()); DeleteResult deleteResult = shard.applyDeleteOperationOnPrimary(Versions.MATCH_ANY, "some_other_type", "id", VersionType.INTERNAL, - UNASSIGNED_SEQ_NO, 0); + UNASSIGNED_SEQ_NO, 1); 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()); closeShards(shard); @@ -4091,7 +4109,7 @@ public void testDoNotTrimCommitsWhenOpenReadOnlyEngine() throws Exception { final ShardRouting replicaRouting = shard.routingEntry(); ShardRouting readonlyShardRouting = newShardRouting(replicaRouting.shardId(), replicaRouting.currentNodeId(), true, 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()) { @Override protected void ensureMaxSeqNoEqualsToGlobalCheckpoint(SeqNoStats seqNoStats) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java index 481aaa233caed..617fffa6d1b16 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -124,10 +124,16 @@ public void testSyncerSendsOffCorrectDocuments() throws Exception { assertThat(resyncRequest.getMaxSeenAutoIdTimestampOnPrimary(), equalTo(shard.getMaxSeenAutoIdTimestamp())); } if (syncNeeded && globalCheckPoint < numDocs - 1) { - 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)); + if (shard.indexSettings.isSoftDeleteEnabled()) { + assertThat(resyncTask.getSkippedOperations(), equalTo(0)); + assertThat(resyncTask.getResyncedOperations(), equalTo(resyncTask.getTotalOperations())); + 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 { assertThat(resyncTask.getSkippedOperations(), equalTo(0)); assertThat(resyncTask.getResyncedOperations(), equalTo(0)); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java index 2933eba1659bd..9ab56df76d5c3 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -34,26 +34,41 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.PlainActionFuture; 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.metadata.IndexMetaData; 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.PeerRecoverySource; 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.service.ClusterService; +import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; 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.TokenFilterFactory; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.recovery.RecoveryStats; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; @@ -70,7 +85,9 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.Scope; +import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.store.MockFSIndexStore; import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.StubbableTransport; @@ -79,7 +96,6 @@ import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportService; -import org.junit.After; import java.io.IOException; import java.util.ArrayList; @@ -97,6 +113,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; +import java.util.stream.StreamSupport; import static java.util.Collections.singletonMap; import static java.util.stream.Collectors.toList; @@ -126,13 +143,21 @@ public class IndexRecoveryIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return Arrays.asList(MockTransportService.TestPlugin.class, MockFSIndexStore.TestPlugin.class, - RecoverySettingsChunkSizePlugin.class, TestAnalysisPlugin.class); + return Arrays.asList( + MockTransportService.TestPlugin.class, + MockFSIndexStore.TestPlugin.class, + RecoverySettingsChunkSizePlugin.class, + TestAnalysisPlugin.class, + InternalSettingsPlugin.class, + MockEngineFactoryPlugin.class); } - @After - public void assertConsistentHistoryInLuceneIndex() throws Exception { + @Override + protected void beforeIndexDeletion() throws Exception { + super.beforeIndexDeletion(); internalCluster().assertConsistentHistoryBetweenTranslogAndLuceneIndex(); + internalCluster().assertSeqNos(); + internalCluster().assertSameDocIdsOnShards(); } private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, RecoverySource recoverySource, boolean primary, @@ -852,6 +877,7 @@ public void testHistoryRetention() throws Exception { internalCluster().stopRandomNode(InternalTestCluster.nameFilter(secondNodeToStop)); final long desyncNanoTime = System.nanoTime(); + //noinspection StatementWithEmptyBody while (System.nanoTime() <= desyncNanoTime) { // time passes } @@ -991,6 +1017,231 @@ public void testRecoveryFlushReplica() throws Exception { 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 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 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 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 { final AtomicBoolean throwParsingError = new AtomicBoolean(); @Override @@ -1008,4 +1259,71 @@ public TokenStream create(TokenStream tokenStream) { } } + 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 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 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)); + } + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 1154ce99078b0..81eb2bc20eaa0 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -19,21 +19,25 @@ 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.IndexInput; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.cluster.metadata.IndexMetaData; 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.UUIDs; 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.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -41,93 +45,23 @@ import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; +import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; +import java.util.Optional; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.CyclicBarrier; import java.util.stream.Collectors; +import java.util.stream.LongStream; +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; 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 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 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 { IndexShard sourceShard = newStartedShard(true); int numDocs = between(20, 100); @@ -202,4 +136,146 @@ public void testWriteFileChunksConcurrently() throws Exception { assertThat(diff.different, empty()); closeShards(sourceShard, targetShard); } + + private SeqNoStats populateRandomData(IndexShard shard) throws IOException { + List 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 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 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); + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index f6e1de0233bf7..a175699ad2cf6 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -64,6 +64,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -78,6 +79,7 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.test.VersionUtils; import org.elasticsearch.threadpool.FixedExecutorBuilder; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; @@ -100,6 +102,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Consumer; import java.util.function.IntSupplier; import java.util.zip.CRC32; @@ -441,6 +444,18 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE ((ActionListener)invocation.getArguments()[0]).onResponse(() -> {}); return null; }).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 prepareTargetForTranslogCalled = new AtomicBoolean(); final AtomicBoolean phase2Called = new AtomicBoolean(); @@ -453,9 +468,10 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE between(1, 8)) { @Override - void phase1(IndexCommit snapshot, long globalCheckpoint, IntSupplier translogOps, ActionListener listener) { + void phase1(IndexCommit snapshot, Consumer> createRetentionLease, + IntSupplier translogOps, ActionListener listener) { phase1Called.set(true); - super.phase1(snapshot, globalCheckpoint, translogOps, listener); + super.phase1(snapshot, createRetentionLease, translogOps, listener); } @Override @@ -670,7 +686,9 @@ public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.Metada final StepListener phase1Listener = new StepListener<>(); try { 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)); latch.await(); phase1Listener.result(); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index cecd7dcd80a4c..1ef039ba168f5 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -79,7 +79,8 @@ public void testTranslogHistoryTransferred() throws Exception { shards.addReplica(); shards.startAll(); 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); } } @@ -294,7 +295,8 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { shards.recoverReplica(newReplica); // file based recovery should be made 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 assertThat(newReplica.getHistoryUUID(), equalTo(historyUUID)); @@ -410,7 +412,8 @@ public void testShouldFlushAfterPeerRecovery() throws Exception { shards.recoverReplica(replica); // Make sure the flushing will eventually be completed (eg. `shouldPeriodicallyFlush` is 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); } } diff --git a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java index 95c9b1adf6ada..eb0e564e92d33 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -1008,7 +1008,10 @@ private void assertCumulativeQueryCacheStats(IndicesStatsResponse response) { } 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()); indexRandom(false, true, client().prepareIndex("index", "type", "1").setSource("foo", "bar"), @@ -1052,6 +1055,13 @@ public void testFilterCacheStats() throws Exception { // we need to flush and make that commit safe so that the SoftDeletesPolicy can drop everything. if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { 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"); } ForceMergeResponse forceMergeResponse = diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index fd1eda21ce6cf..5f435f4f54bb3 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -50,6 +50,7 @@ import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; @@ -672,7 +673,7 @@ public EngineConfig config( SequenceNumbers.NO_OPS_PERFORMED, update -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> listener.onResponse(new ReplicationResponse())); globalCheckpointSupplier = replicationTracker; retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { diff --git a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index b91197bfe0851..2788cd0b2f1de 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -169,7 +169,7 @@ protected class ReplicationGroup implements AutoCloseable, Iterable private final AtomicInteger replicaId = new AtomicInteger(); private final AtomicInteger docId = new AtomicInteger(); boolean closed = false; - private ReplicationTargets replicationTargets; + private volatile ReplicationTargets replicationTargets; private final PrimaryReplicaSyncer primaryReplicaSyncer = new PrimaryReplicaSyncer( new TaskManager(Settings.EMPTY, threadPool, Collections.emptySet()), @@ -288,6 +288,7 @@ public synchronized int startReplicas(int numOfReplicasToStart) throws IOExcepti public void startPrimary() throws IOException { recoverPrimary(primary); + computeReplicationTargets(); HashSet activeIds = new HashSet<>(); activeIds.addAll(activeIds()); activeIds.add(primary.routingEntry().allocationId().getId()); @@ -528,7 +529,7 @@ private synchronized void computeReplicationTargets() { this.replicationTargets = new ReplicationTargets(this.primary, new ArrayList<>(this.replicas)); } - private synchronized ReplicationTargets getReplicationTargets() { + private ReplicationTargets getReplicationTargets() { return replicationTargets; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseUtils.java b/test/framework/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseUtils.java new file mode 100644 index 0000000000000..9b2eda120d8b9 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseUtils.java @@ -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 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 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 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]))); + } + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index a349f985b4e96..4b621e5fe5153 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -224,7 +224,12 @@ protected IndexShard newShard(boolean primary, Settings settings, EngineFactory } 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 @@ protected IndexShard reinitShard(IndexShard current, IndexingOperationListener.. * @param listeners new listerns to use for the newly created shard */ 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 * - * @param routing the shard routing to use for the newly created shard. - * @param listeners new listerns 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 indexMetaData the index metadata to use for the newly created 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 { closeShards(current); return newShard( routing, current.shardPath(), - current.indexSettings().getIndexMetaData(), + indexMetaData, null, null, engineFactory, @@ -623,18 +629,9 @@ protected final void recoverUnstartedReplica(final IndexShard replica, } replica.prepareForIndexRecovery(); final RecoveryTarget recoveryTarget = targetSupplier.apply(replica, pNode); - final String targetAllocationId = recoveryTarget.indexShard().routingEntry().allocationId().getId(); - - final Store.MetadataSnapshot snapshot = getMetadataSnapshotOrEmpty(replica); - 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 long startingSeqNo = recoveryTarget.indexShard().recoverLocallyUpToGlobalCheckpoint(); + final StartRecoveryRequest request = PeerRecoveryTargetService.getStartRecoveryRequest( + logger, rNode, recoveryTarget, startingSeqNo); final RecoverySourceHandler recovery = new RecoverySourceHandler(primary, new AsyncRecoveryTarget(recoveryTarget, threadPool.generic()), threadPool, request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java index 1f4a35a29c28e..fdb623d1d1e91 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalSettingsPlugin.java @@ -50,6 +50,7 @@ public List> getSettings() { PROVIDED_NAME_SETTING, TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING, IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING, + IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING, IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING ); } diff --git a/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java b/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java index cb54248ee3dbc..c9c74e658f4fd 100644 --- a/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java +++ b/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.test.rest.yaml.ObjectPath; import java.io.IOException; @@ -24,7 +25,6 @@ import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; @@ -228,7 +228,9 @@ public void testForgetFollower() throws IOException { final Map shardStatsAsMap = (Map) shardsStats.get(0); final Map retentionLeasesStats = (Map) shardStatsAsMap.get("retention_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)); + } } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java index 2f0aed395a73c..2f6af23f7ad92 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java @@ -36,7 +36,7 @@ import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseActions; 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.IndexShardClosedException; import org.elasticsearch.indices.IndicesService; @@ -97,20 +97,11 @@ public List> getSettings() { } - public static final class RetentionLeaseSyncIntervalSettingPlugin extends Plugin { - - @Override - public List> getSettings() { - return Collections.singletonList(IndexService.RETENTION_LEASE_SYNC_INTERVAL_SETTING); - } - - } - @Override protected Collection> nodePlugins() { return Stream.concat( super.nodePlugins().stream(), - Stream.of(RetentionLeaseRenewIntervalSettingPlugin.class, RetentionLeaseSyncIntervalSettingPlugin.class)) + Stream.of(RetentionLeaseRenewIntervalSettingPlugin.class)) .collect(Collectors.toList()); } @@ -189,10 +180,10 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); + final RetentionLease retentionLease = currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex))); } }); @@ -310,7 +301,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws */ assertBusy(() -> { // sample the leases after recovery - final List retentionLeases = new ArrayList<>(); + final List< Map> retentionLeases = new ArrayList<>(); assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = @@ -320,13 +311,15 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases + = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final ClusterStateResponse followerIndexClusterState = followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); final String expectedRetentionLeaseId = retentionLeaseId( getFollowerCluster().getClusterName(), new Index(followerIndex, followerUUID), @@ -353,16 +346,17 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws continue; } assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final ClusterStateResponse followerIndexClusterState = followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); // 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 void testUnfollowRemovesRetentionLeases() throws Exception { leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); final List shardsStats = getShardsStats(stats); for (final ShardStats shardStats : shardsStats) { - assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); - assertThat( - shardStats.getRetentionLeaseStats().retentionLeases().leases().iterator().next().id(), - equalTo(retentionLeaseId)); + final Map retentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardStats.getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardStats), retentionLeases.values(), hasSize(1)); + 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 @@ -454,7 +448,8 @@ public void testUnfollowRemovesRetentionLeases() throws Exception { leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); final List afterUnfollowShardsStats = getShardsStats(afterUnfollowStats); 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 { for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) { @@ -605,10 +600,11 @@ public void testRetentionLeaseAdvancesWhileFollowing() throws Exception { final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex))); // we assert that retention leases are being advanced assertThat( @@ -665,7 +661,7 @@ public void testRetentionLeaseRenewalIsCancelledWhenFollowingIsPaused() throws E */ assertBusy(() -> { // sample the leases after pausing - final List retentionLeases = new ArrayList<>(); + final List> retentionLeases = new ArrayList<>(); assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = @@ -675,13 +671,15 @@ public void testRetentionLeaseRenewalIsCancelledWhenFollowingIsPaused() throws E final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases + = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final ClusterStateResponse followerIndexClusterState = followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); final String expectedRetentionLeaseId = retentionLeaseId( getFollowerCluster().getClusterName(), new Index(followerIndex, followerUUID), @@ -708,16 +706,17 @@ public void testRetentionLeaseRenewalIsCancelledWhenFollowingIsPaused() throws E continue; } assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final ClusterStateResponse followerIndexClusterState = followerClient().admin().cluster().prepareState().clear().setMetaData(true).setIndices(followerIndex).get(); final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); // 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 void onResponseReceived( final List afterUnfollowShardsStats = getShardsStats(afterUnfollowStats); for (final ShardStats shardStats : afterUnfollowShardsStats) { assertNotNull(shardStats.getRetentionLeaseStats()); - assertThat(Strings.toString(shardStats), shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty()); + assertThat(Strings.toString(shardStats), RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardStats.getRetentionLeaseStats().retentionLeases()).values(), empty()); } } finally { for (final DiscoveryNode senderNode : followerClusterState.getState().nodes()) { @@ -975,7 +975,8 @@ public void testForgetFollower() throws Exception { final List afterForgetFollowerShardsStats = getShardsStats(afterForgetFollowerStats); for (final ShardStats shardStats : afterForgetFollowerShardsStats) { 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 @@ private void assertRetentionLeaseRenewal( final String followerIndex, final String leaderIndex) throws Exception { // ensure that a retention lease has been put in place on each shard, and grab a copy of them - final List retentionLeases = new ArrayList<>(); + final List> retentionLeases = new ArrayList<>(); assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = @@ -995,10 +996,11 @@ private void assertRetentionLeaseRenewal( final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex))); retentionLeases.add(currentRetentionLeases); } @@ -1013,13 +1015,14 @@ private void assertRetentionLeaseRenewal( final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { assertNotNull(shardsStats.get(i).getRetentionLeaseStats()); - final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.leases(), hasSize(1)); + final Map currentRetentionLeases = RetentionLeaseUtils.toMapExcludingPeerRecoveryRetentionLeases( + shardsStats.get(i).getRetentionLeaseStats().retentionLeases()); + assertThat(Strings.toString(shardsStats.get(i)), currentRetentionLeases.values(), hasSize(1)); final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + currentRetentionLeases.values().iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex))); // 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())); } }); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java index 89a5e8045c17e..0c421cef21975 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java @@ -65,7 +65,9 @@ import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; @@ -1187,8 +1189,10 @@ private void runFallBehindTest( final CheckedRunnable afterPausingFollower, final Consumer> exceptionConsumer) throws Exception { final int numberOfPrimaryShards = randomIntBetween(1, 3); - final String leaderIndexSettings = getIndexSettings(numberOfPrimaryShards, between(0, 1), - singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); + final Map extraSettingsMap = new HashMap<>(2); + 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)); ensureLeaderYellow("index1"); @@ -1221,6 +1225,15 @@ private void runFallBehindTest( leaderClient().prepareDelete("index1", "doc", "1").get(); leaderClient().admin().indices().refresh(new RefreshRequest("index1")).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.maxNumSegments(1); leaderClient().admin().indices().forceMerge(forceMergeRequest).actionGet(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesTests.java index f42a50b91ff02..16e7a90c3c679 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/ShardChangesTests.java @@ -15,17 +15,20 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.LocalStateCcr; import java.util.Collection; -import java.util.Collections; import java.util.concurrent.CountDownLatch; 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.equalTo; @@ -35,7 +38,7 @@ public class ShardChangesTests extends ESSingleNodeTestCase { @Override protected Collection> 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 @@ -102,7 +105,8 @@ public void testMissingOperations() throws Exception { .put("index.soft_deletes.enabled", true) .put("index.soft_deletes.retention.operations", 0) .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(); for (int i = 0; i < 32; i++) { @@ -111,6 +115,15 @@ public void testMissingOperations() throws Exception { client().admin().indices().flush(new FlushRequest("index").force(true)).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.maxNumSegments(1); client().admin().indices().forceMerge(forceMergeRequest).actionGet(); diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexRecoveryTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexRecoveryTests.java index b67258dd9b3d0..d091564c7dc54 100644 --- a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexRecoveryTests.java +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexRecoveryTests.java @@ -3,41 +3,92 @@ * 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.client.Client; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; 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 FrozenIndexRecoveryTests 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 - ), 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); +import org.elasticsearch.common.util.set.Sets; +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.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +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 ESIntegTestCase { + + @Override + protected boolean addMockInternalEngine() { + return false; + } + + @Override + protected Collection> nodePlugins() { + List> 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 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()); + } + } } } diff --git a/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexShardTests.java b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexShardTests.java new file mode 100644 index 0000000000000..e3f49fdc2aac3 --- /dev/null +++ b/x-pack/plugin/frozen-indices/src/test/java/org/elasticsearch/index/engine/FrozenIndexShardTests.java @@ -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); + } +}