Skip to content

Commit 01ad532

Browse files
committed
Create peer-recovery retention leases
This creates a peer-recovery retention lease for every shard during recovery, ensuring that the replication group retains history for future peer recoveries. It also ensures that leases for active shard copies do not expire, and leases for inactive shard copies expire immediately if the shard is fully-allocated. Relates elastic#41536
1 parent 4757870 commit 01ad532

21 files changed

+588
-144
lines changed

server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java

Lines changed: 123 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import org.elasticsearch.Version;
2525
import org.elasticsearch.action.ActionListener;
2626
import org.elasticsearch.action.support.replication.ReplicationResponse;
27+
import org.elasticsearch.cluster.metadata.IndexMetaData;
2728
import org.elasticsearch.cluster.routing.AllocationId;
2829
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
2930
import org.elasticsearch.cluster.routing.ShardRouting;
@@ -217,10 +218,22 @@ public synchronized Tuple<Boolean, RetentionLeases> getRetentionLeases(final boo
217218
// the primary calculates the non-expired retention leases and syncs them to replicas
218219
final long currentTimeMillis = currentTimeMillisSupplier.getAsLong();
219220
final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis();
221+
final Set<String> leaseIdsForCurrentPeers
222+
= routingTable.assignedShards().stream().map(ReplicationTracker::getPeerRecoveryRetentionLeaseId).collect(Collectors.toSet());
220223
final Map<Boolean, List<RetentionLease>> partitionByExpiration = retentionLeases
221224
.leases()
222225
.stream()
223-
.collect(Collectors.groupingBy(lease -> currentTimeMillis - lease.timestamp() > retentionLeaseMillis));
226+
.collect(Collectors.groupingBy(lease -> {
227+
if (lease.source().equals(PEER_RECOVERY_RETENTION_LEASE_SOURCE)) {
228+
if (leaseIdsForCurrentPeers.contains(lease.id())) {
229+
return false;
230+
}
231+
if (routingTable.allShardsStarted()) {
232+
return true;
233+
}
234+
}
235+
return currentTimeMillis - lease.timestamp() > retentionLeaseMillis;
236+
}));
224237
final Collection<RetentionLease> expiredLeases = partitionByExpiration.get(true);
225238
if (expiredLeases == null) {
226239
// early out as no retention leases have expired
@@ -242,7 +255,7 @@ public synchronized Tuple<Boolean, RetentionLeases> getRetentionLeases(final boo
242255
* @param source the source of the retention lease
243256
* @param listener the callback when the retention lease is successfully added and synced to replicas
244257
* @return the new retention lease
245-
* @throws IllegalArgumentException if the specified retention lease already exists
258+
* @throws RetentionLeaseAlreadyExistsException if the specified retention lease already exists
246259
*/
247260
public RetentionLease addRetentionLease(
248261
final String id,
@@ -253,30 +266,46 @@ public RetentionLease addRetentionLease(
253266
final RetentionLease retentionLease;
254267
final RetentionLeases currentRetentionLeases;
255268
synchronized (this) {
256-
assert primaryMode;
257-
if (retentionLeases.contains(id)) {
258-
throw new RetentionLeaseAlreadyExistsException(id);
259-
}
260-
retentionLease = new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source);
261-
logger.debug("adding new retention lease [{}] to current retention leases [{}]", retentionLease, retentionLeases);
262-
retentionLeases = new RetentionLeases(
263-
operationPrimaryTerm,
264-
retentionLeases.version() + 1,
265-
Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList()));
269+
retentionLease = innerAddRetentionLease(id, retainingSequenceNumber, source);
266270
currentRetentionLeases = retentionLeases;
267271
}
268272
onSyncRetentionLeases.accept(currentRetentionLeases, listener);
269273
return retentionLease;
270274
}
271275

276+
/**
277+
* Adds a new retention lease, but does not synchronise it with the rest of the replication group.
278+
*
279+
* @param id the identifier of the retention lease
280+
* @param retainingSequenceNumber the retaining sequence number
281+
* @param source the source of the retention lease
282+
* @return the new retention lease
283+
* @throws RetentionLeaseAlreadyExistsException if the specified retention lease already exists
284+
*/
285+
private RetentionLease innerAddRetentionLease(String id, long retainingSequenceNumber, String source) {
286+
assert Thread.holdsLock(this);
287+
assert primaryMode : id + "/" + retainingSequenceNumber + "/" + source;
288+
if (retentionLeases.contains(id)) {
289+
throw new RetentionLeaseAlreadyExistsException(id);
290+
}
291+
final RetentionLease retentionLease
292+
= new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source);
293+
logger.debug("adding new retention lease [{}] to current retention leases [{}]", retentionLease, retentionLeases);
294+
retentionLeases = new RetentionLeases(
295+
operationPrimaryTerm,
296+
retentionLeases.version() + 1,
297+
Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList()));
298+
return retentionLease;
299+
}
300+
272301
/**
273302
* Renews an existing retention lease.
274303
*
275304
* @param id the identifier of the retention lease
276305
* @param retainingSequenceNumber the retaining sequence number
277306
* @param source the source of the retention lease
278307
* @return the renewed retention lease
279-
* @throws IllegalArgumentException if the specified retention lease does not exist
308+
* @throws RetentionLeaseNotFoundException if the specified retention lease does not exist
280309
*/
281310
public synchronized RetentionLease renewRetentionLease(final String id, final long retainingSequenceNumber, final String source) {
282311
assert primaryMode;
@@ -390,6 +419,45 @@ public boolean assertRetentionLeasesPersisted(final Path path) throws IOExceptio
390419
return true;
391420
}
392421

422+
423+
/**
424+
* Retention leases for peer recovery have source {@link ReplicationTracker#PEER_RECOVERY_RETENTION_LEASE_SOURCE}, a lease ID
425+
* containing the persistent node ID calculated by {@link ReplicationTracker#getPeerRecoveryRetentionLeaseId}, and retain operations
426+
* with sequence numbers strictly greater than the given global checkpoint.
427+
*/
428+
public void addPeerRecoveryRetentionLease(String nodeId, long globalCheckpoint, ActionListener<ReplicationResponse> listener) {
429+
addRetentionLease(getPeerRecoveryRetentionLeaseId(nodeId), globalCheckpoint + 1, PEER_RECOVERY_RETENTION_LEASE_SOURCE, listener);
430+
}
431+
432+
/**
433+
* Source for peer recovery retention leases; see {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
434+
*/
435+
public static final String PEER_RECOVERY_RETENTION_LEASE_SOURCE = "peer recovery";
436+
437+
/**
438+
* Id for a peer recovery retention lease for the given node. See {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
439+
*/
440+
static String getPeerRecoveryRetentionLeaseId(String nodeId) {
441+
return "peer_recovery/" + nodeId;
442+
}
443+
444+
/**
445+
* Id for a peer recovery retention lease for the given {@link ShardRouting}.
446+
* See {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
447+
*/
448+
public static String getPeerRecoveryRetentionLeaseId(ShardRouting shardRouting) {
449+
return getPeerRecoveryRetentionLeaseId(shardRouting.currentNodeId());
450+
}
451+
452+
/**
453+
* Renew the peer-recovery retention lease for the given shard, advancing the retained sequence number to discard operations up to the
454+
* given global checkpoint. See {@link ReplicationTracker#addPeerRecoveryRetentionLease}.
455+
*/
456+
public void renewPeerRecoveryRetentionLease(ShardRouting shardRouting, long globalCheckpoint) {
457+
assert primaryMode;
458+
renewRetentionLease(getPeerRecoveryRetentionLeaseId(shardRouting), globalCheckpoint + 1, PEER_RECOVERY_RETENTION_LEASE_SOURCE);
459+
}
460+
393461
public static class CheckpointState implements Writeable {
394462

395463
/**
@@ -616,6 +684,22 @@ private boolean invariant() {
616684
assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked";
617685
}
618686

687+
if (primaryMode
688+
&& indexSettings.isSoftDeleteEnabled()
689+
&& indexSettings.getIndexMetaData().getState() == IndexMetaData.State.OPEN
690+
&& indexSettings.getIndexVersionCreated().onOrAfter(Version.V_8_0_0)) {
691+
// all tracked shard copies have a corresponding peer-recovery retention lease
692+
for (final ShardRouting shardRouting : routingTable.assignedShards()) {
693+
assert checkpoints.get(shardRouting.allocationId().getId()).tracked == false
694+
|| retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) :
695+
"no retention lease for tracked shard " + shardRouting + " in " + retentionLeases;
696+
assert shardRouting.relocating() == false
697+
|| checkpoints.get(shardRouting.allocationId().getRelocationId()).tracked == false
698+
|| retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting.getTargetRelocatingShard()))
699+
: "no retention lease for relocation target " + shardRouting + " in " + retentionLeases;
700+
}
701+
}
702+
619703
return true;
620704
}
621705

@@ -669,6 +753,7 @@ public ReplicationTracker(
669753
this.pendingInSync = new HashSet<>();
670754
this.routingTable = null;
671755
this.replicationGroup = null;
756+
assert Version.V_EMPTY.equals(indexSettings.getIndexVersionCreated()) == false;
672757
assert invariant();
673758
}
674759

@@ -772,6 +857,31 @@ public synchronized void activatePrimaryMode(final long localCheckpoint) {
772857
primaryMode = true;
773858
updateLocalCheckpoint(shardAllocationId, checkpoints.get(shardAllocationId), localCheckpoint);
774859
updateGlobalCheckpointOnPrimary();
860+
861+
if (indexSettings.isSoftDeleteEnabled()) {
862+
final ShardRouting primaryShard = routingTable.primaryShard();
863+
final String leaseId = getPeerRecoveryRetentionLeaseId(primaryShard);
864+
if (retentionLeases.get(leaseId) == null) {
865+
/*
866+
* We might have got here here via a rolling upgrade from an older version that doesn't create peer recovery retention
867+
* leases for every shard copy, but in this case we do not expect any leases to exist.
868+
*/
869+
if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_8_0_0)) {
870+
// We are starting up the whole replication group from scratch: if we were not (i.e. this is a replica promotion) then
871+
// this copy must already be in-sync and active and therefore holds a retention lease for itself.
872+
assert routingTable.activeShards().equals(Collections.singletonList(primaryShard)) : routingTable.activeShards();
873+
assert primaryShard.allocationId().getId().equals(shardAllocationId)
874+
: routingTable.activeShards() + " vs " + shardAllocationId;
875+
assert replicationGroup.getReplicationTargets().equals(Collections.singletonList(primaryShard));
876+
877+
// Safe to call innerAddRetentionLease() without a subsequent sync since there are no other members of this replication
878+
// group.
879+
innerAddRetentionLease(leaseId, Math.max(0L, checkpoints.get(shardAllocationId).globalCheckpoint + 1),
880+
PEER_RECOVERY_RETENTION_LEASE_SOURCE);
881+
}
882+
}
883+
}
884+
775885
assert invariant();
776886
}
777887

server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -211,4 +211,7 @@ public String toString() {
211211
'}';
212212
}
213213

214+
public boolean isNotPeerRecoveryRetentionLease() {
215+
return ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(source) == false;
216+
}
214217
}

server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncer.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -44,7 +44,7 @@ public interface RetentionLeaseSyncer {
4444
RetentionLeaseSyncer EMPTY = new RetentionLeaseSyncer() {
4545
@Override
4646
public void sync(final ShardId shardId, final RetentionLeases retentionLeases, final ActionListener<ReplicationResponse> listener) {
47-
47+
listener.onResponse(new ReplicationResponse());
4848
}
4949

5050
@Override

server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -275,13 +275,20 @@ private static Map<String, RetentionLease> toMap(final Collection<RetentionLease
275275
}
276276

277277
/**
278-
* A utility method to convert a retention lease collection to a map from retention lease ID to retention lease.
278+
* A utility method to convert a retention lease collection to a map from retention lease ID to retention lease and exclude
279+
* the automatically-added peer-recovery retention leases
279280
*
280281
* @param retentionLeases the retention lease collection
281282
* @return the map from retention lease ID to retention lease
282283
*/
283-
static Map<String, RetentionLease> toMap(final RetentionLeases retentionLeases) {
284-
return retentionLeases.leases;
284+
public static Map<String, RetentionLease> toMapExcludingPeerRecoveryRetentionLeases(final RetentionLeases retentionLeases) {
285+
return retentionLeases.leases.values().stream()
286+
.filter(l -> ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(l.source()) == false)
287+
.collect(Collectors.toMap(RetentionLease::id, Function.identity(),
288+
(o1, o2) -> {
289+
throw new AssertionError("unexpectedly merging " + o1 + " and " + o2);
290+
},
291+
LinkedHashMap::new));
285292
}
286293

287294
}

server/src/main/java/org/elasticsearch/index/shard/IndexShard.java

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2415,6 +2415,20 @@ public boolean isRelocatedPrimary() {
24152415
return replicationTracker.isRelocated();
24162416
}
24172417

2418+
public void addPeerRecoveryRetentionLease(String nodeId, long globalCheckpoint, ActionListener<ReplicationResponse> listener) {
2419+
assert assertPrimaryMode();
2420+
replicationTracker.addPeerRecoveryRetentionLease(nodeId, globalCheckpoint, listener);
2421+
}
2422+
2423+
/**
2424+
* Test-only method to advance the primary's peer-recovery retention lease so that operations up to the global checkpoint can be
2425+
* discarded. TODO Remove this when retention leases are advanced by other mechanisms.
2426+
*/
2427+
public void advancePrimaryPeerRecoveryRetentionLeaseToGlobalCheckpoint() {
2428+
assert assertPrimaryMode();
2429+
replicationTracker.renewPeerRecoveryRetentionLease(routingEntry(), getGlobalCheckpoint());
2430+
}
2431+
24182432
class ShardEventListener implements Engine.EventListener {
24192433
private final CopyOnWriteArrayList<Consumer<ShardFailure>> delegates = new CopyOnWriteArrayList<>();
24202434

server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java

Lines changed: 24 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -32,6 +32,8 @@
3232
import org.elasticsearch.ExceptionsHelper;
3333
import org.elasticsearch.action.ActionListener;
3434
import org.elasticsearch.action.StepListener;
35+
import org.elasticsearch.action.support.replication.ReplicationResponse;
36+
import org.elasticsearch.cluster.metadata.IndexMetaData;
3537
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
3638
import org.elasticsearch.cluster.routing.ShardRouting;
3739
import org.elasticsearch.common.CheckedSupplier;
@@ -49,6 +51,7 @@
4951
import org.elasticsearch.index.engine.Engine;
5052
import org.elasticsearch.index.engine.RecoveryEngineException;
5153
import org.elasticsearch.index.seqno.LocalCheckpointTracker;
54+
import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException;
5255
import org.elasticsearch.index.seqno.RetentionLeases;
5356
import org.elasticsearch.index.seqno.SequenceNumbers;
5457
import org.elasticsearch.index.shard.IndexShard;
@@ -188,10 +191,28 @@ public void recoverToTarget(ActionListener<RecoveryResponse> listener) {
188191
}
189192
assert startingSeqNo >= 0 : "startingSeqNo must be non negative. got: " + startingSeqNo;
190193

194+
final StepListener<ReplicationResponse> establishRetentionLeaseStep = new StepListener<>();
195+
if (shard.indexSettings().isSoftDeleteEnabled()
196+
&& shard.indexSettings().getIndexMetaData().getState() != IndexMetaData.State.CLOSE) {
197+
runUnderPrimaryPermit(() -> {
198+
try {
199+
// blindly create the lease. TODO integrate this with the recovery process
200+
shard.addPeerRecoveryRetentionLease(request.targetNode().getId(), startingSeqNo - 1, establishRetentionLeaseStep);
201+
} catch (RetentionLeaseAlreadyExistsException e) {
202+
logger.debug("peer-recovery retention lease already exists", e);
203+
establishRetentionLeaseStep.onResponse(null);
204+
}
205+
}, shardId + " establishing retention lease for [" + request.targetAllocationId() + "]", shard, cancellableThreads, logger);
206+
} else {
207+
establishRetentionLeaseStep.onResponse(null);
208+
}
209+
191210
final StepListener<TimeValue> prepareEngineStep = new StepListener<>();
192-
// For a sequence based recovery, the target can keep its local translog
193-
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false,
194-
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep);
211+
establishRetentionLeaseStep.whenComplete(r -> {
212+
// For a sequence based recovery, the target can keep its local translog
213+
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false,
214+
shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep);
215+
}, onFailure);
195216
final StepListener<SendSnapshotResult> sendSnapshotStep = new StepListener<>();
196217
prepareEngineStep.whenComplete(prepareEngineTime -> {
197218
/*

0 commit comments

Comments
 (0)