From 4d4b0dff7a5c7fe026aa419dd3898032983c1a4d Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 19 Feb 2019 16:59:45 +0000 Subject: [PATCH 01/32] Retain history for peer recovery using leases Today if soft deletes are enabled then we discard history below the global checkpoint, damaging our chances of being able to perform an operations-based peer recovery. This change has each shard copy obtain a history retention lease to retain history that is not included in its local safe commit, making an operations-based peer recovery much more likely. --- .../replication/ReplicationOperation.java | 4 + .../TransportReplicationAction.java | 37 +++-- .../org/elasticsearch/index/IndexService.java | 47 +++++- .../elasticsearch/index/engine/Engine.java | 4 +- .../index/engine/InternalEngine.java | 44 ++--- .../index/engine/ReadOnlyEngine.java | 5 + .../index/engine/SoftDeletesPolicy.java | 6 +- ...erRecoveryRetentionLeaseRenewalAction.java | 155 ++++++++++++++++++ .../index/seqno/ReplicationTracker.java | 134 ++++++++++++--- .../RetentionLeaseBackgroundSyncAction.java | 7 +- .../index/seqno/RetentionLeaseSyncAction.java | 20 ++- .../index/seqno/RetentionLeaseSyncer.java | 2 +- .../index/seqno/RetentionLeases.java | 15 +- .../elasticsearch/index/shard/IndexShard.java | 57 ++++++- .../elasticsearch/indices/IndicesService.java | 6 +- .../cluster/IndicesClusterStateService.java | 18 +- .../recovery/RecoverySourceHandler.java | 66 ++++++-- .../gateway/RecoveryFromGatewayIT.java | 22 +++ .../index/engine/InternalEngineTests.java | 8 +- .../RecoveryDuringReplicationTests.java | 10 ++ ...ReplicationTrackerRetentionLeaseTests.java | 64 +++++--- .../index/seqno/ReplicationTrackerTests.java | 100 ++++++----- .../seqno/RetentionLeaseActionsTests.java | 72 ++++++-- .../index/seqno/RetentionLeaseIT.java | 27 ++- .../index/seqno/RetentionLeaseStatsTests.java | 4 +- .../index/shard/IndexShardIT.java | 3 +- .../shard/IndexShardRetentionLeaseTests.java | 68 +++++--- .../index/shard/IndexShardTests.java | 54 +++++- .../shard/PrimaryReplicaSyncerTests.java | 14 +- .../RemoveCorruptedShardDataCommandTests.java | 5 +- ...dicesLifecycleListenerSingleNodeTests.java | 2 +- ...actIndicesClusterStateServiceTestCase.java | 3 +- ...ClusterStateServiceRandomUpdatesTests.java | 3 +- .../indices/recovery/IndexRecoveryIT.java | 28 +++- .../indices/recovery/RecoveryTests.java | 9 +- .../indices/stats/IndexStatsIT.java | 27 +++ .../BlobStoreRepositoryRestoreTests.java | 2 + .../snapshots/SnapshotResiliencyTests.java | 5 +- .../ESIndexLevelReplicationTestCase.java | 25 ++- .../index/shard/IndexShardTestCase.java | 20 ++- .../elasticsearch/test/ESIntegTestCase.java | 6 +- 41 files changed, 948 insertions(+), 260 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 0da39a593a2c1..3445d78557920 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -155,6 +155,9 @@ private void performOnReplicas(final ReplicaRequest replicaRequest, final long g } } + protected void handleReplicaResponse(final ShardRouting shard, final ReplicaResponse response) { + } + private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest, final long globalCheckpoint, final long maxSeqNoOfUpdatesOrDeletes) { if (logger.isTraceEnabled()) { @@ -177,6 +180,7 @@ public void onResponse(ReplicaResponse response) { final String message = String.format(Locale.ROOT, "primary failed updating local checkpoint for replica %s", shard); primary.failShard(message, e); } + handleReplicaResponse(shard, response); decPendingAndFinishIfNeeded(); } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index a8c187745ac4a..7fae080288c9c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -32,6 +32,7 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; +import org.elasticsearch.action.support.replication.ReplicationOperation.ReplicaResponse; import org.elasticsearch.client.transport.NoNodeAvailableException; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; @@ -475,11 +476,19 @@ public void onFailure(Exception e) { protected ReplicationOperation> createReplicatedOperation( Request request, ActionListener> listener, PrimaryShardReference primaryShardReference) { - return new ReplicationOperation<>(request, primaryShardReference, listener, - newReplicasProxy(primaryTerm), logger, actionName); + return new ReplicationOperation> + (request, primaryShardReference, listener, newReplicasProxy(primaryTerm), logger, actionName) { + @Override + protected void handleReplicaResponse(ShardRouting shard, ReplicaResponse response) { + TransportReplicationAction.this.handleReplicaResponse(shard, response); + } + }; } } + protected void handleReplicaResponse(ShardRouting shard, ReplicationOperation.ReplicaResponse response) { + } + protected static class PrimaryResult, Response extends ReplicationResponse> implements ReplicationOperation.PrimaryResult { @@ -543,6 +552,10 @@ public void respond(ActionListener listener) { listener.onFailure(finalFailure); } } + + public ReplicaResponse getReplicaResponse(IndexShard replica) { + return new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint()); + } } public class ReplicaOperationTransportHandler implements TransportRequestHandler> { @@ -619,9 +632,7 @@ public void onResponse(Releasable releasable) { try { final ReplicaResult replicaResult = shardOperationOnReplica(request, replica); releasable.close(); // release shard operation lock before responding to caller - final TransportReplicationAction.ReplicaResponse response = - new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint()); - replicaResult.respond(new ResponseListener(response)); + replicaResult.respond(new ResponseListener(replicaResult.getReplicaResponse(replica))); } catch (final Exception e) { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller AsyncReplicaAction.this.onFailure(e); @@ -1073,7 +1084,7 @@ public static class ReplicaResponse extends ActionResponse implements Replicatio private long localCheckpoint; private long globalCheckpoint; - ReplicaResponse() { + public ReplicaResponse() { } @@ -1217,6 +1228,12 @@ public void onFailure(Exception shardFailedError) { } } + protected ReplicaResponse readReplicaResponse(StreamInput in) throws IOException { + ReplicaResponse replicaResponse = new ReplicaResponse(); + replicaResponse.readFrom(in); + return replicaResponse; + } + /** * Sends the specified replica request to the specified node. * @@ -1228,12 +1245,8 @@ protected void sendReplicaRequest( final ConcreteReplicaRequest replicaRequest, final DiscoveryNode node, final ActionListener listener) { - final ActionListenerResponseHandler handler = new ActionListenerResponseHandler<>(listener, in -> { - ReplicaResponse replicaResponse = new ReplicaResponse(); - replicaResponse.readFrom(in); - return replicaResponse; - }); - transportService.sendRequest(node, transportReplicaAction, replicaRequest, transportOptions, handler); + transportService.sendRequest(node, transportReplicaAction, replicaRequest, transportOptions, + new ActionListenerResponseHandler<>(listener, this::readReplicaResponse)); } /** a wrapper class to encapsulate a request when being sent to a specific allocation id **/ diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index e3d8732c7fb81..7774a5bb044d8 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -122,6 +122,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private volatile AsyncTranslogFSync fsyncTask; private volatile AsyncGlobalCheckpointTask globalCheckpointTask; private volatile AsyncRetentionLeaseSyncTask retentionLeaseSyncTask; + private volatile AsyncPeerRecoveryRetentionLeaseRenewalTask peerRecoveryRetentionLeaseRenewalTask; // don't convert to Setting<> and register... we only set this in tests and register via a plugin private final String INDEX_TRANSLOG_RETENTION_CHECK_INTERVAL_SETTING = "index.translog.retention.check_interval"; @@ -199,6 +200,7 @@ public IndexService( this.trimTranslogTask = new AsyncTrimTranslogTask(this); this.globalCheckpointTask = new AsyncGlobalCheckpointTask(this); this.retentionLeaseSyncTask = new AsyncRetentionLeaseSyncTask(this); + this.peerRecoveryRetentionLeaseRenewalTask = new AsyncPeerRecoveryRetentionLeaseRenewalTask(this); rescheduleFsyncTask(indexSettings.getTranslogDurability()); } @@ -289,7 +291,8 @@ public synchronized void close(final String reason, boolean delete) throws IOExc fsyncTask, trimTranslogTask, globalCheckpointTask, - retentionLeaseSyncTask); + retentionLeaseSyncTask, + peerRecoveryRetentionLeaseRenewalTask); } } } @@ -317,8 +320,10 @@ private long getAvgShardSizeInBytes() throws IOException { public synchronized IndexShard createShard( final ShardRouting routing, final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer) throws IOException { + final RetentionLeaseSyncer retentionLeaseSyncer, + final Consumer peerRecoveryRetentionLeaseRenewer) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); + Objects.requireNonNull(peerRecoveryRetentionLeaseRenewer); /* * TODO: we execute this in parallel but it's a synced method. Yet, we might * be able to serialize the execution via the cluster state in the future. for now we just @@ -408,7 +413,8 @@ public synchronized IndexShard createShard( indexingOperationListeners, () -> globalCheckpointSyncer.accept(shardId), retentionLeaseSyncer, - circuitBreakerService); + circuitBreakerService, + () -> peerRecoveryRetentionLeaseRenewer.accept(shardId)); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); @@ -793,6 +799,10 @@ private void syncRetentionLeases() { sync(IndexShard::syncRetentionLeases, "retention lease"); } + private void renewPeerRecoveryRetentionLeases() { + sync(IndexShard::renewPeerRecoveryRetentionLeases, "peer recovery retention leases"); + } + private void sync(final Consumer sync, final String source) { for (final IndexShard shard : this.shards.values()) { if (shard.routingEntry().active() && shard.routingEntry().primary()) { @@ -932,6 +942,15 @@ public String toString() { Property.Dynamic, Property.IndexScope); + // this setting is intentionally not registered, it is only used in tests + public static final Setting RETENTION_LEASE_PEER_RECOVERY_SYNC_INTERVAL_SETTING = + Setting.timeSetting( + "index.soft_deletes.retention_lease.peer_recovery.sync_interval", + new TimeValue(5, TimeUnit.MINUTES), + new TimeValue(0, TimeUnit.MILLISECONDS), + Property.Dynamic, + Property.IndexScope); + /** * Background task that syncs the global checkpoint to replicas. */ @@ -981,6 +1000,28 @@ public String toString() { } + final class AsyncPeerRecoveryRetentionLeaseRenewalTask extends BaseAsyncTask { + + AsyncPeerRecoveryRetentionLeaseRenewalTask(final IndexService indexService) { + super(indexService, RETENTION_LEASE_PEER_RECOVERY_SYNC_INTERVAL_SETTING.get(indexService.getIndexSettings().getSettings())); + } + + @Override + protected void runInternal() { + indexService.renewPeerRecoveryRetentionLeases(); + } + + @Override + protected String getThreadPool() { + return ThreadPool.Names.MANAGEMENT; + } + + @Override + public String toString() { + return "peer_recovery_retention_lease_sync"; + } + } + AsyncRefreshTask getRefreshTask() { // for tests return refreshTask; } 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 dbe779864fe47..e619e0883c99f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -264,6 +264,8 @@ protected final DocsStats docsStats(IndexReader indexReader) { return new DocsStats(numDocs, numDeletedDocs, sizeInBytes); } + public abstract long getLocalCheckpointOfSafeCommit(); + /** * Performs the pre-closing checks on the {@link Engine}. * @@ -766,7 +768,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 32354ab4b16d7..ed02821655f2d 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -497,11 +497,16 @@ 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 { - return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return newChangesSnapshot(source, mapperService, Math.max(0, startingSeqNo), Long.MAX_VALUE, false); + } else { + return getTranslog().newSnapshotFromMinSeqNo(startingSeqNo); + } } /** @@ -547,6 +552,11 @@ public long getWritingBytes() { return indexWriter.getFlushingBytes() + versionMap.getRefreshingBytes(); } + @Override + public long getLocalCheckpointOfSafeCommit() { + return softDeletesPolicy.getLocalCheckpointOfSafeCommit(); + } + /** * Reads the current stored translog ID from the last commit data. */ @@ -2534,17 +2544,21 @@ public Translog.Snapshot newChangesSnapshot(String source, MapperService mapperS @Override public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException { - final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); - final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); - try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { - Translog.Operation operation; - while ((operation = snapshot.next()) != null) { - if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - tracker.markSeqNoAsCompleted(operation.seqNo()); + if (engineConfig.getIndexSettings().isSoftDeleteEnabled()) { + return getMinRetainedSeqNo() <= startingSeqNo; + } else { + final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint(); + final LocalCheckpointTracker tracker = new LocalCheckpointTracker(startingSeqNo, startingSeqNo - 1); + try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(startingSeqNo)) { + Translog.Operation operation; + while ((operation = snapshot.next()) != null) { + if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } } } + return tracker.getCheckpoint() >= currentLocalCheckpoint; } - return tracker.getCheckpoint() >= currentLocalCheckpoint; } /** @@ -2559,15 +2573,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/engine/ReadOnlyEngine.java b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java index c464a34e78b01..9d58f8b5612cc 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/ReadOnlyEngine.java @@ -450,6 +450,11 @@ public DocsStats docStats() { return docsStats; } + @Override + public long getLocalCheckpointOfSafeCommit() { + return seqNoStats.getMaxSeqNo(); + } + @Override public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) { diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java index 4c9ee0be92f46..8529eb9a8b2f9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -38,7 +38,7 @@ */ final class SoftDeletesPolicy { private final LongSupplier globalCheckpointSupplier; - private long localCheckpointOfSafeCommit; + private volatile long localCheckpointOfSafeCommit; // This lock count is used to prevent `minRetainedSeqNo` from advancing. private int retentionLockCount; // The extra number of operations before the global checkpoint are retained @@ -80,6 +80,10 @@ synchronized void setLocalCheckpointOfSafeCommit(long newCheckpoint) { this.localCheckpointOfSafeCommit = newCheckpoint; } + long getLocalCheckpointOfSafeCommit() { + return localCheckpointOfSafeCommit; + } + /** * Acquires a lock on soft-deleted documents to prevent them from cleaning up in merge processes. This is necessary to * make sure that all operations that are being retained will be retained until the lock is released. diff --git a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java new file mode 100644 index 0000000000000..0fb0d79e5a6a6 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java @@ -0,0 +1,155 @@ +/* + * 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.ActionFilters; +import org.elasticsearch.action.support.replication.ReplicationOperation; +import org.elasticsearch.action.support.replication.ReplicationRequest; +import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.action.support.replication.TransportReplicationAction; +import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.seqno.PeerRecoveryRetentionLeaseRenewalAction.Request; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.threadpool.ThreadPool.Names; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; + +/** + * Background action to renew retention leases held to ensure that enough history is retained to perform a peer recovery if needed. This + * action renews the leases for each copy of the shard, advancing the corresponding sequence number, and thereby releases any operations + * that are now contained in a safe commit on every copy since they are no longer needed. + */ +public class PeerRecoveryRetentionLeaseRenewalAction extends TransportReplicationAction { + + public static final String ACTION_NAME = "indices:admin/seq_no/peer_recovery_retention_lease_renewal"; + + @Inject + public PeerRecoveryRetentionLeaseRenewalAction( + final Settings settings, + final TransportService transportService, + final ClusterService clusterService, + final IndicesService indicesService, + final ThreadPool threadPool, + final ShardStateAction shardStateAction, + final ActionFilters actionFilters, + final IndexNameExpressionResolver indexNameExpressionResolver) { + + super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters, + indexNameExpressionResolver, Request::new, Request::new, Names.MANAGEMENT); + } + + @Override + protected ReplicationResponse newResponseInstance() { + return new ReplicationResponse(); + } + + @Override + protected PrimaryResult shardOperationOnPrimary(Request shardRequest, IndexShard primary) { + primary.renewPeerRecoveryRetentionLeaseForPrimary(); + return new PrimaryResult<>(shardRequest, new ReplicationResponse()); + } + + @Override + protected ReplicaResponse readReplicaResponse(StreamInput in) throws IOException { + return new ShardCopyResponse(in); + } + + @Override + protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard replica) { + return new ReplicaResult() { + @Override + public ReplicaResponse getReplicaResponse(IndexShard replica) { + return new ShardCopyResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint(), + replica.getLocalCheckpointOfSafeCommit()); + } + }; + } + + @Override + protected void handleReplicaResponse(ShardRouting shard, ReplicationOperation.ReplicaResponse response) { + assert response instanceof ShardCopyResponse : response.getClass(); + final ShardCopyResponse shardCopyResponse = (ShardCopyResponse) response; // TODO introduce type parameter rather than cast here + indicesService.indexServiceSafe(shard.index()).getShard(shard.id()) + .renewPeerRecoveryRetentionLeaseForReplica(shard, shardCopyResponse.localCheckpointOfSafeCommit); + } + + public void renewPeerRecoveryRetentionLease(ShardId shardId) { + execute(new Request(shardId), new ActionListener() { + @Override + public void onResponse(ReplicationResponse response) { + } + + @Override + public void onFailure(Exception e) { + } + }); + } + + static final class ShardCopyResponse extends ReplicaResponse { + private long localCheckpointOfSafeCommit; + + ShardCopyResponse(long localCheckpoint, long globalCheckpoint, long localCheckpointOfSafeCommit) { + super(localCheckpoint, globalCheckpoint); + this.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; + } + + ShardCopyResponse(StreamInput in) throws IOException { + super(); + super.readFrom(in); + localCheckpointOfSafeCommit = in.readLong(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeLong(localCheckpointOfSafeCommit); + } + + @Override + public void readFrom(StreamInput in) { + throw new UnsupportedOperationException("use Writable not Streamable"); + } + } + + static final class Request extends ReplicationRequest { + Request() { + } + + Request(ShardId shardId) { + super(shardId); + } + + @Override + public String toString() { + return "request for minimum seqno needed for peer recovery for " + shardId; + } + } +} 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 566a81b3af4b0..f45525408840d 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -43,7 +43,6 @@ import java.io.IOException; import java.nio.file.Path; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -60,6 +59,10 @@ import java.util.stream.LongStream; import java.util.stream.Stream; +import static java.lang.Math.max; +import static java.util.Collections.singletonList; +import static org.elasticsearch.action.ActionListener.wrap; + /** * This class is responsible for tracking the replication group with its progress and safety markers (local and global checkpoints). * @@ -189,6 +192,16 @@ public RetentionLeases getRetentionLeases() { return getRetentionLeases(false).v2(); } + public static final String PEER_RECOVERY_RETENTION_LEASE_SOURCE = "peer recovery"; + + static String getPeerRecoveryRetentionLeaseId(String nodeId) { + return "peer_recovery/" + nodeId; + } + + public static String getPeerRecoveryRetentionLeaseId(ShardRouting shardRouting) { + return getPeerRecoveryRetentionLeaseId(shardRouting.currentNodeId()); + } + /** * If the expire leases parameter is false, gets all retention leases tracked on this shard and otherwise first calculates * expiration of existing retention leases, and then gets all non-expired retention leases tracked on this shard. Note that only the @@ -205,18 +218,28 @@ 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)); - if (partitionByExpiration.get(true) == null) { - // early out as no retention leases have expired - return Tuple.tuple(false, retentionLeases); - } - final Collection nonExpiredLeases = - partitionByExpiration.get(false) != null ? partitionByExpiration.get(false) : Collections.emptyList(); - retentionLeases = new RetentionLeases(operationPrimaryTerm, retentionLeases.version() + 1, nonExpiredLeases); - return Tuple.tuple(true, retentionLeases); + .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 nonExpiredLeases = partitionByExpiration.get(false); + retentionLeases = new RetentionLeases(operationPrimaryTerm, retentionLeases.version() + 1, nonExpiredLeases.stream() + .map(lease -> leaseIdsForCurrentPeers.contains(lease.id()) ? + new RetentionLease(lease.id(), lease.retainingSequenceNumber(), currentTimeMillis, lease.source()) : lease) + .collect(Collectors.toList())); + return Tuple.tuple(partitionByExpiration.containsKey(true), retentionLeases); } /** @@ -238,21 +261,38 @@ public RetentionLease addRetentionLease( final RetentionLease retentionLease; final RetentionLeases currentRetentionLeases; synchronized (this) { - assert primaryMode; - if (retentionLeases.contains(id)) { - throw new RetentionLeaseAlreadyExistsException(id); - } - retentionLease = new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source); - retentionLeases = new RetentionLeases( - operationPrimaryTerm, - retentionLeases.version() + 1, - Stream.concat(retentionLeases.leases().stream(), Stream.of(retentionLease)).collect(Collectors.toList())); - currentRetentionLeases = retentionLeases; + retentionLease = innerAddRetentionLease(id, retainingSequenceNumber, source); + currentRetentionLeases = this.retentionLeases; } 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 IllegalArgumentException 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); + } + // should we abort if we have already discarded operations >= retainingSequenceNumber? + final RetentionLease retentionLease + = new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source); + 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. * @@ -321,6 +361,30 @@ public synchronized void updateRetentionLeasesOnReplica(final RetentionLeases re } } + public synchronized void renewPeerRecoveryRetentionLease(ShardRouting shardRouting, long localCheckpointOfSafeCommit) { + assert primaryMode; + final String leaseId = getPeerRecoveryRetentionLeaseId(shardRouting); + final RetentionLease retentionLease = retentionLeases.get(leaseId); + if (retentionLease == null) { + assert routingTable.activeShards().contains(shardRouting) == false : "no retention lease found for current " + shardRouting; + logger.debug("attempted to renew peer recovery retention lease for unknown {}", shardRouting); + } else if (retentionLease.retainingSequenceNumber() < localCheckpointOfSafeCommit + 1) { + renewRetentionLease(leaseId, localCheckpointOfSafeCommit + 1, PEER_RECOVERY_RETENTION_LEASE_SOURCE); + } + } + + public void addPeerRecoveryRetentionLease(String nodeId, long startingSeqNo, ActionListener listener) { + addRetentionLease(getPeerRecoveryRetentionLeaseId(nodeId), max(0L, startingSeqNo), + PEER_RECOVERY_RETENTION_LEASE_SOURCE, wrap(r -> listener.onResponse(null), listener::onFailure)); + } + + public synchronized boolean peerRetentionLeasesNeedRenewal(long localCheckpointOfSafeCommit) { + assert primaryMode; + final RetentionLeases retentionLeases = getRetentionLeases(); + return routingTable.activeShards().stream().anyMatch( + sr -> retentionLeases.get(getPeerRecoveryRetentionLeaseId(sr)).retainingSequenceNumber() < localCheckpointOfSafeCommit + 1); + } + /** * Loads the latest retention leases from their dedicated state file. * @@ -596,6 +660,17 @@ private boolean invariant() { assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked"; } + if (checkpoints.get(shardAllocationId).inSync + && (primaryMode || shardAllocationId.equals(routingTable.primaryShard().allocationId().getId()) == false)) { + // a newly-recovered primary creates its own retention lease when entering primaryMode, which is done later, so it doesn't + // exist yet + + for (ShardRouting shardRouting : routingTable.activeShards()) { + assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : + "no retention lease for active shard " + shardRouting + " in " + retentionLeases + " on " + shardAllocationId; + } + } + return true; } @@ -742,16 +817,31 @@ private void updateGlobalCheckpoint(final String allocationId, final long global /** * Initializes the global checkpoint tracker in primary mode (see {@link #primaryMode}. Called on primary activation or promotion. */ - public synchronized void activatePrimaryMode(final long localCheckpoint) { + public synchronized void activatePrimaryMode(final long localCheckpoint, final long localCheckpointOfSafeCommit) { assert invariant(); assert primaryMode == false; assert checkpoints.get(shardAllocationId) != null && checkpoints.get(shardAllocationId).inSync && checkpoints.get(shardAllocationId).localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO : "expected " + shardAllocationId + " to have initialized entry in " + checkpoints + " when activating primary"; - assert localCheckpoint >= SequenceNumbers.NO_OPS_PERFORMED; + assert localCheckpointOfSafeCommit >= SequenceNumbers.NO_OPS_PERFORMED : localCheckpointOfSafeCommit; + assert localCheckpoint >= localCheckpointOfSafeCommit : localCheckpoint + " < " + localCheckpointOfSafeCommit; primaryMode = true; updateLocalCheckpoint(shardAllocationId, checkpoints.get(shardAllocationId), localCheckpoint); updateGlobalCheckpointOnPrimary(); + + final ShardRouting primaryShard = routingTable.primaryShard(); + final String leaseId = getPeerRecoveryRetentionLeaseId(primaryShard); + if (retentionLeases.get(leaseId) == null) { + // We are starting up the whole replication group from scratch: if we were not (i.e. this is a replica promotion) then + // this copy must already be in-sync and active and therefore holds a retention lease for itself. + assert routingTable.activeShards().equals(singletonList(primaryShard)) : routingTable.activeShards(); + assert primaryShard.allocationId().getId().equals(shardAllocationId) : routingTable.activeShards() + " vs " + shardAllocationId; + assert replicationGroup.getReplicationTargets().equals(singletonList(primaryShard)); + + // Safe to call innerAddRetentionLease() without a subsequent sync because there are no other members of this replication gp. + innerAddRetentionLease(leaseId, max(0L, localCheckpointOfSafeCommit + 1), PEER_RECOVERY_RETENTION_LEASE_SOURCE); + } + assert invariant(); } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java index 4033dcf0c4bef..4d500e5a1f502 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java @@ -26,10 +26,12 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; @@ -107,8 +109,10 @@ public void backgroundSync( try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the sync is authorized threadContext.markAsSystemContext(); + final Request request = new Request(shardId, retentionLeases); + request.waitForActiveShards(ActiveShardCount.ONE); execute( - new Request(shardId, retentionLeases), + request, ActionListener.wrap( r -> {}, e -> { @@ -184,5 +188,4 @@ public String toString() { protected ReplicationResponse newResponseInstance() { return new ReplicationResponse(); } - } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index 760271e53ee1e..ae44339441ce8 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -26,11 +26,13 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.WriteResponse; import org.elasticsearch.action.support.replication.ReplicatedWriteRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportWriteAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; @@ -107,8 +109,9 @@ public void sync( try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the sync is authorized threadContext.markAsSystemContext(); - execute( - new RetentionLeaseSyncAction.Request(shardId, retentionLeases), + final Request request = new Request(shardId, retentionLeases); + request.waitForActiveShards(ActiveShardCount.ONE); + execute(request, ActionListener.wrap( listener::onResponse, e -> { @@ -197,4 +200,17 @@ protected Response newResponseInstance() { return new Response(); } + @Override + protected ClusterBlockLevel globalBlockLevel() { + // see e.g. ShrinkIndexIT#testCreateShrinkIndexFails which tries to relocate shards in a read-only index, needing updates to the + // peer recovery retention leases. TODO orly? + return null; + } + + @Override + protected ClusterBlockLevel indexBlockLevel() { + // see e.g. ShrinkIndexIT#testCreateShrinkIndexFails which tries to relocate shards in a read-only index, needing updates to the + // peer recovery retention leases. TODO orly? + return null; + } } 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 3bad887282502..b13419d34601c 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeases.java @@ -38,6 +38,8 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.elasticsearch.index.seqno.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; + /** * Represents a versioned collection of retention leases. We version the collection of retention leases to ensure that sync requests that * arrive out of order on the replica, using the version to ensure that older sync requests are rejected. @@ -260,13 +262,18 @@ private static Map toMap(final Collection toMap(final RetentionLeases retentionLeases) { - return retentionLeases.leases; + static Map toMapExcludingPeerRecoveryRetentionLeases(final RetentionLeases retentionLeases) { + return retentionLeases.leases.values().stream().filter(l -> 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)); } - } 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 7a5ec6bd28685..682770fcd6e2c 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -47,6 +47,7 @@ import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.RecoverySource.SnapshotRecoverySource; +import org.elasticsearch.cluster.routing.RecoverySource.Type; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.CheckedRunnable; @@ -140,6 +141,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.threadpool.ThreadPool.Names; import java.io.Closeable; import java.io.IOException; @@ -217,6 +219,8 @@ Runnable getGlobalCheckpointSyncer() { private final RetentionLeaseSyncer retentionLeaseSyncer; + private final Runnable peerRecoveryRetentionLeaseRenewer; + @Nullable private RecoveryState recoveryState; @@ -275,7 +279,8 @@ public IndexShard( final List listeners, final Runnable globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final CircuitBreakerService circuitBreakerService) throws IOException { + final CircuitBreakerService circuitBreakerService, + final Runnable peerRecoveryRetentionLeaseRenewer) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); this.shardRouting = shardRouting; @@ -329,6 +334,7 @@ public IndexShard( threadPool::absoluteTimeInMillis, (retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, retentionLeases, listener)); this.replicationTracker = replicationTracker; + this.peerRecoveryRetentionLeaseRenewer = peerRecoveryRetentionLeaseRenewer; // the query cache is a node-level thing, however we want the most popular filters // to be computed on a per-shard basis @@ -478,7 +484,7 @@ public void updateShardState(final ShardRouting newRouting, if (newPrimaryTerm == pendingPrimaryTerm) { if (currentRouting.initializing() && currentRouting.isRelocationTarget() == false && newRouting.active()) { // the master started a recovering primary, activate primary mode. - replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + replicationTracker.activatePrimaryMode(getLocalCheckpoint(), getLocalCheckpointOfSafeCommit()); } } else { assert currentRouting.primary() == false : "term is only increased as part of primary promotion"; @@ -520,7 +526,7 @@ public void updateShardState(final ShardRouting newRouting, ", current routing: " + currentRouting + ", new routing: " + newRouting; assert getOperationPrimaryTerm() == newPrimaryTerm; try { - replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + replicationTracker.activatePrimaryMode(getLocalCheckpoint(), getLocalCheckpointOfSafeCommit()); /* * 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 @@ -1433,7 +1439,8 @@ private void innerOpenEngineAndTranslog() throws IOException { final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint"); - updateRetentionLeasesOnReplica(loadRetentionLeases()); + updateRetentionLeasesOnReplica( + recoveryState.getRecoverySource().getType() == Type.EXISTING_STORE ? loadRetentionLeases() : RetentionLeases.EMPTY); trimUnsafeCommits(); synchronized (mutex) { verifyNotClosed(); @@ -1907,10 +1914,10 @@ public RetentionLeases getRetentionLeases() { } /** - * If the expire leases parameter is false, gets all retention leases tracked on this shard and otherwise first calculates - * expiration of existing retention leases, and then gets all non-expired retention leases tracked on this shard. Note that only the - * primary shard calculates which leases are expired, and if any have expired, syncs the retention leases to any replicas. If the - * expire leases parameter is true, this replication tracker must be in primary mode. + * If the expire leases parameter is false, gets all retention leases tracked on this shard and otherwise first calculates expiration of + * existing retention leases, renews all peer-recovery retention leases for active shard copies, and then gets all non-expired retention + * leases tracked on this shard. Note that only the primary shard calculates which leases are expired, and if any have expired, syncs + * the retention leases to any replicas. If the expire leases parameter is true, this replication tracker must be in primary mode. * * @return a tuple indicating whether or not any retention leases were expired, and the non-expired retention leases */ @@ -2419,6 +2426,40 @@ public boolean isRelocatedPrimary() { return replicationTracker.isRelocated(); } + public void addPeerRecoveryRetentionLease(String nodeId, long startingSeqNo, ActionListener listener) { + assert assertPrimaryMode(); + replicationTracker.addPeerRecoveryRetentionLease(nodeId, startingSeqNo, listener); + } + + public void renewPeerRecoveryRetentionLeaseForReplica(ShardRouting shardRouting, long localCheckpointOfSafeCommit) { + assert shardRouting.primary() == false : shardRouting; + runUnderPrimaryPermit(() -> replicationTracker.renewPeerRecoveryRetentionLease(shardRouting, localCheckpointOfSafeCommit), + e -> logger.debug(new ParameterizedMessage("exception renewing peer-recovery retention lease for {}", shardRouting), e), + Names.SAME, Tuple.tuple(shardRouting, localCheckpointOfSafeCommit)); + } + + public void renewPeerRecoveryRetentionLeaseForPrimary() { + assert assertPrimaryMode(); + // already running under primary permit + assert indexShardOperationPermits.getActiveOperationsCount() != 0 : "no operation permit for " + routingEntry(); + replicationTracker.renewPeerRecoveryRetentionLease(routingEntry(), getLocalCheckpointOfSafeCommit()); + } + + public void renewPeerRecoveryRetentionLeases() { + assert assertPrimaryMode(); + if (replicationTracker.peerRetentionLeasesNeedRenewal(getLocalCheckpointOfSafeCommit())) { + peerRecoveryRetentionLeaseRenewer.run(); + } + } + + public long getLocalCheckpointOfSafeCommit() { + final Engine engine = getEngineOrNull(); + if (engine == null) { + throw new ElasticsearchException("minimum sequence number for peer recovery is unavailable"); + } + return engine.getLocalCheckpointOfSafeCommit(); + } + class ShardEventListener implements Engine.EventListener { private final CopyOnWriteArrayList> delegates = new CopyOnWriteArrayList<>(); diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index f22bbbd471f69..e6a84083ebbc2 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -628,11 +628,13 @@ public IndexShard createShard( final RepositoriesService repositoriesService, final Consumer onShardFailure, final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer) throws IOException { + final RetentionLeaseSyncer retentionLeaseSyncer, + final Consumer peerRecoveryRetentionLeaseRenewer) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); ensureChangesAllowed(); IndexService indexService = indexService(shardRouting.index()); - IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer, retentionLeaseSyncer); + IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer, retentionLeaseSyncer, + peerRecoveryRetentionLeaseRenewer); indexShard.addShardFailureCallback(onShardFailure); indexShard.startRecovery(recoveryState, recoveryTargetService, recoveryListener, repositoriesService, (type, mapping) -> { diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index cceefc13d59dd..27fab9d1a5222 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -56,6 +56,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; +import org.elasticsearch.index.seqno.PeerRecoveryRetentionLeaseRenewalAction; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; @@ -127,6 +128,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final PrimaryReplicaSyncer primaryReplicaSyncer; private final Consumer globalCheckpointSyncer; private final RetentionLeaseSyncer retentionLeaseSyncer; + private final Consumer peerRecoveryRetentionLeaseRenewer; @Inject public IndicesClusterStateService( @@ -145,7 +147,8 @@ public IndicesClusterStateService( final PrimaryReplicaSyncer primaryReplicaSyncer, final GlobalCheckpointSyncAction globalCheckpointSyncAction, final RetentionLeaseSyncAction retentionLeaseSyncAction, - final RetentionLeaseBackgroundSyncAction retentionLeaseBackgroundSyncAction) { + final RetentionLeaseBackgroundSyncAction retentionLeaseBackgroundSyncAction, + final PeerRecoveryRetentionLeaseRenewalAction peerRecoveryRetentionLeaseRenewalAction) { this( settings, (AllocatedIndices>) indicesService, @@ -174,7 +177,7 @@ public void sync( public void backgroundSync(final ShardId shardId, final RetentionLeases retentionLeases) { Objects.requireNonNull(retentionLeaseBackgroundSyncAction).backgroundSync(shardId, retentionLeases); } - }); + }, peerRecoveryRetentionLeaseRenewalAction::renewPeerRecoveryRetentionLease); } // for tests @@ -193,7 +196,8 @@ public void backgroundSync(final ShardId shardId, final RetentionLeases retentio final SnapshotShardsService snapshotShardsService, final PrimaryReplicaSyncer primaryReplicaSyncer, final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer) { + final RetentionLeaseSyncer retentionLeaseSyncer, + final Consumer peerRecoveryRetentionLeaseRenewer) { this.settings = settings; this.buildInIndexListener = Arrays.asList( @@ -213,6 +217,7 @@ public void backgroundSync(final ShardId shardId, final RetentionLeases retentio this.globalCheckpointSyncer = globalCheckpointSyncer; this.retentionLeaseSyncer = Objects.requireNonNull(retentionLeaseSyncer); this.sendRefreshMapping = settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); + this.peerRecoveryRetentionLeaseRenewer = peerRecoveryRetentionLeaseRenewer; } @Override @@ -602,7 +607,8 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR repositoriesService, failedShardHandler, globalCheckpointSyncer, - retentionLeaseSyncer); + retentionLeaseSyncer, + peerRecoveryRetentionLeaseRenewer); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); } @@ -907,6 +913,7 @@ U createIndex(IndexMetaData indexMetaData, * @param onShardFailure a callback when this shard fails * @param globalCheckpointSyncer a callback when this shard syncs the global checkpoint * @param retentionLeaseSyncer a callback when this shard syncs retention leases + * @param peerRecoveryRetentionLeaseRenewer a callback when this shard renews peer recovery retention leases for each copy * @return a new shard * @throws IOException if an I/O exception occurs when creating the shard */ @@ -918,7 +925,8 @@ T createShard( RepositoriesService repositoriesService, Consumer onShardFailure, Consumer globalCheckpointSyncer, - RetentionLeaseSyncer retentionLeaseSyncer) throws IOException; + RetentionLeaseSyncer retentionLeaseSyncer, + Consumer peerRecoveryRetentionLeaseRenewer) throws IOException; /** * Returns shard for the specified id if it exists otherwise returns null. 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 e7a8fbfb523a1..d053f3f360c76 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -29,6 +29,7 @@ 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; @@ -50,6 +51,8 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.RetentionLease; +import org.elasticsearch.index.seqno.RetentionLeaseAlreadyExistsException; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -78,6 +81,7 @@ import java.util.function.Supplier; import java.util.stream.StreamSupport; +import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; /** @@ -144,29 +148,43 @@ public void recoverToTarget(ActionListener listener) { final Consumer onFailure = e -> IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); + final SetOnce retentionLease = new SetOnce<>(); runUnderPrimaryPermit(() -> { final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); - ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); + final ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); if (targetShardRouting == null) { logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), request.targetNode()); throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); } - assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; + assert targetShardRouting.initializing() + : "expected recovery target to be initializing but was " + targetShardRouting; + retentionLease.set(shard.getRetentionLeases().get(getPeerRecoveryRetentionLeaseId(targetShardRouting))); }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - final Closeable retentionLock = shard.acquireRetentionLock(); + + final Closeable retentionLock = retentionLease.get() == null ? () -> {} : shard.acquireRetentionLock(); resources.add(retentionLock); final long startingSeqNo; final long requiredSeqNoRangeStart; + final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && - isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); + isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()) + && (shard.indexSettings().isSoftDeleteEnabled() == false || retentionLease.get() != null); + final SendFileResult sendFileResult; + final StepListener addPeerRecoveryRetentionLeaseStep = new StepListener<>(); if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); requiredSeqNoRangeStart = startingSeqNo; sendFileResult = SendFileResult.EMPTY; + + if (shard.indexSettings().isSoftDeleteEnabled()) { + assert retentionLease.get() != null; + assert retentionLease.get().retainingSequenceNumber() <= request.startingSeqNo() + : retentionLease.get() + " vs " + request.startingSeqNo(); + } } else { final Engine.IndexCommitRef phase1Snapshot; try { @@ -177,9 +195,10 @@ public void recoverToTarget(ActionListener listener) { // We must have everything above the local checkpoint in the commit requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; - // 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; + // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have + // the same history on the target. However, with translog, 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 = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); sendFileResult = phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); @@ -197,10 +216,35 @@ public void recoverToTarget(ActionListener listener) { assert requiredSeqNoRangeStart >= startingSeqNo : "requiredSeqNoRangeStart [" + requiredSeqNoRangeStart + "] is lower than [" + startingSeqNo + "]"; + if (retentionLease.get() == null) { + assert isSequenceNumberBasedRecovery == false; + runUnderPrimaryPermit(() -> { + try { + shard.addPeerRecoveryRetentionLease(request.targetNode().getId(), startingSeqNo, addPeerRecoveryRetentionLeaseStep); + } catch (RetentionLeaseAlreadyExistsException e) { + logger.debug("peer-recovery retention lease somehow exists now", e); + addPeerRecoveryRetentionLeaseStep.onFailure(e); + } + }, shardId + " adding peer-recovery retention lease for " + request.targetNode(), shard, cancellableThreads, logger); + } else { + assert retentionLease.get().retainingSequenceNumber() <= requiredSeqNoRangeStart + : shard.shardId() + ": seqno " + requiredSeqNoRangeStart + " should be retained by " + retentionLease.get(); + // The target shard has a lease (and it is now in the routing table so its lease will not expire) + addPeerRecoveryRetentionLeaseStep.onResponse(null); + } + + addPeerRecoveryRetentionLeaseStep.whenComplete(r -> { + // we can release the retention lock here because the retention lease now reatils the required operations. + retentionLock.close(); + }, onFailure); + final StepListener prepareEngineStep = new StepListener<>(); - // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, - shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep); + addPeerRecoveryRetentionLeaseStep.whenComplete(r -> { + // For a sequence based recovery, the target can keep its local translog + prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo), prepareEngineStep); + }, onFailure); + final StepListener sendSnapshotStep = new StepListener<>(); prepareEngineStep.whenComplete(prepareEngineTime -> { /* @@ -225,8 +269,6 @@ public void recoverToTarget(ActionListener listener) { } 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(); // 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(); diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 4b0e431c66352..e249d4d1b4cd5 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -52,6 +52,7 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; @@ -73,6 +74,7 @@ import static org.elasticsearch.gateway.GatewayService.RECOVER_AFTER_NODES_SETTING; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.termQuery; +import static org.elasticsearch.index.seqno.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; @@ -81,6 +83,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; @ClusterScope(numDataNodes = 0, scope = Scope.TEST) public class RecoveryFromGatewayIT extends ESIntegTestCase { @@ -465,11 +468,30 @@ 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(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) ).get(); client(primaryNode).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(); + // expire retention lease for replica; since number_of_replicas is 0 it is no longer needed + internalCluster().getInstance(IndicesService.class, primaryNode).indexServiceSafe(resolveIndex("test", primaryNode)) + .forEach(is -> { + is.renewPeerRecoveryRetentionLeases(); + is.syncRetentionLeases(); + try { + assertBusy(() -> { + assertThat(is.getRetentionLeases().leases().stream() + .filter(l -> PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(l.source())).count(), equalTo(1L)); + assertEquals(is.getMinRetainedSeqNo(), is.getLocalCheckpointOfSafeCommit() + 1); + }); + } catch (Exception e) { + throw new AssertionError(e); + } + }); } + client(primaryNode).admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 1) + ).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 8d0865a652578..aba1a5a4c36f3 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -170,10 +170,12 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.shuffle; +import static org.elasticsearch.action.ActionListener.wrap; import static org.elasticsearch.index.engine.Engine.Operation.Origin.LOCAL_TRANSLOG_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PEER_RECOVERY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; +import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; @@ -2300,9 +2302,13 @@ public void testSeqNoAndCheckpoints() throws IOException { TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED); 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).build(), Collections.emptySet()); + gcpTracker.activatePrimaryMode(primarySeqNo, NO_OPS_PERFORMED); + gcpTracker.addPeerRecoveryRetentionLease("node2", 0, wrap(() -> {})); + gcpTracker.updateFromMaster(2L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), replica.allocationId().getId())), new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build(), Collections.emptySet()); - gcpTracker.activatePrimaryMode(primarySeqNo); for (int op = 0; op < opCount; op++) { final String id; // mostly index, sometimes delete 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 725225773a682..516cb5ea5041c 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -59,6 +59,8 @@ import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.threadpool.ThreadPool.Names; +import org.junit.Assert; import java.io.IOException; import java.util.ArrayList; @@ -74,6 +76,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static org.elasticsearch.action.ActionListener.wrap; +import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.empty; @@ -259,6 +263,7 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { } shards.promoteReplicaToPrimary(newPrimary).get(); + shards.removeReplica(oldPrimary); // check that local checkpoint of new primary is properly tracked after primary promotion assertThat(newPrimary.getLocalCheckpoint(), equalTo(totalDocs - 1L)); @@ -291,6 +296,11 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { }); newPrimary.flush(new FlushRequest().force(true)); if (replica.indexSettings().isSoftDeleteEnabled()) { + replica.flush(new FlushRequest().force(true)); + newPrimary.removeRetentionLease(getPeerRecoveryRetentionLeaseId(oldPrimary.routingEntry()), wrap(() -> {})); + newPrimary.runUnderPrimaryPermit( + newPrimary::renewPeerRecoveryRetentionLeaseForPrimary, Assert::assertNull, Names.SAME, ""); + newPrimary.renewPeerRecoveryRetentionLeaseForReplica(replica.routingEntry(), replica.getLocalCheckpointOfSafeCommit()); // 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)); 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 967328514a98d..52200f6a661c3 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.AllocationId; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; @@ -44,6 +45,8 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; +import static org.elasticsearch.index.seqno.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; +import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; @@ -72,7 +75,7 @@ public void testAddOrRenewRetentionLease() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final int length = randomIntBetween(0, 8); final long[] minimumRetainingSequenceNumbers = new long[length]; for (int i = 0; i < length; i++) { @@ -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); } } @@ -114,7 +117,7 @@ public void testAddDuplicateRetentionLease() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final String id = randomAlphaOfLength(8); final long retainingSequenceNumber = randomNonNegativeLong(); final String source = randomAlphaOfLength(8); @@ -143,7 +146,7 @@ public void testRenewNotFoundRetentionLease() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final String id = randomAlphaOfLength(8); final RetentionLeaseNotFoundException e = expectThrows( RetentionLeaseNotFoundException.class, @@ -175,12 +178,14 @@ public void testAddRetentionLeaseCausesRetentionLeaseSync() { equalTo(retainingSequenceNumbers)); }); reference.set(replicationTracker); + final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), - routingTable(Collections.emptySet(), allocationId), + routingTable, Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); + retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { @@ -215,7 +220,7 @@ public void testRemoveRetentionLease() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final int length = randomIntBetween(0, 8); final long[] minimumRetainingSequenceNumbers = new long[length]; for (int i = 0; i < length; i++) { @@ -243,7 +248,7 @@ public void testRemoveRetentionLease() { length - i - 1, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } @@ -266,7 +271,7 @@ public void testRemoveNotFound() { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final String id = randomAlphaOfLength(8); final RetentionLeaseNotFoundException e = expectThrows( RetentionLeaseNotFoundException.class, @@ -298,12 +303,14 @@ public void testRemoveRetentionLeaseCausesRetentionLeaseSync() { equalTo(retainingSequenceNumbers)); }); reference.set(replicationTracker); + final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), - routingTable(Collections.emptySet(), allocationId), + routingTable, Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); + retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { @@ -356,7 +363,7 @@ private void runExpirationTest(final boolean primaryMode) { routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); if (primaryMode) { - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); } final long[] retainingSequenceNumbers = new long[1]; retainingSequenceNumbers[0] = randomLongBetween(0, Long.MAX_VALUE); @@ -372,11 +379,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 @@ -394,18 +402,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); @@ -482,7 +491,7 @@ public void testLoadAndPersistRetentionLeases() throws IOException { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { if (rarely() && primaryTerm < Long.MAX_VALUE) { @@ -522,7 +531,7 @@ public void testPersistRetentionLeasesUnderConcurrency() throws IOException { Collections.singleton(allocationId.getId()), routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); - replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { if (rarely() && primaryTerm < Long.MAX_VALUE) { @@ -595,7 +604,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 (PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()) == false) { + idToRetentionLease.put(retentionLease.id(), retentionLease); + } } assertThat(idToRetentionLease.entrySet(), hasSize(size)); @@ -606,5 +617,4 @@ private void assertRetentionLeases( assertThat(retentionLease.source(), equalTo("test-" + i)); } } - } 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 037d2130b5c7b..885781a301679 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -57,6 +57,9 @@ import java.util.stream.Stream; import static java.util.Collections.emptySet; +import static java.util.Collections.singleton; +import static java.util.Collections.unmodifiableSet; +import static org.elasticsearch.action.ActionListener.wrap; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; @@ -120,8 +123,9 @@ public void testGlobalCheckpointUpdate() { logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); }); - tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + final IndexShardRoutingTable routingTable = routingTable(initializing, primaryId); + tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable, emptySet()); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1)); initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1 + initializing.size())); @@ -147,7 +151,8 @@ public void testGlobalCheckpointUpdate() { Set newInitializing = new HashSet<>(initializing); newInitializing.add(extraId); - tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId), emptySet()); + tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), + routingTable(newInitializing, routingTable.primaryShard()), emptySet()); tracker.initiateTracking(extraId.getId()); @@ -189,7 +194,7 @@ public void testMarkAllocationIdAsInSync() throws BrokenBarrierException, Interr final ReplicationTracker tracker = newTracker(primaryId); tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1); - tracker.activatePrimaryMode(localCheckpoint); + tracker.activatePrimaryMode(localCheckpoint, NO_OPS_PERFORMED); tracker.initiateTracking(replicaId.getId()); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { @@ -230,7 +235,7 @@ public void testMissingActiveIdsPreventAdvance() { AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); randomSubsetOf(initializing.keySet()).forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); final AllocationId missingActiveID = randomFrom(active.keySet()); assigned @@ -257,7 +262,7 @@ public void testMissingInSyncIdsPreventAdvance() { AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); randomSubsetOf(randomIntBetween(1, initializing.size() - 1), initializing.keySet()).forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); @@ -279,7 +284,7 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { final AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); initializing.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); nonApproved.keySet().forEach(k -> expectThrows(IllegalStateException.class, () -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED))); @@ -313,8 +318,9 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { allocations.putAll(initializingToBeRemoved); } final ReplicationTracker tracker = newTracker(primaryId); - tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + final IndexShardRoutingTable routingTable = routingTable(initializing, primaryId); + tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable, emptySet()); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); if (randomBoolean()) { initializingToStay.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); } else { @@ -329,7 +335,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { tracker.updateFromMaster( initialClusterStateVersion + 1, ids(activeToStay.keySet()), - routingTable(initializingToStay.keySet(), primaryId), + routingTable(initializingToStay.keySet(), routingTable.primaryShard()), emptySet()); allocations.forEach((aid, ckp) -> updateLocalCheckpoint(tracker, aid.getId(), ckp + 10L)); } else { @@ -337,7 +343,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { tracker.updateFromMaster( initialClusterStateVersion + 2, ids(activeToStay.keySet()), - routingTable(initializingToStay.keySet(), primaryId), + routingTable(initializingToStay.keySet(), routingTable.primaryShard()), emptySet()); } @@ -356,9 +362,9 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { final AllocationId trackingAllocationId = AllocationId.newInitializing(); final ReplicationTracker tracker = newTracker(inSyncAllocationId); final long clusterStateVersion = randomNonNegativeLong(); - tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), - routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet()); - tracker.activatePrimaryMode(globalCheckpoint); + final IndexShardRoutingTable routingTable = routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId); + tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), routingTable, emptySet()); + tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -397,7 +403,7 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { } else { // master changes its mind and cancels the allocation tracker.updateFromMaster(clusterStateVersion + 1, Collections.singleton(inSyncAllocationId.getId()), - routingTable(emptySet(), inSyncAllocationId), emptySet()); + routingTable(emptySet(), routingTable.primaryShard()), emptySet()); barrier.await(); assertTrue(complete.get()); assertNull(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId.getId())); @@ -422,7 +428,7 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar final ReplicationTracker tracker = newTracker(inSyncAllocationId); tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()), routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet()); - tracker.activatePrimaryMode(globalCheckpoint); + tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -471,7 +477,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { IndexShardRoutingTable routingTable = routingTable(initializingIds, primaryId); final ReplicationTracker tracker = newTracker(primaryId); tracker.updateFromMaster(initialClusterStateVersion, ids(activeAllocationIds), routingTable, emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); assertThat(tracker.getReplicationGroup().getInSyncAllocationIds(), equalTo(ids(activeAllocationIds))); assertThat(tracker.getReplicationGroup().getRoutingTable(), equalTo(routingTable)); @@ -499,7 +505,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { final List removingInitializingAllocationIds = randomSubsetOf(initializingIds); final Set newInitializingAllocationIds = initializingIds.stream().filter(a -> !removingInitializingAllocationIds.contains(a)).collect(Collectors.toSet()); - routingTable = routingTable(newInitializingAllocationIds, primaryId); + routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); tracker.updateFromMaster(initialClusterStateVersion + 1, ids(newActiveAllocationIds), routingTable, emptySet()); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); assertTrue(removingActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()) == null)); @@ -517,7 +523,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { tracker.updateFromMaster( initialClusterStateVersion + 2, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, primaryId), + routingTable(newInitializingAllocationIds, routingTable.primaryShard()), emptySet()); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); assertTrue( @@ -565,7 +571,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { tracker.updateFromMaster( initialClusterStateVersion + 3, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, primaryId), + routingTable(newInitializingAllocationIds, routingTable.primaryShard()), emptySet()); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { @@ -604,7 +610,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { tracker.updateFromMaster( initialClusterStateVersion + 4, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, primaryId), + routingTable(newInitializingAllocationIds, routingTable.primaryShard()), emptySet()); assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); @@ -635,7 +641,7 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok Collections.singleton(active.getId()), routingTable(Collections.singleton(initializing), active), emptySet()); - tracker.activatePrimaryMode(activeLocalCheckpoint); + tracker.activatePrimaryMode(activeLocalCheckpoint, NO_OPS_PERFORMED); final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE); final Thread activeThread = new Thread(() -> { try { @@ -690,16 +696,26 @@ public void testPrimaryContextHandoff() throws IOException { final BiConsumer> onNewRetentionLease = (leases, listener) -> {}; ReplicationTracker oldPrimary = new ReplicationTracker( - shardId, aId.getId(), indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); + shardId, clusterState.routingTable.primaryShard().allocationId().getId(), + indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); + + clusterState.apply(oldPrimary); + oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10), NO_OPS_PERFORMED); + + clusterState = randomUpdateClusterState(singleton(oldPrimary.shardAllocationId), clusterState); + ReplicationTracker newPrimary = new ReplicationTracker( - shardId, aId.getRelocationId(), indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); + shardId, clusterState.routingTable.primaryShard().allocationId().getRelocationId(), + indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); - Set allocationIds = new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId)); + oldPrimary.addPeerRecoveryRetentionLease(clusterState.routingTable.primaryShard().relocatingNodeId(), 0L, wrap(() -> {})); + newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); clusterState.apply(oldPrimary); clusterState.apply(newPrimary); - activatePrimary(oldPrimary); + final Set allocationIds + = unmodifiableSet(new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId))); final int numUpdates = randomInt(10); for (int i = 0; i < numUpdates; i++) { @@ -836,7 +852,7 @@ public void testIllegalStateExceptionIfUnknownAllocationId() { final ReplicationTracker tracker = newTracker(active); tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(active.getId()), routingTable(Collections.singleton(initializing), active), emptySet()); - tracker.activatePrimaryMode(NO_OPS_PERFORMED); + tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); expectThrows(IllegalStateException.class, () -> tracker.initiateTracking(randomAlphaOfLength(10))); expectThrows(IllegalStateException.class, () -> tracker.markAllocationIdAsInSync(randomAlphaOfLength(10), randomNonNegativeLong())); @@ -876,24 +892,16 @@ private static FakeClusterState initialState() { final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); final Set initializingAllocationIds = activeAndInitializingAllocationIds.v2(); final AllocationId primaryId = randomFrom(activeAllocationIds); - final AllocationId relocatingId = AllocationId.newRelocation(primaryId); - activeAllocationIds.remove(primaryId); - activeAllocationIds.add(relocatingId); final ShardId shardId = new ShardId("test", "_na_", 0); final ShardRouting primaryShard = - TestShardRouting.newShardRouting( - shardId, randomAlphaOfLength(10), randomAlphaOfLength(10), true, ShardRoutingState.RELOCATING, relocatingId); - + TestShardRouting.newShardRouting( + shardId, randomAlphaOfLength(10), null, true, ShardRoutingState.STARTED, primaryId); return new FakeClusterState( initialClusterStateVersion, activeAllocationIds, 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(); @@ -914,12 +922,24 @@ 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 initializingIds + = Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove); + final ShardRouting primaryShard = clusterState.routingTable.primaryShard(); + assert !initializingIds.contains(primaryShard.allocationId()); + final ShardId shardId = new ShardId("test", "_na_", 0); + 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)); + } + + builder.addShard(primaryShard.state() == ShardRoutingState.RELOCATING + ? primaryShard : primaryShard.relocate(randomAlphaOfLength(10), 0)); + return new FakeClusterState( clusterState.version + randomIntBetween(1, 5), remainingInSyncIds.isEmpty() ? clusterState.inSyncIds : remainingInSyncIds, - routingTable( - Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove), - clusterState.routingTable.primaryShard())); + builder.build()); } private static Set exclude(Collection allocationIds, Set excludeIds) { 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..434c37212fe18 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseActionsTests.java @@ -36,17 +36,25 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; +import java.util.stream.Collectors; import static org.elasticsearch.index.seqno.RetentionLeaseActions.RETAIN_ALL; import static org.hamcrest.Matchers.arrayWithSize; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; +import static org.hamcrest.Matchers.contains; public class RetentionLeaseActionsTests extends ESSingleNodeTestCase { + private String getPeerRecoveryRetentionLeaseId() { + return ReplicationTracker.getPeerRecoveryRetentionLeaseId( + client().admin().cluster().prepareState().get().getState().nodes().getLocalNodeId()); + } + public void testAddAction() { final Settings settings = Settings.builder() .put("index.number_of_shards", 1) @@ -73,8 +81,13 @@ 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(); + + final RetentionLeases retentionLeases = stats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(retentionLeases.leases(), hasSize(2)); + assertThat(retentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + + final RetentionLease retentionLease = retentionLeases.get(id); assertThat(retentionLease.id(), equalTo(id)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber)); assertThat(retentionLease.source(), equalTo(source)); @@ -160,9 +173,13 @@ 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)); - final RetentionLease initialRetentionLease = - initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + + final RetentionLeases initialRetentionLeases = initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(initialRetentionLeases.leases(), hasSize(2)); + assertThat(initialRetentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + + final RetentionLease initialRetentionLease = initialRetentionLeases.get(id); final long nextRetainingSequenceNumber = retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL @@ -195,9 +212,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)); - final RetentionLease renewedRetentionLease = - renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases().leases().iterator().next(); + final RetentionLeases renewedRetentionLeases = renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(renewedRetentionLeases.leases(), hasSize(2)); + assertThat(renewedRetentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + final RetentionLease renewedRetentionLease = renewedRetentionLeases.get(id); assertThat(renewedRetentionLease.id(), equalTo(id)); assertThat( renewedRetentionLease.retainingSequenceNumber(), @@ -265,7 +284,10 @@ 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)); + final RetentionLeases retentionLeases = stats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(retentionLeases.leases(), hasSize(1)); + assertThat(retentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + contains(getPeerRecoveryRetentionLeaseId())); } public void testRemoveNotFound() { @@ -328,8 +350,13 @@ 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(); + + final RetentionLeases retentionLeases = stats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(retentionLeases.leases(), hasSize(2)); + assertThat(retentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + + final RetentionLease retentionLease = retentionLeases.get(id); assertThat(retentionLease.id(), equalTo(id)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(retainingSequenceNumber == RETAIN_ALL ? 0L : retainingSequenceNumber)); assertThat(retentionLease.source(), equalTo(source)); @@ -378,9 +405,13 @@ 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(); + + final RetentionLeases initialRetentionLeases = initialStats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(initialRetentionLeases.leases(), hasSize(2)); + assertThat(initialRetentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + + final RetentionLease initialRetentionLease = initialRetentionLeases.get(id); final long nextRetainingSequenceNumber = retainingSequenceNumber == RETAIN_ALL && randomBoolean() ? RETAIN_ALL @@ -427,9 +458,11 @@ 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(); + final RetentionLeases renewedRetentionLeases = renewedStats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(renewedRetentionLeases.leases(), hasSize(2)); + assertThat(renewedRetentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + containsInAnyOrder(id, getPeerRecoveryRetentionLeaseId())); + final RetentionLease renewedRetentionLease = renewedRetentionLeases.get(id); assertThat(renewedRetentionLease.id(), equalTo(id)); assertThat( renewedRetentionLease.retainingSequenceNumber(), @@ -484,7 +517,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(0)); + final RetentionLeases retentionLeases = stats.getShards()[0].getRetentionLeaseStats().retentionLeases(); + assertThat(retentionLeases.leases(), hasSize(1)); + assertThat(retentionLeases.leases().stream().map(RetentionLease::id).collect(Collectors.toList()), + contains(getPeerRecoveryRetentionLeaseId())); } /* 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 44a8cd70c42eb..4ad38200a22c2 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java @@ -47,6 +47,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.elasticsearch.index.seqno.RetentionLeases.toMapExcludingPeerRecoveryRetentionLeases; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.contains; @@ -103,7 +104,7 @@ 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(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()) { @@ -112,11 +113,10 @@ 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()); - assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); + assertThat(toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()), equalTo(currentRetentionLeases)); // check retention leases have been written on the replica - assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases()))); + assertThat(currentRetentionLeases, equalTo(toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases()))); } } } @@ -161,7 +161,7 @@ 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(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()) { @@ -170,11 +170,10 @@ 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()); - assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); + assertThat(toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()), equalTo(currentRetentionLeases)); // check retention leases have been written on the replica - assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases()))); + assertThat(currentRetentionLeases, equalTo(toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases()))); } } } @@ -227,7 +226,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(toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()).keySet(), + anyOf(empty(), contains(currentRetentionLease.id()))); } // update the index for retention leases to short a long time, to force expiration @@ -244,7 +244,7 @@ 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(toMapExcludingPeerRecoveryRetentionLeases(primary.getRetentionLeases()).entrySet(), empty())); // now that all retention leases are expired should have been synced to all replicas assertBusy(() -> { @@ -254,7 +254,7 @@ 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(toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()).entrySet(), empty()); } }); } @@ -366,11 +366,10 @@ 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()); - assertThat(retentionLeasesOnReplica, equalTo(currentRetentionLeases)); + assertThat(toMapExcludingPeerRecoveryRetentionLeases(replica.getRetentionLeases()), equalTo(currentRetentionLeases)); // check retention leases have been written on the replica; see RecoveryTarget#finalizeRecovery - assertThat(currentRetentionLeases, equalTo(RetentionLeases.toMap(replica.loadRetentionLeases()))); + assertThat(currentRetentionLeases, equalTo(toMapExcludingPeerRecoveryRetentionLeases(replica.loadRetentionLeases()))); } } 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 8721450073531..aecfcc39e3da3 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseStatsTests.java @@ -32,6 +32,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; +import static org.elasticsearch.index.seqno.RetentionLeases.toMapExcludingPeerRecoveryRetentionLeases; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; @@ -61,7 +62,6 @@ 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(toMapExcludingPeerRecoveryRetentionLeases(retentionLeaseStats.retentionLeases()), equalTo(currentRetentionLeases)); } - } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index fb8574594a874..3148cc3d35244 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -676,7 +676,8 @@ public static final IndexShard newIndexShard( Arrays.asList(listeners), () -> {}, RetentionLeaseSyncer.EMPTY, - cbs); + cbs, + () -> {}); } private static ShardRouting getInitializingShardRouting(ShardRouting existingShardRouting) { 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 566d1feaf007d..57a91ca0c85c2 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -34,7 +34,7 @@ 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.ExecutorService; @@ -42,6 +42,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import static org.elasticsearch.index.seqno.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; +import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; @@ -81,7 +83,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++) { @@ -92,7 +94,7 @@ public void testAddOrRenewRetentionLease() throws IOException { length, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } @@ -112,7 +114,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++) { @@ -122,7 +124,7 @@ public void testRemoveRetentionLease() throws IOException { length - i - 1, minimumRetainingSequenceNumbers, primaryTerm, - 1 + length + i, + 2 + length + i, true, false); } @@ -139,6 +141,11 @@ public void testExpirationOnReplica() throws IOException { runExpirationTest(false); } + private RetentionLease peerRecoveryRetentionLease(IndexShard indexShard) { + return new RetentionLease( + getPeerRecoveryRetentionLeaseId(indexShard.routingEntry()), 0, currentTimeMillis.get(), 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 @@ -153,23 +160,29 @@ 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) { indexShard.addRetentionLease("0", retainingSequenceNumbers[0], "test-0", ActionListener.wrap(() -> {})); + initialVersion = 2; } else { + assertRetentionLeases(indexShard, 0, new long[0], primaryTerm, 2, false, false); + 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), // add a fake peer-recovery retention lease for this shard + 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)); + logger.info("retentionLeases = {}", retentionLeases); + 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 @@ -180,28 +193,29 @@ 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), // add a fake peer-recovery retention lease for this shard + 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); @@ -239,8 +253,8 @@ public void testPersistence() throws IOException { 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(length + 1L)); + assertThat(writtenRetentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0]))); } // when we recover, we should recover the retention leases @@ -254,7 +268,7 @@ public void testPersistence() throws IOException { assertThat(recoveredRetentionLeases.version(), equalTo(0L)); assertThat(recoveredRetentionLeases.leases(), empty()); } else { - assertThat(recoveredRetentionLeases.version(), equalTo((long) length)); + assertThat(recoveredRetentionLeases.version(), equalTo(length + 1L)); assertThat( recoveredRetentionLeases.leases(), contains(retentionLeases.leases().toArray(new RetentionLease[0]))); @@ -282,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); } @@ -330,7 +344,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 (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 316ed39574c0c..124c2742e7366 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -101,6 +101,9 @@ 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; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -123,7 +126,10 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.FieldMaskingReader; import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.threadpool.ThreadPool.Names; +import org.junit.Assert; import java.io.IOException; import java.nio.charset.Charset; @@ -164,6 +170,7 @@ import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -1047,7 +1054,8 @@ public void testGlobalCheckpointSync() throws IOException { IndexMetaData.builder(shardRouting.getIndexName()).settings(settings).primaryTerm(0, 1); final AtomicBoolean synced = new AtomicBoolean(); final IndexShard primaryShard = - newShard(shardRouting, indexMetadata.build(), null, new InternalEngineFactory(), () -> synced.set(true)); + newShard(shardRouting, indexMetadata.build(), null, new InternalEngineFactory(), () -> synced.set(true), + RetentionLeaseSyncer.EMPTY); // add a replica recoverShardFromStore(primaryShard); final IndexShard replicaShard = newShard(shardId, false); @@ -1464,7 +1472,7 @@ public String[] listAll() throws IOException { try (Store store = createStore(shardId, new IndexSettings(metaData, Settings.EMPTY), directory)) { IndexShard shard = newShard(shardRouting, shardPath, metaData, i -> store, null, new InternalEngineFactory(), () -> { - }, EMPTY_EVENT_LISTENER); + }, RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); AtomicBoolean failureCallbackTriggered = new AtomicBoolean(false); shard.addShardFailureCallback((ig)->failureCallbackTriggered.set(true)); @@ -2122,6 +2130,7 @@ public void testRecoverFromStoreRemoveStaleOperations() throws Exception { null, shard.getEngineFactory(), shard.getGlobalCheckpointSyncer(), + RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); newShard.markAsRecovering("store", new RecoveryState(newShard.routingEntry(), localNode, null)); @@ -2242,6 +2251,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { wrapper, new InternalEngineFactory(), () -> {}, + RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); recoverShardFromStore(newShard); @@ -2396,6 +2406,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { wrapper, new InternalEngineFactory(), () -> {}, + RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); recoverShardFromStore(newShard); @@ -2783,6 +2794,29 @@ public void testDocStats() throws Exception { flushRequest.waitIfOngoing(false); indexShard.flush(flushRequest); + if (indexShard.indexSettings.isSoftDeleteEnabled()) { + // We still retain the deletes because of the peer-recovery retention lease - need to update the lease and flush again + if (indexShard.routingEntry().primary()) { + indexShard.runUnderPrimaryPermit( + indexShard::renewPeerRecoveryRetentionLeaseForPrimary, Assert::assertNull, Names.SAME, ""); + } else { + final RetentionLeases retentionLeases = indexShard.getRetentionLeases(); + final long localCheckpointOfSafeCommit = indexShard.getLocalCheckpointOfSafeCommit(); + indexShard.updateRetentionLeasesOnReplica(new RetentionLeases(retentionLeases.primaryTerm(), + retentionLeases.version() + 1, + retentionLeases.leases().stream().map(l -> { + assertEquals(ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE, l.source()); + assertThat(l.retainingSequenceNumber(), lessThanOrEqualTo(localCheckpointOfSafeCommit + 1)); + return new RetentionLease(l.id(), localCheckpointOfSafeCommit + 1, l.timestamp(), l.source()); + }).collect(Collectors.toList()))); + indexShard.updateGlobalCheckpointOnReplica(indexShard.getLocalCheckpoint(), "test"); + } + final FlushRequest flushRequest2 = new FlushRequest(); + flushRequest2.force(true); + flushRequest2.waitIfOngoing(false); + indexShard.flush(flushRequest2); + } + if (randomBoolean()) { indexShard.refresh("test"); } @@ -2964,7 +2998,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO IndexShard corruptedShard = newShard(shardRouting, shardPath, indexMetaData, null, null, indexShard.engineFactory, - indexShard.getGlobalCheckpointSyncer(), EMPTY_EVENT_LISTENER); + indexShard.getGlobalCheckpointSyncer(), RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); final IndexShardRecoveryException indexShardRecoveryException = expectThrows(IndexShardRecoveryException.class, () -> newStartedShard(p -> corruptedShard, true)); @@ -3009,7 +3043,7 @@ public void testShardDoesNotStartIfCorruptedMarkerIsPresent() throws Exception { // try to start shard on corrupted files final IndexShard corruptedShard = newShard(shardRouting, shardPath, indexMetaData, null, null, indexShard.engineFactory, - indexShard.getGlobalCheckpointSyncer(), EMPTY_EVENT_LISTENER); + indexShard.getGlobalCheckpointSyncer(), RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); final IndexShardRecoveryException exception1 = expectThrows(IndexShardRecoveryException.class, () -> newStartedShard(p -> corruptedShard, true)); @@ -3032,7 +3066,7 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO // try to start another time shard on corrupted files final IndexShard corruptedShard2 = newShard(shardRouting, shardPath, indexMetaData, null, null, indexShard.engineFactory, - indexShard.getGlobalCheckpointSyncer(), EMPTY_EVENT_LISTENER); + indexShard.getGlobalCheckpointSyncer(), RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); final IndexShardRecoveryException exception2 = expectThrows(IndexShardRecoveryException.class, () -> newStartedShard(p -> corruptedShard2, true)); @@ -3072,7 +3106,7 @@ public void testReadSnapshotAndCheckIndexConcurrently() throws Exception { .build(); final IndexShard newShard = newShard(shardRouting, indexShard.shardPath(), indexMetaData, null, null, indexShard.engineFactory, - indexShard.getGlobalCheckpointSyncer(), EMPTY_EVENT_LISTENER); + indexShard.getGlobalCheckpointSyncer(), RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); Store.MetadataSnapshot storeFileMetaDatas = newShard.snapshotStoreMetadata(); assertTrue("at least 2 files, commit and data: " + storeFileMetaDatas.toString(), storeFileMetaDatas.size() > 1); @@ -3340,7 +3374,7 @@ public void testSegmentMemoryTrackedInBreaker() throws Exception { indexDoc(primary, "_doc", "4", "{\"foo\": \"potato\"}"); indexDoc(primary, "_doc", "5", "{\"foo\": \"potato\"}"); // Forces a refresh with the INTERNAL scope - ((InternalEngine) primary.getEngine()).writeIndexingBuffer(); + primary.getEngine().writeIndexingBuffer(); ss = primary.segmentStats(randomBoolean()); breaker = primary.circuitBreakerService.getBreaker(CircuitBreaker.ACCOUNTING); @@ -3356,6 +3390,8 @@ public void testSegmentMemoryTrackedInBreaker() throws Exception { if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { primary.sync(); flushShard(primary); + primary.runUnderPrimaryPermit(primary::renewPeerRecoveryRetentionLeaseForPrimary, Assert::assertNull, Names.SAME, ""); + flushShard(primary, true); // force since the last flush didn't discard the retained ops; TODO should an unforced flush work? } primary.refresh("force refresh"); @@ -3484,7 +3520,7 @@ public void testFlushOnInactive() throws Exception { AtomicReference primaryRef = new AtomicReference<>(); IndexShard primary = newShard(shardRouting, shardPath, metaData, null, null, new InternalEngineFactory(), () -> { - }, new IndexEventListener() { + }, RetentionLeaseSyncer.EMPTY, new IndexEventListener() { @Override public void onShardInactive(IndexShard indexShard) { markedInactive.set(true); @@ -3707,7 +3743,7 @@ public void testTypelessDelete() throws IOException { IndexMetaData metaData = IndexMetaData.builder("index") .putMapping("some_type", "{ \"properties\": {}}") .settings(settings) - .build(); + .primaryTerm(0, 1).build(); IndexShard shard = newShard(new ShardId(metaData.getIndex(), 0), true, "n1", metaData, null); recoverShardFromStore(shard); Engine.IndexResult indexResult = indexDoc(shard, "some_type", "id", "{}"); 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 85e381b176ccb..d074ef3375833 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/PrimaryReplicaSyncerTests.java @@ -115,10 +115,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/index/shard/RemoveCorruptedShardDataCommandTests.java b/server/src/test/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandTests.java index 1c3c3b28773cf..e181269dd8884 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RemoveCorruptedShardDataCommandTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.index.MergePolicyConfig; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.seqno.RetentionLeaseSyncer; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TestTranslog; import org.elasticsearch.index.translog.TranslogCorruptedException; @@ -110,7 +111,7 @@ public void setup() throws IOException { indexShard = newStartedShard(p -> newShard(routing, shardPath, indexMetaData, null, null, new InternalEngineFactory(), () -> { - }, EMPTY_EVENT_LISTENER), + }, RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER), true); translogPath = shardPath.resolveTranslog(); @@ -372,7 +373,7 @@ private IndexShard reopenIndexShard(boolean corrupted) throws IOException { }; return newShard(shardRouting, shardPath, metaData, storeProvider, null, - indexShard.engineFactory, indexShard.getGlobalCheckpointSyncer(), EMPTY_EVENT_LISTENER); + indexShard.engineFactory, indexShard.getGlobalCheckpointSyncer(), RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); } private int indexDocs(IndexShard indexShard, boolean flushLast) throws IOException { diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index 122d74121a718..b54482e58474a 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -131,7 +131,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem newRouting = newRouting.moveToUnassigned(unassignedInfo) .updateUnassigned(unassignedInfo, RecoverySource.EmptyStoreRecoverySource.INSTANCE); newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); - IndexShard shard = index.createShard(newRouting, s -> {}, RetentionLeaseSyncer.EMPTY); + IndexShard shard = index.createShard(newRouting, s -> {}, RetentionLeaseSyncer.EMPTY, s -> {}); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); final DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 9b6cae43081ad..8a9515488bf4f 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -235,7 +235,8 @@ public MockIndexShard createShard( final RepositoriesService repositoriesService, final Consumer onShardFailure, final Consumer globalCheckpointSyncer, - final RetentionLeaseSyncer retentionLeaseSyncer) throws IOException { + final RetentionLeaseSyncer retentionLeaseSyncer, + final Consumer peerRecoveryRetentionleaseRenewer) throws IOException { failRandomly(); MockIndexService indexService = indexService(recoveryState.getShardId().getIndex()); MockIndexShard indexShard = indexService.createShard(shardRouting); diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index 9fd7f24db024b..ac7ae413d15a7 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -482,7 +482,8 @@ private IndicesClusterStateService createIndicesClusterStateService(DiscoveryNod null, primaryReplicaSyncer, s -> {}, - RetentionLeaseSyncer.EMPTY); + RetentionLeaseSyncer.EMPTY, + s -> {}); } private class RecordingIndicesService extends MockIndicesService { 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 ea3e933a88314..049393198a21b 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -47,6 +47,8 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; import org.elasticsearch.index.recovery.RecoveryStats; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState.Stage; @@ -80,7 +82,11 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import static java.util.Objects.requireNonNull; import static org.elasticsearch.node.RecoverySettingsChunkSizePlugin.CHUNK_SIZE_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -814,9 +820,7 @@ public void testHistoryRetention() throws Exception { internalCluster().stopRandomNode(s -> true); final long desyncNanoTime = System.nanoTime(); - while (System.nanoTime() <= desyncNanoTime) { - // time passes - } + assertBusy(() -> assertThat(System.nanoTime(), greaterThan(desyncNanoTime))); // time passes final int numNewDocs = scaledRandomIntBetween(25, 250); for (int i = 0; i < numNewDocs; i++) { @@ -838,5 +842,23 @@ public void testHistoryRetention() throws Exception { assertThat(recoveryStates, hasSize(1)); assertThat(recoveryStates.get(0).getIndex().totalFileCount(), is(0)); assertThat(recoveryStates.get(0).getTranslog().recoveredOperations(), greaterThan(0)); + + final Map indexShardsByPrimary + = StreamSupport.stream(internalCluster().getInstances(IndicesService.class).spliterator(), false) + .map(is -> is.getShardOrNull(new ShardId(resolveIndex("test"), 0))) + .collect(Collectors.toMap(is -> is.routingEntry().primary(), Function.identity(), + (o1, o2) -> { + throw new AssertionError("should not need to combine " + o1 + " with " + o2); + })); + + final IndexShard primary = requireNonNull(indexShardsByPrimary.get(true)); + assertThat(client().admin().indices().prepareFlush().setForce(true).get().getFailedShards(), equalTo(0)); // make a safe commit + primary.syncRetentionLeases(); // happens periodically, removes retention leases for old shard copies + primary.renewPeerRecoveryRetentionLeases(); // happens periodically, advances retention leases according to last safe commit + assertBusy(() -> assertThat(primary.getMinRetainedSeqNo(), equalTo(primary.seqNoStats().getMaxSeqNo() + 1))); + primary.syncRetentionLeases(); // happens periodically, pushes updated retention leases to replica + + final IndexShard replica = requireNonNull(indexShardsByPrimary.get(false)); + assertBusy(() -> assertThat(replica.getMinRetainedSeqNo(), equalTo(replica.seqNoStats().getMaxSeqNo() + 1))); } } 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 2761333ef5628..48061b11d58c7 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -68,7 +68,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); } } @@ -281,7 +282,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)); @@ -385,7 +387,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 5fb67a64d9db5..6e0be22f871a6 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -1057,6 +1057,7 @@ public void testFilterCacheStats() throws Exception { if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { persistGlobalCheckpoint("index"); flush("index"); + releaseHistory("index"); } ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge("index").setFlush(true).setMaxNumSegments(1).get(); @@ -1213,4 +1214,30 @@ private void persistGlobalCheckpoint(String index) throws Exception { } } } + + private void releaseHistory(String index) throws Exception { + // TODO maybe we want an (internal) API to await the release of history, rather than busy-waiting like this? + final Set nodes = internalCluster().nodesInclude(index); + for (String node : nodes) { + final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + if (indexShard.routingEntry().primary()) { + indexShard.renewPeerRecoveryRetentionLeases(); + } + } + } + } + assertBusy(() -> { + for (String node : nodes) { + final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + assertFalse(indexShard.routingEntry().toString(), + indexShard.hasCompleteHistoryOperations("test", indexShard.getLocalCheckpointOfSafeCommit())); + } + } + } + }); + } } diff --git a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java index ba3fa84a19641..1b59f558db584 100644 --- a/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java +++ b/server/src/test/java/org/elasticsearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.seqno.RetentionLeaseSyncer; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.IndexShardTestCase; @@ -109,6 +110,7 @@ public void testRestoreSnapshotWithExistingFiles() throws IOException { null, new InternalEngineFactory(), () -> {}, + RetentionLeaseSyncer.EMPTY, EMPTY_EVENT_LISTENER); // restore the shard diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index ece8bbd7194e9..e955808b61cab 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -97,6 +97,7 @@ import org.elasticsearch.gateway.TransportNodesListGatewayStartedShards; import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; +import org.elasticsearch.index.seqno.PeerRecoveryRetentionLeaseRenewalAction; import org.elasticsearch.index.seqno.RetentionLeaseBackgroundSyncAction; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.shard.PrimaryReplicaSyncer; @@ -906,7 +907,9 @@ protected void assertSnapshotOrGenericThread() { threadPool, shardStateAction, actionFilters, - indexNameExpressionResolver)); + indexNameExpressionResolver), + new PeerRecoveryRetentionLeaseRenewalAction(settings, transportService, clusterService, indicesService, threadPool, + shardStateAction, actionFilters, indexNameExpressionResolver)); Map actions = new HashMap<>(); actions.put(CreateIndexAction.INSTANCE, new TransportCreateIndexAction( 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 37fc1c748c189..99b8a6772a0d6 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 @@ -67,6 +67,8 @@ import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; +import org.elasticsearch.index.seqno.RetentionLeaseSyncer; +import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.PrimaryReplicaSyncer; @@ -97,6 +99,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import static org.elasticsearch.action.ActionListener.wrap; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -167,6 +170,20 @@ protected class ReplicationGroup implements AutoCloseable, Iterable boolean closed = false; private ReplicationTargets replicationTargets; + private final RetentionLeaseSyncer retentionLeaseSyncer = new RetentionLeaseSyncer() { + @Override + public void sync(ShardId shardId, RetentionLeases retentionLeases, ActionListener listener) { + assertEquals(shardId, primary.shardId()); + replicas.forEach(replica -> replica.updateRetentionLeasesOnReplica(retentionLeases)); + listener.onResponse(null); + } + + @Override + public void backgroundSync(ShardId shardId, RetentionLeases retentionLeases) { + sync(shardId, retentionLeases, wrap(() -> {})); + } + }; + private final PrimaryReplicaSyncer primaryReplicaSyncer = new PrimaryReplicaSyncer( new TaskManager(Settings.EMPTY, threadPool, Collections.emptySet()), (request, parentTask, primaryAllocationId, primaryTerm, listener) -> { @@ -179,7 +196,7 @@ protected class ReplicationGroup implements AutoCloseable, Iterable protected ReplicationGroup(final IndexMetaData indexMetaData) throws IOException { final ShardRouting primaryRouting = this.createShardRouting("s0", true); - primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}); + primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting), () -> {}, retentionLeaseSyncer); replicas = new CopyOnWriteArrayList<>(); this.indexMetaData = indexMetaData; updateAllocationIDsOnPrimary(); @@ -235,7 +252,7 @@ public BulkItemResponse delete(DeleteRequest deleteRequest) throws Exception { private BulkItemResponse executeWriteRequest( DocWriteRequest writeRequest, WriteRequest.RefreshPolicy refreshPolicy) throws Exception { PlainActionFuture listener = new PlainActionFuture<>(); - final ActionListener wrapBulkListener = ActionListener.wrap( + final ActionListener wrapBulkListener = wrap( bulkShardResponse -> listener.onResponse(bulkShardResponse.getResponses()[0]), listener::onFailure); BulkItemRequest[] items = new BulkItemRequest[1]; @@ -284,7 +301,7 @@ public void startPrimary() throws IOException { public IndexShard addReplica() throws IOException { final ShardRouting replicaRouting = createShardRouting("s" + replicaId.incrementAndGet(), false); final IndexShard replica = - newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting), () -> {}); + newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting), () -> {}, retentionLeaseSyncer); addReplica(replica); return replica; } @@ -315,7 +332,7 @@ public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardP final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null, null, getEngineFactory(shardRouting), - () -> {}, EMPTY_EVENT_LISTENER); + () -> {}, retentionLeaseSyncer, EMPTY_EVENT_LISTENER); replicas.add(newReplica); if (replicationTargets != null) { replicationTargets.addReplica(newReplica); 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 f59ae8b9683ac..dfeffa70eeffe 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 @@ -250,7 +250,7 @@ protected IndexShard newShard( .settings(indexSettings) .primaryTerm(0, primaryTerm) .putMapping("_doc", "{ \"properties\": {} }"); - return newShard(shardRouting, metaData.build(), null, engineFactory, () -> {}, listeners); + return newShard(shardRouting, metaData.build(), null, engineFactory, () -> {}, RetentionLeaseSyncer.EMPTY, listeners); } /** @@ -293,7 +293,8 @@ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, I @Nullable IndexSearcherWrapper searcherWrapper, Runnable globalCheckpointSyncer) throws IOException { ShardRouting shardRouting = TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING, primary ? RecoverySource.EmptyStoreRecoverySource.INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE); - return newShard(shardRouting, indexMetaData, searcherWrapper, new InternalEngineFactory(), globalCheckpointSyncer); + return newShard(shardRouting, indexMetaData, searcherWrapper, new InternalEngineFactory(), globalCheckpointSyncer, + RetentionLeaseSyncer.EMPTY); } /** @@ -307,7 +308,7 @@ protected IndexShard newShard(ShardId shardId, boolean primary, String nodeId, I protected IndexShard newShard( ShardRouting routing, IndexMetaData indexMetaData, EngineFactory engineFactory, IndexingOperationListener... listeners) throws IOException { - return newShard(routing, indexMetaData, null, engineFactory, () -> {}, listeners); + return newShard(routing, indexMetaData, null, engineFactory, () -> {}, RetentionLeaseSyncer.EMPTY, listeners); } /** @@ -323,6 +324,7 @@ protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, @Nullable IndexSearcherWrapper indexSearcherWrapper, @Nullable EngineFactory engineFactory, Runnable globalCheckpointSyncer, + RetentionLeaseSyncer retentionLeaseSyncer, IndexingOperationListener... listeners) throws IOException { // add node id as name to settings for proper logging @@ -330,7 +332,7 @@ protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir()); ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId); return newShard(routing, shardPath, indexMetaData, null, indexSearcherWrapper, engineFactory, globalCheckpointSyncer, - EMPTY_EVENT_LISTENER, listeners); + retentionLeaseSyncer, EMPTY_EVENT_LISTENER, listeners); } /** @@ -341,6 +343,7 @@ protected IndexShard newShard(ShardRouting routing, IndexMetaData indexMetaData, * @param storeProvider an optional custom store provider to use. If null a default file based store will be created * @param indexSearcherWrapper an optional wrapper to be used during searchers * @param globalCheckpointSyncer callback for syncing global checkpoints + * @param retentionLeaseSyncer callback for syncing retention leases * @param indexEventListener index event listener * @param listeners an optional set of listeners to add to the shard */ @@ -349,6 +352,7 @@ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMe @Nullable IndexSearcherWrapper indexSearcherWrapper, @Nullable EngineFactory engineFactory, Runnable globalCheckpointSyncer, + RetentionLeaseSyncer retentionLeaseSyncer, IndexEventListener indexEventListener, IndexingOperationListener... listeners) throws IOException { final Settings nodeSettings = Settings.builder().put("node.name", routing.currentNodeId()).build(); final IndexSettings indexSettings = new IndexSettings(indexMetaData, nodeSettings); @@ -386,8 +390,9 @@ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMe Collections.emptyList(), Arrays.asList(listeners), globalCheckpointSyncer, - RetentionLeaseSyncer.EMPTY, - breakerService); + retentionLeaseSyncer, + breakerService, + () -> {}); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); success = true; } finally { @@ -438,7 +443,8 @@ protected IndexShard reinitShard(IndexShard current, ShardRouting routing, Engin null, engineFactory, current.getGlobalCheckpointSyncer(), - EMPTY_EVENT_LISTENER, listeners); + RetentionLeaseSyncer.EMPTY, + EMPTY_EVENT_LISTENER, listeners); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 4582e27d027da..dceeef0ece02d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -2327,7 +2327,11 @@ private static boolean isSuiteScopedTest(Class clazz) { } public static Index resolveIndex(String index) { - GetIndexResponse getIndexResponse = client().admin().indices().prepareGetIndex().setIndices(index).get(); + return resolveIndex(index, null); + } + + public static Index resolveIndex(String index, String viaNode) { + GetIndexResponse getIndexResponse = client(viaNode).admin().indices().prepareGetIndex().setIndices(index).get(); assertTrue("index " + index + " not found", getIndexResponse.getSettings().containsKey(index)); String uuid = getIndexResponse.getSettings().get(index).get(IndexMetaData.SETTING_INDEX_UUID); return new Index(index, uuid); From 1db54e6d530f3f93c53e2048ac72ad503c247f95 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 19 Feb 2019 18:08:32 +0000 Subject: [PATCH 02/32] Imports --- .../index/seqno/RetentionLeaseBackgroundSyncAction.java | 1 - .../java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java | 2 -- .../java/org/elasticsearch/index/shard/IndexShardTests.java | 3 --- 3 files changed, 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java index 4d500e5a1f502..feb3fa300fe23 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java @@ -31,7 +31,6 @@ import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; -import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index e249d4d1b4cd5..87c92ede576ae 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -52,7 +52,6 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalSettingsPlugin; import org.elasticsearch.test.InternalTestCluster.RestartCallback; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; @@ -83,7 +82,6 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; -import static org.junit.Assert.assertEquals; @ClusterScope(numDataNodes = 0, scope = Scope.TEST) public class RecoveryFromGatewayIT extends ESIntegTestCase { 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 124c2742e7366..e2eb7a2566ee0 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -84,7 +84,6 @@ import org.elasticsearch.index.engine.Engine.DeleteResult; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineTestCase; -import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.engine.SegmentsStats; @@ -126,7 +125,6 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.FieldMaskingReader; import org.elasticsearch.test.VersionUtils; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; import org.junit.Assert; @@ -170,7 +168,6 @@ import static org.elasticsearch.common.lucene.Lucene.cleanLuceneIndex; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; import static org.hamcrest.Matchers.containsInAnyOrder; From bfaf7345c290124fc9b7ff72aec5a6bf9062682d Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 19 Feb 2019 18:10:47 +0000 Subject: [PATCH 03/32] No need for this --- .../index/seqno/RetentionLeaseSyncAction.java | 13 ------------- 1 file changed, 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index baf0fca824a10..6fa39b020fd9e 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -205,17 +205,4 @@ protected Response newResponseInstance() { return new Response(); } - @Override - protected ClusterBlockLevel globalBlockLevel() { - // see e.g. ShrinkIndexIT#testCreateShrinkIndexFails which tries to relocate shards in a read-only index, needing updates to the - // peer recovery retention leases. TODO orly? - return null; - } - - @Override - protected ClusterBlockLevel indexBlockLevel() { - // see e.g. ShrinkIndexIT#testCreateShrinkIndexFails which tries to relocate shards in a read-only index, needing updates to the - // peer recovery retention leases. TODO orly? - return null; - } } From e9971f95970cd37c095fe626eabac32cf6b128b5 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 19 Feb 2019 18:48:38 +0000 Subject: [PATCH 04/32] Compile error --- .../elasticsearch/snapshots/SourceOnlySnapshotShardTests.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java index 28244b523e129..45ce8f3d7b1e9 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotShardTests.java @@ -48,6 +48,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.RetentionLeaseSyncer; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -208,7 +209,8 @@ public void testRestoreMinmal() throws IOException { new RecoverySource.SnapshotRecoverySource( UUIDs.randomBase64UUID(), new Snapshot("src_only", snapshotId), Version.CURRENT, indexId.getId())); IndexMetaData metaData = runAsSnapshot(threadPool, () -> repository.getSnapshotIndexMetaData(snapshotId, indexId)); - IndexShard restoredShard = newShard(shardRouting, metaData, null, SourceOnlySnapshotRepository.getEngineFactory(), () -> {}); + IndexShard restoredShard = newShard(shardRouting, metaData, null, SourceOnlySnapshotRepository.getEngineFactory(), () -> {}, + RetentionLeaseSyncer.EMPTY); restoredShard.mapperService().merge(shard.indexSettings().getIndexMetaData(), MapperService.MergeReason.MAPPING_RECOVERY); DiscoveryNode discoveryNode = new DiscoveryNode("node_g", buildNewFakeTransportAddress(), Version.CURRENT); restoredShard.markAsRecovering("test from snap", new RecoveryState(restoredShard.routingEntry(), discoveryNode, null)); From 316c4ed4a07b4c6dc0e837f6ed85efbbfe21f5ec Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 20 Feb 2019 10:35:12 +0000 Subject: [PATCH 05/32] contains -> hasItem --- .../java/org/elasticsearch/index/seqno/ReplicationTracker.java | 2 ++ .../java/org/elasticsearch/index/seqno/RetentionLeaseIT.java | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) 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 f45525408840d..b95621f0eaed9 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -665,6 +665,8 @@ private boolean invariant() { // a newly-recovered primary creates its own retention lease when entering primaryMode, which is done later, so it doesn't // exist yet + // TODO also expect to have a lease for tracked shard copies + // TODO what about relocating shards? for (ShardRouting shardRouting : routingTable.activeShards()) { assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : "no retention lease for active shard " + shardRouting + " in " + retentionLeases + " on " + shardAllocationId; 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 e5a96d02cc46d..e7de882932c66 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java @@ -57,6 +57,7 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) public class RetentionLeaseIT extends ESIntegTestCase { @@ -407,7 +408,7 @@ 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(primary.loadRetentionLeases().leases(), hasItem(retentionLease.get()))); } catch (final Exception e) { fail(e.toString()); } From 98e2708722d450e13c57e719776820329b7ce6bd Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 20 Feb 2019 10:39:28 +0000 Subject: [PATCH 06/32] Must renew leases now --- .../xpack/ccr/action/ShardChangesTests.java | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) 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..0bbd54ab5519a 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 @@ -16,11 +16,16 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.xpack.ccr.Ccr; import org.elasticsearch.xpack.ccr.LocalStateCcr; +import org.junit.Assert; import java.util.Collection; import java.util.Collections; @@ -110,6 +115,10 @@ public void testMissingOperations() throws Exception { client().prepareDelete("index", "_doc", "1").get(); client().admin().indices().flush(new FlushRequest("index").force(true)).actionGet(); } + + final IndexShard primary = getInstanceFromNode(IndicesService.class).getShardOrNull(new ShardId(resolveIndex("index"), 0)); + primary.runUnderPrimaryPermit(primary::renewPeerRecoveryRetentionLeaseForPrimary, Assert::assertNull, Names.SAME, ""); + client().admin().indices().refresh(new RefreshRequest("index")).actionGet(); ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index"); forceMergeRequest.maxNumSegments(1); @@ -117,7 +126,7 @@ public void testMissingOperations() throws Exception { ShardStats shardStats = client().admin().indices().prepareStats("index").get().getIndex("index").getShards()[0]; String historyUUID = shardStats.getCommitStats().getUserData().get(Engine.HISTORY_UUID_KEY); - ShardChangesAction.Request request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId(), historyUUID); + ShardChangesAction.Request request = new ShardChangesAction.Request(shardStats.getShardRouting().shardId(), historyUUID); request.setFromSeqNo(0L); request.setMaxOperationCount(1); From 4241335723e5622aa18ebe11ed12bf2ebcf15c2f Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 20 Feb 2019 13:42:31 +0000 Subject: [PATCH 07/32] Now renew/sync returns a future so that tests can wait --- .../org/elasticsearch/index/IndexService.java | 6 ++- ...erRecoveryRetentionLeaseRenewalAction.java | 12 +----- .../elasticsearch/index/shard/IndexShard.java | 37 +++++++++++++++++-- .../elasticsearch/indices/IndicesService.java | 4 +- .../cluster/IndicesClusterStateService.java | 6 +-- .../gateway/RecoveryFromGatewayIT.java | 12 +++--- .../index/shard/IndexShardIT.java | 2 +- ...dicesLifecycleListenerSingleNodeTests.java | 2 +- ...actIndicesClusterStateServiceTestCase.java | 2 +- ...ClusterStateServiceRandomUpdatesTests.java | 2 +- .../indices/recovery/IndexRecoveryIT.java | 12 +++--- .../indices/stats/IndexStatsIT.java | 37 +++++-------------- .../index/shard/IndexShardTestCase.java | 2 +- .../test/InternalTestCluster.java | 33 +++++++++++++++++ .../xpack/ccr/IndexFollowingIT.java | 5 +++ 15 files changed, 110 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 7774a5bb044d8..4d63d2e5f9406 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -26,6 +26,7 @@ import org.apache.lucene.util.Accountable; import org.elasticsearch.Assertions; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; @@ -88,6 +89,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -321,7 +323,7 @@ public synchronized IndexShard createShard( final ShardRouting routing, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final Consumer peerRecoveryRetentionLeaseRenewer) throws IOException { + final BiConsumer> peerRecoveryRetentionLeaseRenewer) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); Objects.requireNonNull(peerRecoveryRetentionLeaseRenewer); /* @@ -414,7 +416,7 @@ public synchronized IndexShard createShard( () -> globalCheckpointSyncer.accept(shardId), retentionLeaseSyncer, circuitBreakerService, - () -> peerRecoveryRetentionLeaseRenewer.accept(shardId)); + listener -> peerRecoveryRetentionLeaseRenewer.accept(shardId, listener)); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); diff --git a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java index 0fb0d79e5a6a6..791a134a63aa9 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java @@ -101,16 +101,8 @@ protected void handleReplicaResponse(ShardRouting shard, ReplicationOperation.Re .renewPeerRecoveryRetentionLeaseForReplica(shard, shardCopyResponse.localCheckpointOfSafeCommit); } - public void renewPeerRecoveryRetentionLease(ShardId shardId) { - execute(new Request(shardId), new ActionListener() { - @Override - public void onResponse(ReplicationResponse response) { - } - - @Override - public void onFailure(Exception e) { - } - }); + public void renewPeerRecoveryRetentionLease(ShardId shardId, ActionListener listener) { + execute(new Request(shardId), ActionListener.wrap(v -> listener.onResponse(null), listener::onFailure)); } static final class ShardCopyResponse extends ReplicaResponse { 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 682770fcd6e2c..f72686065770e 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -37,10 +37,12 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; @@ -159,6 +161,7 @@ import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; @@ -219,7 +222,7 @@ Runnable getGlobalCheckpointSyncer() { private final RetentionLeaseSyncer retentionLeaseSyncer; - private final Runnable peerRecoveryRetentionLeaseRenewer; + private final Consumer> peerRecoveryRetentionLeaseRenewer; @Nullable private RecoveryState recoveryState; @@ -280,7 +283,7 @@ public IndexShard( final Runnable globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, final CircuitBreakerService circuitBreakerService, - final Runnable peerRecoveryRetentionLeaseRenewer) throws IOException { + final Consumer> peerRecoveryRetentionLeaseRenewer) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); this.shardRouting = shardRouting; @@ -2049,6 +2052,28 @@ public void syncRetentionLeases() { } } + public Future foregroundSyncRetentionLeases() { + // TODO merge this with syncRetentionLeases, allowing to wait until completion of a backgroundSync too. + assert assertPrimaryMode(); + verifyNotClosed(); + final Tuple retentionLeases = getRetentionLeases(true); + logger.trace("foreground syncing retention leases [{}] after expiration check", retentionLeases.v2()); + final PlainActionFuture future = new PlainActionFuture<>(); + retentionLeaseSyncer.sync( + shardId, + retentionLeases.v2(), + ActionListener.wrap( + r -> future.onResponse(null), + e -> { + logger.warn(new ParameterizedMessage( + "failed to sync retention leases [{}] after expiration check", + retentionLeases), + e); + future.onFailure(e); + })); + return future; + } + /** * Waits for all operations up to the provided sequence number to complete. * @@ -2445,11 +2470,15 @@ public void renewPeerRecoveryRetentionLeaseForPrimary() { replicationTracker.renewPeerRecoveryRetentionLease(routingEntry(), getLocalCheckpointOfSafeCommit()); } - public void renewPeerRecoveryRetentionLeases() { + public Future renewPeerRecoveryRetentionLeases() { assert assertPrimaryMode(); + final PlainActionFuture plainActionFuture = new PlainActionFuture<>(); if (replicationTracker.peerRetentionLeasesNeedRenewal(getLocalCheckpointOfSafeCommit())) { - peerRecoveryRetentionLeaseRenewer.run(); + peerRecoveryRetentionLeaseRenewer.accept(plainActionFuture); + } else { + plainActionFuture.onResponse(null); } + return plainActionFuture; } public long getLocalCheckpointOfSafeCommit() { diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index e6a84083ebbc2..c7f1c97d6c16e 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -30,6 +30,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ResourceAlreadyExistsException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; @@ -143,6 +144,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.LongSupplier; @@ -629,7 +631,7 @@ public IndexShard createShard( final Consumer onShardFailure, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final Consumer peerRecoveryRetentionLeaseRenewer) throws IOException { + final BiConsumer> peerRecoveryRetentionLeaseRenewer) throws IOException { Objects.requireNonNull(retentionLeaseSyncer); ensureChangesAllowed(); IndexService indexService = indexService(shardRouting.index()); diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 27fab9d1a5222..d6fc194fb335e 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -128,7 +128,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final PrimaryReplicaSyncer primaryReplicaSyncer; private final Consumer globalCheckpointSyncer; private final RetentionLeaseSyncer retentionLeaseSyncer; - private final Consumer peerRecoveryRetentionLeaseRenewer; + private final BiConsumer> peerRecoveryRetentionLeaseRenewer; @Inject public IndicesClusterStateService( @@ -197,7 +197,7 @@ public void backgroundSync(final ShardId shardId, final RetentionLeases retentio final PrimaryReplicaSyncer primaryReplicaSyncer, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final Consumer peerRecoveryRetentionLeaseRenewer) { + final BiConsumer> peerRecoveryRetentionLeaseRenewer) { this.settings = settings; this.buildInIndexListener = Arrays.asList( @@ -926,7 +926,7 @@ T createShard( Consumer onShardFailure, Consumer globalCheckpointSyncer, RetentionLeaseSyncer retentionLeaseSyncer, - Consumer peerRecoveryRetentionLeaseRenewer) throws IOException; + BiConsumer> peerRecoveryRetentionLeaseRenewer) throws IOException; /** * Returns shard for the specified id if it exists otherwise returns null. diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 87c92ede576ae..e65e2d30f99fe 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -474,14 +474,12 @@ public Settings onNodeStopped(String nodeName) throws Exception { // expire retention lease for replica; since number_of_replicas is 0 it is no longer needed internalCluster().getInstance(IndicesService.class, primaryNode).indexServiceSafe(resolveIndex("test", primaryNode)) .forEach(is -> { - is.renewPeerRecoveryRetentionLeases(); - is.syncRetentionLeases(); try { - assertBusy(() -> { - assertThat(is.getRetentionLeases().leases().stream() - .filter(l -> PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(l.source())).count(), equalTo(1L)); - assertEquals(is.getMinRetainedSeqNo(), is.getLocalCheckpointOfSafeCommit() + 1); - }); + is.renewPeerRecoveryRetentionLeases().get(); + is.foregroundSyncRetentionLeases().get(); + assertThat(is.getRetentionLeases().leases().stream() + .filter(l -> PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(l.source())).count(), equalTo(1L)); + assertEquals(is.getMinRetainedSeqNo(), is.getLocalCheckpointOfSafeCommit() + 1); } catch (Exception e) { throw new AssertionError(e); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 3148cc3d35244..fbf8a458c24ce 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -677,7 +677,7 @@ public static final IndexShard newIndexShard( () -> {}, RetentionLeaseSyncer.EMPTY, cbs, - () -> {}); + l -> {}); } private static ShardRouting getInitializingShardRouting(ShardRouting existingShardRouting) { diff --git a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java index b54482e58474a..67e04bc5088c2 100644 --- a/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java +++ b/server/src/test/java/org/elasticsearch/indices/IndicesLifecycleListenerSingleNodeTests.java @@ -131,7 +131,7 @@ public void afterIndexRemoved(Index index, IndexSettings indexSettings, IndexRem newRouting = newRouting.moveToUnassigned(unassignedInfo) .updateUnassigned(unassignedInfo, RecoverySource.EmptyStoreRecoverySource.INSTANCE); newRouting = ShardRoutingHelper.initialize(newRouting, nodeId); - IndexShard shard = index.createShard(newRouting, s -> {}, RetentionLeaseSyncer.EMPTY, s -> {}); + IndexShard shard = index.createShard(newRouting, s -> {}, RetentionLeaseSyncer.EMPTY, (s, l) -> {}); IndexShardTestCase.updateRoutingEntry(shard, newRouting); assertEquals(5, counter.get()); final DiscoveryNode localNode = new DiscoveryNode("foo", buildNewFakeTransportAddress(), diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index 8a9515488bf4f..f5c4a0fa2b508 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -236,7 +236,7 @@ public MockIndexShard createShard( final Consumer onShardFailure, final Consumer globalCheckpointSyncer, final RetentionLeaseSyncer retentionLeaseSyncer, - final Consumer peerRecoveryRetentionleaseRenewer) throws IOException { + final BiConsumer> peerRecoveryRetentionLeaseRenewer) throws IOException { failRandomly(); MockIndexService indexService = indexService(recoveryState.getShardId().getIndex()); MockIndexShard indexShard = indexService.createShard(shardRouting); diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java index ac7ae413d15a7..b723023a1323d 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/IndicesClusterStateServiceRandomUpdatesTests.java @@ -483,7 +483,7 @@ private IndicesClusterStateService createIndicesClusterStateService(DiscoveryNod primaryReplicaSyncer, s -> {}, RetentionLeaseSyncer.EMPTY, - s -> {}); + (s, l) -> {}); } private class RecordingIndicesService extends MockIndicesService { 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 049393198a21b..60abb45e314f2 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; @@ -32,6 +33,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; 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.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -853,12 +855,12 @@ public void testHistoryRetention() throws Exception { final IndexShard primary = requireNonNull(indexShardsByPrimary.get(true)); assertThat(client().admin().indices().prepareFlush().setForce(true).get().getFailedShards(), equalTo(0)); // make a safe commit - primary.syncRetentionLeases(); // happens periodically, removes retention leases for old shard copies - primary.renewPeerRecoveryRetentionLeases(); // happens periodically, advances retention leases according to last safe commit - assertBusy(() -> assertThat(primary.getMinRetainedSeqNo(), equalTo(primary.seqNoStats().getMaxSeqNo() + 1))); - primary.syncRetentionLeases(); // happens periodically, pushes updated retention leases to replica + primary.getRetentionLeases(true); // happens periodically, expires leases for unassigned shards + primary.renewPeerRecoveryRetentionLeases().get(); // happens periodically, advances retention leases according to last safe commit + assertThat(primary.getMinRetainedSeqNo(), equalTo(primary.seqNoStats().getMaxSeqNo() + 1)); + primary.foregroundSyncRetentionLeases().get(); // happens periodically, pushes updated retention leases to replica final IndexShard replica = requireNonNull(indexShardsByPrimary.get(false)); - assertBusy(() -> assertThat(replica.getMinRetainedSeqNo(), equalTo(replica.seqNoStats().getMaxSeqNo() + 1))); + assertThat(replica.getMinRetainedSeqNo(), equalTo(replica.seqNoStats().getMaxSeqNo() + 1)); } } 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 6e0be22f871a6..c3e2848d36313 100644 --- a/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/stats/IndexStatsIT.java @@ -1057,7 +1057,16 @@ public void testFilterCacheStats() throws Exception { if (IndexSettings.INDEX_SOFT_DELETES_SETTING.get(settings)) { persistGlobalCheckpoint("index"); flush("index"); - releaseHistory("index"); + internalCluster().renewAndSyncPeerRecoveryRetentionLeases(resolveIndex("index")); + for (String node : internalCluster().nodesInclude("index")) { + final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); + for (IndexService indexService : indexServices) { + for (IndexShard indexShard : indexService) { + assertFalse(indexShard.routingEntry().toString(), + indexShard.hasCompleteHistoryOperations("test", indexShard.getLocalCheckpointOfSafeCommit())); + } + } + } } ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge("index").setFlush(true).setMaxNumSegments(1).get(); @@ -1214,30 +1223,4 @@ private void persistGlobalCheckpoint(String index) throws Exception { } } } - - private void releaseHistory(String index) throws Exception { - // TODO maybe we want an (internal) API to await the release of history, rather than busy-waiting like this? - final Set nodes = internalCluster().nodesInclude(index); - for (String node : nodes) { - final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - if (indexShard.routingEntry().primary()) { - indexShard.renewPeerRecoveryRetentionLeases(); - } - } - } - } - assertBusy(() -> { - for (String node : nodes) { - final IndicesService indexServices = internalCluster().getInstance(IndicesService.class, node); - for (IndexService indexService : indexServices) { - for (IndexShard indexShard : indexService) { - assertFalse(indexShard.routingEntry().toString(), - indexShard.hasCompleteHistoryOperations("test", indexShard.getLocalCheckpointOfSafeCommit())); - } - } - } - }); - } } 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 dfeffa70eeffe..30e8c915a4f32 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 @@ -392,7 +392,7 @@ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMe globalCheckpointSyncer, retentionLeaseSyncer, breakerService, - () -> {}); + l -> {}); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); success = true; } finally { diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 2c02abab9dc1d..50ac397099f54 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -31,6 +31,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -139,6 +140,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -861,6 +863,37 @@ public synchronized void close() throws IOException { public static final int REMOVED_MINIMUM_MASTER_NODES = Integer.MAX_VALUE; + /** + * Renews the peer recovery retention leases for the given indices (updating each lease to the local checkpoint of the safe commit) + * and pushes the updated leases out to all the replicas. + */ + public void renewAndSyncPeerRecoveryRetentionLeases(Index... indices) { + final List indexShards = new ArrayList<>(); + for (final IndicesService indicesService : getInstances(IndicesService.class)) { + for (final Index index : indices) { + final IndexService indexService = indicesService.indexService(index); + if (indexService != null) { + for (IndexShard indexShard : indexService) { + if (indexShard.routingEntry().primary()) { + indexShards.add(indexShard); + } + } + } + } + } + + final Consumer> futureConsumer = f -> { + try { + f.get(); + } catch (Exception e) { + throw new AssertionError(e); + } + }; + + indexShards.stream().map(IndexShard::renewPeerRecoveryRetentionLeases).forEach(futureConsumer); + indexShards.stream().map(IndexShard::foregroundSyncRetentionLeases).forEach(futureConsumer); + } + private final class NodeAndClient implements Closeable { private MockNode node; private final Settings originalNodeSettings; 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 5a8a7feb34716..cf4ca72a9d5c0 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 @@ -59,9 +59,12 @@ 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.RetentionLeaseActions; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.SnapshotRestoreException; @@ -1087,6 +1090,8 @@ 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(); + getLeaderCluster().renewAndSyncPeerRecoveryRetentionLeases(resolveLeaderIndex("index1")); + ForceMergeRequest forceMergeRequest = new ForceMergeRequest("index1"); forceMergeRequest.maxNumSegments(1); leaderClient().admin().indices().forceMerge(forceMergeRequest).actionGet(); From 5d2f50afc372e1716e53a07f573a829b6b0ad19a Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 21 Feb 2019 07:44:51 +0000 Subject: [PATCH 08/32] WIP on CcrRetentionLeaseIT --- .../index/seqno/RetentionLease.java | 3 ++ ...ReplicationTrackerRetentionLeaseTests.java | 10 ++--- .../xpack/ccr/CcrRetentionLeaseIT.java | 39 ++++++++++++------- 3 files changed, 30 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index e6d6ed3fe825f..035f2a8b97af1 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -211,4 +211,7 @@ public String toString() { '}'; } + public boolean isNotPeerRecoveryRetentionLease() { + return ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(source) == false; + } } 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 52200f6a661c3..61813536e2d04 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -43,6 +43,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.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; @@ -602,13 +603,8 @@ private void assertRetentionLeases( } assertThat(retentionLeases.primaryTerm(), equalTo(primaryTerm)); assertThat(retentionLeases.version(), equalTo(version)); - final Map idToRetentionLease = new HashMap<>(); - for (final RetentionLease retentionLease : retentionLeases.leases()) { - if (PEER_RECOVERY_RETENTION_LEASE_SOURCE.equals(retentionLease.source()) == false) { - idToRetentionLease.put(retentionLease.id(), retentionLease); - } - } - + final Map idToRetentionLease = retentionLeases.leases().stream() + .filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toMap(RetentionLease::id, Function.identity())); assertThat(idToRetentionLease.entrySet(), hasSize(size)); for (int i = 0; i < size; i++) { assertThat(idToRetentionLease.keySet(), hasItem(Integer.toString(i))); 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 c42887e6b52f1..dca61eb953a7f 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 @@ -28,6 +28,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeases; import org.elasticsearch.plugins.Plugin; @@ -50,6 +51,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -160,13 +162,15 @@ public void testRetentionLeaseIsTakenAtTheStartOfRecovery() throws Exception { assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { - final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); + final List ccrLeases + = shardStats.get(i).getRetentionLeaseStats().retentionLeases() + .leases().stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease) + .collect(Collectors.toList()); + assertThat(ccrLeases, 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(); + final RetentionLease retentionLease = ccrLeases.iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); } }); @@ -314,7 +318,7 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws final String leaderUUID = leaderIndexClusterState.getState().metaData().index(leaderIndex).getIndexUUID(); // sample the leases after recovery - final List retentionLeases = new ArrayList<>(); + final List> retentionLeases = new ArrayList<>(); assertBusy(() -> { retentionLeases.clear(); final IndicesStatsResponse stats = @@ -323,20 +327,23 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { - final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); + final List ccrLeases + = shardStats.get(i).getRetentionLeaseStats().retentionLeases() + .leases().stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease) + .collect(Collectors.toList()); + assertThat(ccrLeases, 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(); + final RetentionLease retentionLease = ccrLeases.iterator().next(); + assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, followerUUID, leaderIndex, leaderUUID))); final String expectedRetentionLeaseId = retentionLeaseId( getFollowerCluster().getClusterName(), new Index(followerIndex, followerUUID), getLeaderCluster().getClusterName(), new Index(leaderIndex, leaderUUID)); assertThat(retentionLease.id(), equalTo(expectedRetentionLeaseId)); - retentionLeases.add(currentRetentionLeases); + retentionLeases.add(() -> retentionLease); } }); @@ -351,16 +358,18 @@ public void testRetentionLeasesAreNotBeingRenewedAfterRecoveryCompletes() throws assertThat(stats.getShards(), arrayWithSize(numberOfShards * (1 + numberOfReplicas))); final List shardStats = getShardStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { - final RetentionLeases currentRetentionLeases = shardStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); + final List ccrLeases + = shardStats.get(i).getRetentionLeaseStats().retentionLeases() + .leases().stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease) + .collect(Collectors.toList()); + assertThat(ccrLeases, 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(); + final RetentionLease retentionLease = ccrLeases.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).get().timestamp())); } }); From 08a6831e97395c1d21537af2b962302a7e8e9ee6 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:12:50 +0000 Subject: [PATCH 09/32] We already set waitForActiveShards --- .../index/seqno/RetentionLeaseBackgroundSyncAction.java | 5 ++--- .../elasticsearch/index/seqno/RetentionLeaseSyncAction.java | 5 ++--- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java index b57934fae609b..d454c2de75b28 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseBackgroundSyncAction.java @@ -108,10 +108,8 @@ public void backgroundSync( try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the sync is authorized threadContext.markAsSystemContext(); - final Request request = new Request(shardId, retentionLeases); - request.waitForActiveShards(ActiveShardCount.ONE); execute( - request, + new Request(shardId, retentionLeases), ActionListener.wrap( r -> {}, e -> { @@ -189,4 +187,5 @@ public String toString() { protected ReplicationResponse newResponseInstance() { return new ReplicationResponse(); } + } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index cfd34802ef3b4..d4845d92a3a6f 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -109,9 +109,8 @@ public void sync( try (ThreadContext.StoredContext ignore = threadContext.stashContext()) { // we have to execute under the system context so that if security is enabled the sync is authorized threadContext.markAsSystemContext(); - final Request request = new Request(shardId, retentionLeases); - request.waitForActiveShards(ActiveShardCount.ONE); - execute(request, + execute( + new RetentionLeaseSyncAction.Request(shardId, retentionLeases), ActionListener.wrap( listener::onResponse, e -> { From d2ab44378d964290f4eb3eee64cbe51210942024 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:21:41 +0000 Subject: [PATCH 10/32] Imports --- .../src/main/java/org/elasticsearch/index/shard/IndexShard.java | 1 - 1 file changed, 1 deletion(-) 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 0e0cd058fdd65..04f9e54f71034 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -37,7 +37,6 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; From 4cae8c5fd0c38a480d102294aaaabf7dcff12c9b Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:22:44 +0000 Subject: [PATCH 11/32] More imports --- .../index/seqno/ReplicationTrackerRetentionLeaseTests.java | 1 - .../org/elasticsearch/indices/recovery/IndexRecoveryIT.java | 2 -- 2 files changed, 3 deletions(-) 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 61813536e2d04..0718a40e1b808 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -46,7 +46,6 @@ import java.util.function.Function; import java.util.stream.Collectors; -import static org.elasticsearch.index.seqno.ReplicationTracker.PEER_RECOVERY_RETENTION_LEASE_SOURCE; import static org.elasticsearch.index.seqno.ReplicationTracker.getPeerRecoveryRetentionLeaseId; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.contains; 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 60abb45e314f2..1253c77160cab 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/IndexRecoveryIT.java @@ -20,7 +20,6 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; @@ -33,7 +32,6 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; 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.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexMetaData; From 6639a579487e0c9be19f56b202142efa8862bd22 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:31:41 +0000 Subject: [PATCH 12/32] Imports --- .../main/java/org/elasticsearch/test/InternalTestCluster.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index 27a8ed8b572f0..759f4da3692a0 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -31,7 +31,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; @@ -155,8 +154,8 @@ import static org.elasticsearch.discovery.DiscoveryModule.ZEN2_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoveryModule.ZEN_DISCOVERY_TYPE; import static org.elasticsearch.discovery.DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING; -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.FileBasedSeedHostsProvider.UNICAST_HOSTS_FILE; +import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.test.ESTestCase.assertBusy; import static org.elasticsearch.test.ESTestCase.awaitBusy; import static org.elasticsearch.test.ESTestCase.getTestTransportType; From fdd1c973b4f93ae4ffc5c26cae2c9d34fced9a55 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:45:07 +0000 Subject: [PATCH 13/32] Imports --- .../java/org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java | 1 - .../java/org/elasticsearch/xpack/ccr/IndexFollowingIT.java | 3 --- 2 files changed, 4 deletions(-) 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 25e22ea103e4f..bb99ba0a5b50d 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 @@ -31,7 +31,6 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseActions; import org.elasticsearch.index.seqno.RetentionLeases; 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 e87cb804332ce..e447cecd80fa7 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 @@ -60,12 +60,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.RetentionLeaseActions; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.IndicesService; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.SnapshotRestoreException; From 63fb8182e2b4d8e1e45066949e20366f77b56933 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:45:46 +0000 Subject: [PATCH 14/32] Revert --- .../index/replication/ESIndexLevelReplicationTestCase.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 a4160d2a6ad35..85e69de8824c9 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 @@ -101,7 +101,6 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; -import static org.elasticsearch.action.ActionListener.wrap; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -256,7 +255,7 @@ public BulkItemResponse delete(DeleteRequest deleteRequest) throws Exception { private BulkItemResponse executeWriteRequest( DocWriteRequest writeRequest, WriteRequest.RefreshPolicy refreshPolicy) throws Exception { PlainActionFuture listener = new PlainActionFuture<>(); - final ActionListener wrapBulkListener = wrap( + final ActionListener wrapBulkListener = ActionListener.wrap( bulkShardResponse -> listener.onResponse(bulkShardResponse.getResponses()[0]), listener::onFailure); BulkItemRequest[] items = new BulkItemRequest[1]; From 12064acf2b296ea59f0c158ea0ae9021e254c959 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 10:50:36 +0000 Subject: [PATCH 15/32] Set up initial replication targets for retention lease syncing --- .../index/replication/ESIndexLevelReplicationTestCase.java | 1 + 1 file changed, 1 insertion(+) 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 85e69de8824c9..9e1c30e9e0036 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 @@ -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()); From 8379650b0bf40e2cbf8f06b5d7b7e4d965fd1d3d Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 11:08:45 +0000 Subject: [PATCH 16/32] Fix new test --- .../replication/RetentionLeasesReplicationTests.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) 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..8052a9e54f1d4 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.seqno.RetentionLeases; @@ -35,6 +36,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -44,7 +46,8 @@ public class RetentionLeasesReplicationTests extends ESIndexLevelReplicationTest public void testSimpleSyncRetentionLeases() throws Exception { Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true).build(); - try (ReplicationGroup group = createGroup(between(0, 2), settings)) { + final int numberOfReplicas = between(0, 2); + try (ReplicationGroup group = createGroup(numberOfReplicas, settings)) { group.startAll(); List leases = new ArrayList<>(); int iterations = between(1, 100); @@ -61,9 +64,10 @@ public void testSimpleSyncRetentionLeases() throws Exception { } } RetentionLeases leasesOnPrimary = group.getPrimary().getRetentionLeases(); - assertThat(leasesOnPrimary.version(), equalTo((long) iterations)); + assertThat(leasesOnPrimary.version(), equalTo(iterations + numberOfReplicas + 1L)); assertThat(leasesOnPrimary.primaryTerm(), equalTo(group.getPrimary().getOperationPrimaryTerm())); - assertThat(leasesOnPrimary.leases(), containsInAnyOrder(leases.toArray(new RetentionLease[0]))); + assertThat(leasesOnPrimary.leases().stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease) + .collect(Collectors.toList()), containsInAnyOrder(leases.toArray(new RetentionLease[0]))); latch.await(); for (IndexShard replica : group.getReplicas()) { assertThat(replica.getRetentionLeases(), equalTo(leasesOnPrimary)); From d5707467791dbb5e75746ee7827653be914c1c7d Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 11:20:11 +0000 Subject: [PATCH 17/32] hasItem --- .../java/org/elasticsearch/index/seqno/RetentionLeaseIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 452660f5e2d2b..31d747d0454b1 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseIT.java @@ -526,7 +526,7 @@ 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(primary.loadRetentionLeases().leases(), hasItem(retentionLease.get()))); } catch (final Exception e) { fail(e.toString()); } From 884d84e4122b33825e2f105869690fa21ef996fe Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 14:13:21 +0000 Subject: [PATCH 18/32] Fix testSnapshotFileFailureDuringSnapshot Today this test catches an exception and asserts that its proximate cause has message `Random IOException` but occasionally this exception is wrapped two layers deep, causing the test to fail. This commit adjusts the test to look at the root cause of the exception instead. 1> [2019-02-25T12:31:50,837][INFO ][o.e.s.SharedClusterSnapshotRestoreIT] [testSnapshotFileFailureDuringSnapshot] --> caught a top level exception, asserting what's expected 1> org.elasticsearch.snapshots.SnapshotException: [test-repo:test-snap/e-hn_pLGRmOo97ENEXdQMQ] Snapshot could not be read 1> at org.elasticsearch.snapshots.SnapshotsService.snapshots(SnapshotsService.java:212) ~[main/:?] 1> at org.elasticsearch.action.admin.cluster.snapshots.get.TransportGetSnapshotsAction.masterOperation(TransportGetSnapshotsAction.java:135) ~[main/:?] 1> at org.elasticsearch.action.admin.cluster.snapshots.get.TransportGetSnapshotsAction.masterOperation(TransportGetSnapshotsAction.java:54) ~[main/:?] 1> at org.elasticsearch.action.support.master.TransportMasterNodeAction.masterOperation(TransportMasterNodeAction.java:127) ~[main/:?] 1> at org.elasticsearch.action.support.master.TransportMasterNodeAction$AsyncSingleAction$2.doRun(TransportMasterNodeAction.java:208) ~[main/:?] 1> at org.elasticsearch.common.util.concurrent.ThreadContext$ContextPreservingAbstractRunnable.doRun(ThreadContext.java:751) ~[main/:?] 1> at org.elasticsearch.common.util.concurrent.AbstractRunnable.run(AbstractRunnable.java:37) ~[main/:?] 1> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[?:1.8.0_202] 1> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[?:1.8.0_202] 1> at java.lang.Thread.run(Thread.java:748) [?:1.8.0_202] 1> Caused by: org.elasticsearch.snapshots.SnapshotException: [test-repo:test-snap/e-hn_pLGRmOo97ENEXdQMQ] failed to get snapshots 1> at org.elasticsearch.repositories.blobstore.BlobStoreRepository.getSnapshotInfo(BlobStoreRepository.java:564) ~[main/:?] 1> at org.elasticsearch.snapshots.SnapshotsService.snapshots(SnapshotsService.java:206) ~[main/:?] 1> ... 9 more 1> Caused by: java.io.IOException: Random IOException 1> at org.elasticsearch.snapshots.mockstore.MockRepository$MockBlobStore$MockBlobContainer.maybeIOExceptionOrBlock(MockRepository.java:275) ~[test/:?] 1> at org.elasticsearch.snapshots.mockstore.MockRepository$MockBlobStore$MockBlobContainer.readBlob(MockRepository.java:317) ~[test/:?] 1> at org.elasticsearch.repositories.blobstore.ChecksumBlobStoreFormat.readBlob(ChecksumBlobStoreFormat.java:101) ~[main/:?] 1> at org.elasticsearch.repositories.blobstore.BlobStoreFormat.read(BlobStoreFormat.java:90) ~[main/:?] 1> at org.elasticsearch.repositories.blobstore.BlobStoreRepository.getSnapshotInfo(BlobStoreRepository.java:560) ~[main/:?] 1> at org.elasticsearch.snapshots.SnapshotsService.snapshots(SnapshotsService.java:206) ~[main/:?] 1> ... 9 more FAILURE 0.59s J0 | SharedClusterSnapshotRestoreIT.testSnapshotFileFailureDuringSnapshot <<< FAILURES! > Throwable #1: java.lang.AssertionError: > Expected: a string containing "Random IOException" > but: was "[test-repo:test-snap/e-hn_pLGRmOo97ENEXdQMQ] failed to get snapshots" > at __randomizedtesting.SeedInfo.seed([B73CA847D4B4F52D:884E042D2D899330]:0) > at org.hamcrest.MatcherAssert.assertThat(MatcherAssert.java:20) > at org.elasticsearch.snapshots.SharedClusterSnapshotRestoreIT.testSnapshotFileFailureDuringSnapshot(SharedClusterSnapshotRestoreIT.java:821) > at java.lang.Thread.run(Thread.java:748) --- .../snapshots/SharedClusterSnapshotRestoreIT.java | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index f6de7cade2697..ad1415cdd98e6 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -20,6 +20,7 @@ package org.elasticsearch.snapshots; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionFuture; @@ -768,7 +769,7 @@ public void testIncludeGlobalState() throws Exception { } - public void testSnapshotFileFailureDuringSnapshot() throws Exception { + public void testSnapshotFileFailureDuringSnapshot() { Client client = client(); logger.info("--> creating repository"); @@ -817,8 +818,12 @@ public void testSnapshotFileFailureDuringSnapshot() throws Exception { } catch (Exception ex) { logger.info("--> caught a top level exception, asserting what's expected", ex); assertThat(getFailureCount("test-repo"), greaterThan(0L)); - assertThat(ex.getCause(), notNullValue()); - assertThat(ex.getCause().getMessage(), containsString("Random IOException")); + + final Throwable cause = ex.getCause(); + assertThat(cause, notNullValue()); + final Throwable rootCause = new ElasticsearchException(cause).getRootCause(); + assertThat(rootCause, notNullValue()); + assertThat(rootCause.getMessage(), containsString("Random IOException")); } } From fd7198409ef8d7c0548b73d20c48a5ccb1fed31c Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 16:35:09 +0000 Subject: [PATCH 19/32] Finer assertion --- .../indices/recovery/RecoverySourceHandler.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) 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 6dd410c5c03f2..d19ad1ae5a874 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -173,9 +173,11 @@ public void recoverToTarget(ActionListener listener) { final SendFileResult sendFileResult; final StepListener addPeerRecoveryRetentionLeaseStep = new StepListener<>(); + final long requiredSeqNoRangeStart; if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); + requiredSeqNoRangeStart = startingSeqNo; sendFileResult = SendFileResult.EMPTY; if (shard.indexSettings().isSoftDeleteEnabled()) { @@ -192,7 +194,7 @@ public void recoverToTarget(ActionListener listener) { } // We must have everything above the local checkpoint in the commit - final long requiredSeqNoRangeStart + requiredSeqNoRangeStart = Long.parseLong(phase1Snapshot.getIndexCommit().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) + 1; // If soft-deletes enabled, we need to transfer only operations after the local_checkpoint of the commit to have // the same history on the target. However, with translog, we need to set this to 0 to create a translog roughly @@ -225,7 +227,7 @@ public void recoverToTarget(ActionListener listener) { } }, shardId + " adding peer-recovery retention lease for " + request.targetNode(), shard, cancellableThreads, logger); } else { - assert retentionLease.get().retainingSequenceNumber() <= startingSeqNo + assert retentionLease.get().retainingSequenceNumber() <= requiredSeqNoRangeStart : shard.shardId() + ": seqno " + startingSeqNo + " should be retained by " + retentionLease.get(); // The target shard has a lease (and it is now in the routing table so its lease will not expire) addPeerRecoveryRetentionLeaseStep.onResponse(null); From ee3efa761e46c82364a54a5882405bf5bfe031f0 Mon Sep 17 00:00:00 2001 From: David Turner Date: Mon, 25 Feb 2019 16:59:59 +0000 Subject: [PATCH 20/32] Imports --- .../index/replication/RetentionLeasesReplicationTests.java | 1 - 1 file changed, 1 deletion(-) 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 8052a9e54f1d4..dd556156c9a99 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RetentionLeasesReplicationTests.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.RetentionLeaseSyncAction; import org.elasticsearch.index.seqno.RetentionLeases; From b2ec2d00b370c5bd5b94455a2f3587f1e450bee8 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 26 Feb 2019 11:15:00 +0000 Subject: [PATCH 21/32] Fix up CcrRetentionLeaseIT --- .../xpack/ccr/CcrRetentionLeaseIT.java | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) 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 bb99ba0a5b50d..62b5fa380c111 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 @@ -244,9 +244,10 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + final List ccrLeases = currentRetentionLeases.leases().stream() + .filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toList()); + assertThat(ccrLeases, hasSize(1)); + final RetentionLease retentionLease = ccrLeases.iterator().next(); assertThat(retentionLease.id(), equalTo(getRetentionLeaseId(followerIndex, leaderIndex))); retentionLeases.add(currentRetentionLeases); } @@ -261,12 +262,13 @@ public void testRetentionLeaseIsRenewedDuringRecovery() throws Exception { final List shardsStats = getShardsStats(stats); for (int i = 0; i < numberOfShards * (1 + numberOfReplicas); i++) { final RetentionLeases currentRetentionLeases = shardsStats.get(i).getRetentionLeaseStats().retentionLeases(); - assertThat(currentRetentionLeases.leases(), hasSize(1)); - final RetentionLease retentionLease = - currentRetentionLeases.leases().iterator().next(); + final List ccrLeases = currentRetentionLeases.leases().stream() + .filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toList()); + assertThat(ccrLeases, hasSize(1)); + final RetentionLease retentionLease = ccrLeases.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).get(retentionLease.id()).timestamp())); } }); latch.countDown(); @@ -423,9 +425,11 @@ 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(shardStats.getRetentionLeaseStats().retentionLeases().leases(), hasSize(1)); + final Collection ccrLeases = shardStats.getRetentionLeaseStats().retentionLeases().leases() + .stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toList()); + assertThat(ccrLeases, hasSize(1)); assertThat( - shardStats.getRetentionLeaseStats().retentionLeases().leases().iterator().next().id(), + ccrLeases.iterator().next().id(), equalTo(retentionLeaseId)); } @@ -488,7 +492,9 @@ 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(shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty()); + final Collection ccrLeases = shardStats.getRetentionLeaseStats().retentionLeases().leases() + .stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toList()); + assertThat(ccrLeases, empty()); } } finally { for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { From 30cf6d4eb1a6d873bbca4e29fdf31a1e8491f9c7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 27 Feb 2019 12:14:45 +0000 Subject: [PATCH 22/32] Fix up test --- .../org/elasticsearch/xpack/ccr/CcrRetentionLeaseIT.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) 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 d03981608e787..de24b31fa2dd7 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 @@ -726,8 +726,8 @@ public void testRetentionLeaseRenewalIsCancelledWhenFollowingIsPaused() throws E final String followerUUID = followerIndexClusterState.getState().metaData().index(followerIndex).getIndexUUID(); final RetentionLease retentionLease = ccrLeases.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())); + // we assert that retentxion leases are not being renewed by an unchanged timestamp + assertThat(retentionLease.timestamp(), equalTo(retentionLeases.get(i).get(retentionLease.id()).timestamp())); } }); } @@ -925,7 +925,9 @@ public void onResponseReceived( leaderClient().admin().indices().stats(new IndicesStatsRequest().clear().indices(leaderIndex)).actionGet(); final List afterUnfollowShardsStats = getShardsStats(afterUnfollowStats); for (final ShardStats shardStats : afterUnfollowShardsStats) { - assertThat(shardStats.getRetentionLeaseStats().retentionLeases().leases(), empty()); + final Collection ccrLeases = shardStats.getRetentionLeaseStats().retentionLeases().leases() + .stream().filter(RetentionLease::isNotPeerRecoveryRetentionLease).collect(Collectors.toList()); + assertThat(ccrLeases, empty()); } } finally { for (final ObjectCursor senderNode : followerClusterState.getState().nodes().getDataNodes().values()) { From 6709175861032c1beaa9516693aee44946b061fc Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 27 Feb 2019 12:40:23 +0000 Subject: [PATCH 23/32] WIP sharing LCPoSC with every replication response --- .../TransportResyncReplicationAction.java | 2 +- .../replication/ReplicationOperation.java | 19 ++++-- .../TransportReplicationAction.java | 65 +++++++++++-------- .../seqno/GlobalCheckpointSyncAction.java | 2 +- ...erRecoveryRetentionLeaseRenewalAction.java | 49 +------------- .../elasticsearch/index/shard/IndexShard.java | 12 ++++ ...portVerifyShardBeforeCloseActionTests.java | 6 +- .../ReplicationOperationTests.java | 22 ++++++- .../TransportReplicationActionTests.java | 2 +- .../TransportWriteActionTests.java | 2 +- .../ESIndexLevelReplicationTestCase.java | 8 ++- 11 files changed, 102 insertions(+), 87 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java index e9a6e7b48152d..18edbc6f90f60 100644 --- a/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/resync/TransportResyncReplicationAction.java @@ -99,7 +99,7 @@ protected void sendReplicaRequest( super.sendReplicaRequest(replicaRequest, node, listener); } else { final long pre60NodeCheckpoint = SequenceNumbers.PRE_60_NODE_CHECKPOINT; - listener.onResponse(new ReplicaResponse(pre60NodeCheckpoint, pre60NodeCheckpoint)); + listener.onResponse(new ReplicaResponse(pre60NodeCheckpoint, pre60NodeCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO)); } } diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 1f9223da175a3..ede1726129063 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -155,9 +155,6 @@ private void performOnReplicas(final ReplicaRequest replicaRequest, final long g } } - protected void handleReplicaResponse(final ShardRouting shard, final ReplicaResponse response) { - } - private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest, final long globalCheckpoint, final long maxSeqNoOfUpdatesOrDeletes) { if (logger.isTraceEnabled()) { @@ -173,6 +170,7 @@ public void onResponse(ReplicaResponse response) { try { primary.updateLocalCheckpointForShard(shard.allocationId().getId(), response.localCheckpoint()); primary.updateGlobalCheckpointForShard(shard.allocationId().getId(), response.globalCheckpoint()); + primary.updateLocalCheckpointOfSafeCommitForShard(shard.allocationId().getId(), response.localCheckpointOfSafeCommit()); } catch (final AlreadyClosedException e) { // okay, the index was deleted or this shard was never activated after a relocation; fall through and finish normally } catch (final Exception e) { @@ -180,7 +178,6 @@ public void onResponse(ReplicaResponse response) { final String message = String.format(Locale.ROOT, "primary failed updating local checkpoint for replica %s", shard); primary.failShard(message, e); } - handleReplicaResponse(shard, response); decPendingAndFinishIfNeeded(); } @@ -323,6 +320,14 @@ public interface Primary< */ void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint); + /** + * Update the local knowledge of the local checkpoint of the safe commit for the specified allocation ID. + * + * @param allocationId the allocation ID to update the local checkpoint of the safe commit for + * @param localCheckpointOfSafeCommit the local checkpoint of the safe commit + */ + void updateLocalCheckpointOfSafeCommitForShard(String allocationId, long localCheckpointOfSafeCommit); + /** * Returns the local checkpoint on the primary shard. * @@ -420,6 +425,12 @@ public interface ReplicaResponse { **/ long globalCheckpoint(); + /** + * The local checkpoint of the safe commit for the shard copy. + * + * @return the local checkpoint of the safe commit + **/ + long localCheckpointOfSafeCommit(); } public static class RetryOnPrimaryException extends ElasticsearchException { diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 55380be193065..1532b19986695 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -32,7 +32,6 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.action.support.TransportActions; -import org.elasticsearch.action.support.replication.ReplicationOperation.ReplicaResponse; import org.elasticsearch.client.transport.NoNodeAvailableException; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; @@ -476,19 +475,11 @@ public void onFailure(Exception e) { protected ReplicationOperation> createReplicatedOperation( Request request, ActionListener> listener, PrimaryShardReference primaryShardReference) { - return new ReplicationOperation> - (request, primaryShardReference, listener, newReplicasProxy(primaryTerm), logger, actionName) { - @Override - protected void handleReplicaResponse(ShardRouting shard, ReplicaResponse response) { - TransportReplicationAction.this.handleReplicaResponse(shard, response); - } - }; + return new ReplicationOperation<>(request, primaryShardReference, listener, + newReplicasProxy(primaryTerm), logger, actionName); } } - protected void handleReplicaResponse(ShardRouting shard, ReplicationOperation.ReplicaResponse response) { - } - protected static class PrimaryResult, Response extends ReplicationResponse> implements ReplicationOperation.PrimaryResult { @@ -552,10 +543,6 @@ public void respond(ActionListener listener) { listener.onFailure(finalFailure); } } - - public ReplicaResponse getReplicaResponse(IndexShard replica) { - return new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint()); - } } public class ReplicaOperationTransportHandler implements TransportRequestHandler> { @@ -632,7 +619,10 @@ public void onResponse(Releasable releasable) { try { final ReplicaResult replicaResult = shardOperationOnReplica(request, replica); releasable.close(); // release shard operation lock before responding to caller - replicaResult.respond(new ResponseListener(replicaResult.getReplicaResponse(replica))); + final TransportReplicationAction.ReplicaResponse response = + new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint(), + replica.getLocalCheckpointOfSafeCommit()); + replicaResult.respond(new ResponseListener(response)); } catch (final Exception e) { Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller AsyncReplicaAction.this.onFailure(e); @@ -1058,6 +1048,11 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long indexShard.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); } + @Override + public void updateLocalCheckpointOfSafeCommitForShard(final String allocationId, final long localCheckpointOfSafeCommit) { + indexShard.updateLocalCheckpointOfSafeCommitForShard(allocationId, localCheckpointOfSafeCommit); + } + @Override public long localCheckpoint() { return indexShard.getLocalCheckpoint(); @@ -1083,12 +1078,13 @@ public ReplicationGroup getReplicationGroup() { public static class ReplicaResponse extends ActionResponse implements ReplicationOperation.ReplicaResponse { private long localCheckpoint; private long globalCheckpoint; + private long localCheckpointOfSafeCommit; - public ReplicaResponse() { + ReplicaResponse() { } - public ReplicaResponse(long localCheckpoint, long globalCheckpoint) { + public ReplicaResponse(long localCheckpoint, long globalCheckpoint, long localCheckpointOfSafeCommit) { /* * A replica should always know its own local checkpoints so this should always be a valid sequence number or the pre-6.0 * checkpoint value when simulating responses to replication actions that pre-6.0 nodes are not aware of (e.g., the global @@ -1097,6 +1093,7 @@ public ReplicaResponse(long localCheckpoint, long globalCheckpoint) { assert localCheckpoint != SequenceNumbers.UNASSIGNED_SEQ_NO; this.localCheckpoint = localCheckpoint; this.globalCheckpoint = globalCheckpoint; + this.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; } @Override @@ -1113,6 +1110,11 @@ public void readFrom(StreamInput in) throws IOException { } else { globalCheckpoint = SequenceNumbers.PRE_60_NODE_CHECKPOINT; } + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + localCheckpointOfSafeCommit = in.readZLong(); + } else { + localCheckpointOfSafeCommit = SequenceNumbers.UNASSIGNED_SEQ_NO; + } } @Override @@ -1124,6 +1126,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_6_0_0_rc1)) { out.writeZLong(globalCheckpoint); } + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeZLong(localCheckpointOfSafeCommit); + } } @Override @@ -1136,18 +1141,24 @@ public long globalCheckpoint() { return globalCheckpoint; } + @Override + public long localCheckpointOfSafeCommit() { + return localCheckpointOfSafeCommit; + } + @Override public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; ReplicaResponse that = (ReplicaResponse) o; return localCheckpoint == that.localCheckpoint && - globalCheckpoint == that.globalCheckpoint; + globalCheckpoint == that.globalCheckpoint && + localCheckpointOfSafeCommit == that.localCheckpointOfSafeCommit; } @Override public int hashCode() { - return Objects.hash(localCheckpoint, globalCheckpoint); + return Objects.hash(localCheckpoint, globalCheckpoint, localCheckpointOfSafeCommit); } } @@ -1228,12 +1239,6 @@ public void onFailure(Exception shardFailedError) { } } - protected ReplicaResponse readReplicaResponse(StreamInput in) throws IOException { - ReplicaResponse replicaResponse = new ReplicaResponse(); - replicaResponse.readFrom(in); - return replicaResponse; - } - /** * Sends the specified replica request to the specified node. * @@ -1245,8 +1250,12 @@ protected void sendReplicaRequest( final ConcreteReplicaRequest replicaRequest, final DiscoveryNode node, final ActionListener listener) { - transportService.sendRequest(node, transportReplicaAction, replicaRequest, transportOptions, - new ActionListenerResponseHandler<>(listener, this::readReplicaResponse)); + final ActionListenerResponseHandler handler = new ActionListenerResponseHandler<>(listener, in -> { + ReplicaResponse replicaResponse = new ReplicaResponse(); + replicaResponse.readFrom(in); + return replicaResponse; + }); + transportService.sendRequest(node, transportReplicaAction, replicaRequest, transportOptions, handler); } /** a wrapper class to encapsulate a request when being sent to a specific allocation id **/ diff --git a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java index 9b55cff8cff9a..070924de14585 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointSyncAction.java @@ -112,7 +112,7 @@ protected void sendReplicaRequest( super.sendReplicaRequest(replicaRequest, node, listener); } else { final long pre60NodeCheckpoint = SequenceNumbers.PRE_60_NODE_CHECKPOINT; - listener.onResponse(new ReplicaResponse(pre60NodeCheckpoint, pre60NodeCheckpoint)); + listener.onResponse(new ReplicaResponse(pre60NodeCheckpoint, pre60NodeCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO)); } } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java index 791a134a63aa9..7a81817dea502 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java @@ -77,60 +77,15 @@ protected PrimaryResult shardOperationOnPrimary(Re return new PrimaryResult<>(shardRequest, new ReplicationResponse()); } - @Override - protected ReplicaResponse readReplicaResponse(StreamInput in) throws IOException { - return new ShardCopyResponse(in); - } - @Override protected ReplicaResult shardOperationOnReplica(Request shardRequest, IndexShard replica) { - return new ReplicaResult() { - @Override - public ReplicaResponse getReplicaResponse(IndexShard replica) { - return new ShardCopyResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint(), - replica.getLocalCheckpointOfSafeCommit()); - } - }; - } - - @Override - protected void handleReplicaResponse(ShardRouting shard, ReplicationOperation.ReplicaResponse response) { - assert response instanceof ShardCopyResponse : response.getClass(); - final ShardCopyResponse shardCopyResponse = (ShardCopyResponse) response; // TODO introduce type parameter rather than cast here - indicesService.indexServiceSafe(shard.index()).getShard(shard.id()) - .renewPeerRecoveryRetentionLeaseForReplica(shard, shardCopyResponse.localCheckpointOfSafeCommit); + return new ReplicaResult(); } public void renewPeerRecoveryRetentionLease(ShardId shardId, ActionListener listener) { execute(new Request(shardId), ActionListener.wrap(v -> listener.onResponse(null), listener::onFailure)); } - static final class ShardCopyResponse extends ReplicaResponse { - private long localCheckpointOfSafeCommit; - - ShardCopyResponse(long localCheckpoint, long globalCheckpoint, long localCheckpointOfSafeCommit) { - super(localCheckpoint, globalCheckpoint); - this.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; - } - - ShardCopyResponse(StreamInput in) throws IOException { - super(); - super.readFrom(in); - localCheckpointOfSafeCommit = in.readLong(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - out.writeLong(localCheckpointOfSafeCommit); - } - - @Override - public void readFrom(StreamInput in) { - throw new UnsupportedOperationException("use Writable not Streamable"); - } - } - static final class Request extends ReplicationRequest { Request() { } @@ -141,7 +96,7 @@ static final class Request extends ReplicationRequest { @Override public String toString() { - return "request for minimum seqno needed for peer recovery for " + shardId; + return "request for update of local checkpoint of safe commit for " + shardId; } } } 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 a13d3bd778546..9b4af2c5c08ab 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1889,6 +1889,18 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long replicationTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); } + /** + * Update the local knowledge of the local checkpoint of the safe commit for the specified allocation ID. + * + * @param allocationId the allocation ID to update the local checkpoint of the safe commit for + * @param localCheckpointOfSafeCommit the local checkpoint of the safe commit + */ + public void updateLocalCheckpointOfSafeCommitForShard(final String allocationId, final long localCheckpointOfSafeCommit) { + assert assertPrimaryMode(); + verifyNotClosed(); + replicationTracker.updateGlobalCheckpointForShard(allocationId, localCheckpointOfSafeCommit); + } + /** * Add a global checkpoint listener. If the global checkpoint is equal to or above the global checkpoint the listener is waiting for, * then the listener will be notified immediately via an executor (so possibly not on the current thread). If the specified timeout diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 687b01680704e..81ac1c8d791b5 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -238,7 +238,7 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { String allocationId = ((ConcreteShardRequest) capturedRequest.request).getTargetAllocationID(); assertFalse(unavailableShards.stream().anyMatch(shardRouting -> shardRouting.allocationId().getId().equals(allocationId))); assertTrue(inSyncAllocationIds.stream().anyMatch(inSyncAllocationId -> inSyncAllocationId.equals(allocationId))); - transport.handleResponse(capturedRequest.requestId, new TransportReplicationAction.ReplicaResponse(0L, 0L)); + transport.handleResponse(capturedRequest.requestId, new TransportReplicationAction.ReplicaResponse(0L, 0L, 0L)); } else { fail("Test does not support action " + capturedRequest.action); @@ -288,6 +288,10 @@ public void updateLocalCheckpointForShard(String allocationId, long checkpoint) public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { } + @Override + public void updateLocalCheckpointOfSafeCommitForShard(String allocationId, long localCheckpointOfSafeCommit) { + } + @Override public long localCheckpoint() { return 0; diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index 8fa10c4ee26d7..a6fca89c996f8 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -447,6 +447,7 @@ static class TestPrimary implements ReplicationOperation.Primary replicationGroupSupplier; final Map knownLocalCheckpoints = new HashMap<>(); final Map knownGlobalCheckpoints = new HashMap<>(); + final Map knownLocalCheckpointsOfSafeCommits = new HashMap<>(); // TODO assert things about this TestPrimary(ShardRouting routing, Supplier replicationGroupSupplier) { this.routing = routing; @@ -507,6 +508,11 @@ public void updateGlobalCheckpointForShard(String allocationId, long globalCheck knownGlobalCheckpoints.put(allocationId, globalCheckpoint); } + @Override + public void updateLocalCheckpointOfSafeCommitForShard(String allocationId, long localCheckpointOfSafeCommit) { + knownLocalCheckpointsOfSafeCommits.put(allocationId, localCheckpointOfSafeCommit); + } + @Override public long localCheckpoint() { return localCheckpoint; @@ -532,10 +538,12 @@ public ReplicationGroup getReplicationGroup() { static class ReplicaResponse implements ReplicationOperation.ReplicaResponse { final long localCheckpoint; final long globalCheckpoint; + final long localCheckpointOfSafeCommit; - ReplicaResponse(long localCheckpoint, long globalCheckpoint) { + ReplicaResponse(long localCheckpoint, long globalCheckpoint, long localCheckpointOfSafeCommit) { this.localCheckpoint = localCheckpoint; this.globalCheckpoint = globalCheckpoint; + this.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; } @Override @@ -548,6 +556,11 @@ public long globalCheckpoint() { return globalCheckpoint; } + @Override + public long localCheckpointOfSafeCommit() { + return localCheckpointOfSafeCommit; + } + } static class TestReplicaProxy implements ReplicationOperation.Replicas { @@ -560,6 +573,8 @@ static class TestReplicaProxy implements ReplicationOperation.Replicas final Map generatedGlobalCheckpoints = ConcurrentCollections.newConcurrentMap(); + final Map generatedLocalCheckpointsOfSafeCommits = ConcurrentCollections.newConcurrentMap(); + final Set markedAsStaleCopies = ConcurrentCollections.newConcurrentSet(); final long primaryTerm; @@ -586,10 +601,13 @@ public void performOn( } else { final long generatedLocalCheckpoint = random().nextLong(); final long generatedGlobalCheckpoint = random().nextLong(); + final long generatedLocalCheckpointOfSafeCommit = random().nextLong(); final String allocationId = replica.allocationId().getId(); assertNull(generatedLocalCheckpoints.put(allocationId, generatedLocalCheckpoint)); assertNull(generatedGlobalCheckpoints.put(allocationId, generatedGlobalCheckpoint)); - listener.onResponse(new ReplicaResponse(generatedLocalCheckpoint, generatedGlobalCheckpoint)); + assertNull(generatedLocalCheckpointsOfSafeCommits.put(allocationId, generatedLocalCheckpointOfSafeCommit)); + listener.onResponse( + new ReplicaResponse(generatedLocalCheckpoint, generatedGlobalCheckpoint, generatedLocalCheckpointOfSafeCommit)); } } diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 110ab9bcb99a2..3cfa625aee277 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -729,7 +729,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { assertThat(captures, arrayWithSize(1)); if (randomBoolean()) { final TransportReplicationAction.ReplicaResponse response = - new TransportReplicationAction.ReplicaResponse(randomLong(), randomLong()); + new TransportReplicationAction.ReplicaResponse(randomLong(), randomLong(), randomLong()); transport.handleResponse(captures[0].requestId, response); assertTrue(listener.isDone()); assertThat(listener.get(), equalTo(response)); diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java index 6e1ec3c76797d..393b38fc6cac2 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -296,7 +296,7 @@ public void testReplicaProxy() throws InterruptedException, ExecutionException { assertThat(captures, arrayWithSize(1)); if (randomBoolean()) { final TransportReplicationAction.ReplicaResponse response = - new TransportReplicationAction.ReplicaResponse(randomLong(), randomLong()); + new TransportReplicationAction.ReplicaResponse(randomLong(), randomLong(), randomLong()); transport.handleResponse(captures[0].requestId, response); assertTrue(listener.isDone()); assertThat(listener.get(), equalTo(response)); 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 1c41f719673e5..29baacf734760 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 @@ -664,6 +664,11 @@ public void updateGlobalCheckpointForShard(String allocationId, long globalCheck getPrimaryShard().updateGlobalCheckpointForShard(allocationId, globalCheckpoint); } + @Override + public void updateLocalCheckpointOfSafeCommitForShard(String allocationId, long localCheckpointOfSafeCommit) { + getPrimaryShard().updateLocalCheckpointOfSafeCommitForShard(allocationId, localCheckpointOfSafeCommit); + } + @Override public long localCheckpoint() { return getPrimaryShard().getLocalCheckpoint(); @@ -706,7 +711,8 @@ public void onResponse(Releasable releasable) { try { performOnReplica(request, replica); releasable.close(); - listener.onResponse(new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint())); + listener.onResponse(new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint(), + replica.getLocalCheckpointOfSafeCommit())); } catch (final Exception e) { Releasables.closeWhileHandlingException(releasable); listener.onFailure(e); From 2395b7a480abf3e458e071e2f1b04ce0b854beb4 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 27 Feb 2019 12:58:06 +0000 Subject: [PATCH 24/32] WIP continues --- .../index/seqno/ReplicationTracker.java | 58 +++++++++++++++++-- .../elasticsearch/index/shard/IndexShard.java | 2 +- 2 files changed, 53 insertions(+), 7 deletions(-) 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 b95621f0eaed9..d4876214a2d6a 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -431,6 +431,12 @@ public static class CheckpointState implements Writeable { * the tracker is in primary mode and received from the primary if in replica mode. */ long globalCheckpoint; + + /** + * The last local-checkpoint-of-safe-commit that we have for this shard + */ + long localCheckpointOfSafeCommit; + /** * whether this shard is treated as in-sync and thus contributes to the global checkpoint calculation */ @@ -441,9 +447,11 @@ public static class CheckpointState implements Writeable { */ boolean tracked; - public CheckpointState(long localCheckpoint, long globalCheckpoint, boolean inSync, boolean tracked) { + public CheckpointState(long localCheckpoint, long globalCheckpoint, long localCheckpointOfSafeCommit, + boolean inSync, boolean tracked) { this.localCheckpoint = localCheckpoint; this.globalCheckpoint = globalCheckpoint; + this.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; this.inSync = inSync; this.tracked = tracked; } @@ -451,6 +459,11 @@ public CheckpointState(long localCheckpoint, long globalCheckpoint, boolean inSy public CheckpointState(StreamInput in) throws IOException { this.localCheckpoint = in.readZLong(); this.globalCheckpoint = in.readZLong(); + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + this.localCheckpointOfSafeCommit = in.readZLong(); + } else { + this.localCheckpointOfSafeCommit = SequenceNumbers.UNASSIGNED_SEQ_NO; + } this.inSync = in.readBoolean(); if (in.getVersion().onOrAfter(Version.V_6_3_0)) { this.tracked = in.readBoolean(); @@ -468,6 +481,9 @@ public CheckpointState(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { out.writeZLong(localCheckpoint); out.writeZLong(globalCheckpoint); + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + out.writeZLong(this.localCheckpointOfSafeCommit); + } out.writeBoolean(inSync); if (out.getVersion().onOrAfter(Version.V_6_3_0)) { out.writeBoolean(tracked); @@ -478,7 +494,7 @@ public void writeTo(StreamOutput out) throws IOException { * Returns a full copy of this object */ public CheckpointState copy() { - return new CheckpointState(localCheckpoint, globalCheckpoint, inSync, tracked); + return new CheckpointState(localCheckpoint, globalCheckpoint, localCheckpointOfSafeCommit, inSync, tracked); } public long getLocalCheckpoint() { @@ -489,11 +505,16 @@ public long getGlobalCheckpoint() { return globalCheckpoint; } + public long getLocalCheckpointOfSafeCommit() { + return localCheckpointOfSafeCommit; + } + @Override public String toString() { return "LocalCheckpointState{" + "localCheckpoint=" + localCheckpoint + ", globalCheckpoint=" + globalCheckpoint + + ", localCheckpointOfSafeCommit=" + localCheckpointOfSafeCommit + ", inSync=" + inSync + ", tracked=" + tracked + '}'; @@ -508,6 +529,7 @@ public boolean equals(Object o) { if (localCheckpoint != that.localCheckpoint) return false; if (globalCheckpoint != that.globalCheckpoint) return false; + if (localCheckpointOfSafeCommit != that.localCheckpointOfSafeCommit) return false; if (inSync != that.inSync) return false; return tracked == that.tracked; } @@ -516,6 +538,7 @@ public boolean equals(Object o) { public int hashCode() { int result = Long.hashCode(localCheckpoint); result = 31 * result + Long.hashCode(globalCheckpoint); + result = 31 * result + Long.hashCode(localCheckpointOfSafeCommit); result = 31 * result + Boolean.hashCode(inSync); result = 31 * result + Boolean.hashCode(tracked); return result; @@ -719,7 +742,8 @@ public ReplicationTracker( this.handoffInProgress = false; this.appliedClusterStateVersion = -1L; this.checkpoints = new HashMap<>(1 + indexSettings.getNumberOfReplicas()); - checkpoints.put(allocationId, new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, false, false)); + checkpoints.put(allocationId, + new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO, false, false)); this.onGlobalCheckpointUpdated = Objects.requireNonNull(onGlobalCheckpointUpdated); this.currentTimeMillisSupplier = Objects.requireNonNull(currentTimeMillisSupplier); this.onSyncRetentionLeases = Objects.requireNonNull(onSyncRetentionLeases); @@ -816,6 +840,25 @@ private void updateGlobalCheckpoint(final String allocationId, final long global } } + public synchronized void updateLocalCheckpointOfSafeCommitForShard(String allocationId, long localCheckpointOfSafeCommit) { + assert primaryMode; + assert handoffInProgress == false; + assert invariant(); + + final CheckpointState cps = checkpoints.get(allocationId); + assert !this.shardAllocationId.equals(allocationId) || cps != null; + if (cps != null && localCheckpointOfSafeCommit > cps.localCheckpointOfSafeCommit) { + long previousValue = cps.localCheckpointOfSafeCommit; + cps.localCheckpointOfSafeCommit = localCheckpointOfSafeCommit; + logger.trace( + "updated local knowledge for [{}] on the primary of the local checkpoint of the safe commit from [{}] to [{}]", + allocationId, + previousValue, + localCheckpointOfSafeCommit); + } + assert invariant(); + } + /** * Initializes the global checkpoint tracker in primary mode (see {@link #primaryMode}. Called on primary activation or promotion. */ @@ -880,7 +923,8 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion final long localCheckpoint = pre60AllocationIds.contains(initializingId) ? SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, inSync, inSync)); + checkpoints.put(initializingId, + new CheckpointState(localCheckpoint, globalCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO, inSync, inSync)); } } if (removedEntries) { @@ -892,7 +936,8 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion final long localCheckpoint = pre60AllocationIds.contains(initializingId) ? SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false)); + checkpoints.put(initializingId, + new CheckpointState(localCheckpoint, globalCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO, false, false)); } } for (String inSyncId : inSyncAllocationIds) { @@ -905,7 +950,8 @@ public synchronized void updateFromMaster(final long applyingClusterStateVersion final long localCheckpoint = pre60AllocationIds.contains(inSyncId) ? SequenceNumbers.PRE_60_NODE_CHECKPOINT : SequenceNumbers.UNASSIGNED_SEQ_NO; final long globalCheckpoint = localCheckpoint; - checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true)); + checkpoints.put(inSyncId, + new CheckpointState(localCheckpoint, globalCheckpoint, SequenceNumbers.UNASSIGNED_SEQ_NO, true, true)); } } } 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 9b4af2c5c08ab..c40827a96c459 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1898,7 +1898,7 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long public void updateLocalCheckpointOfSafeCommitForShard(final String allocationId, final long localCheckpointOfSafeCommit) { assert assertPrimaryMode(); verifyNotClosed(); - replicationTracker.updateGlobalCheckpointForShard(allocationId, localCheckpointOfSafeCommit); + replicationTracker.updateLocalCheckpointOfSafeCommitForShard(allocationId, localCheckpointOfSafeCommit); } /** From fa6ae129fc8d8325b32b1d95c5bdce383addec59 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 27 Feb 2019 13:33:43 +0000 Subject: [PATCH 25/32] More WIP --- .../support/replication/ReplicationOperation.java | 8 ++++++++ .../replication/TransportReplicationAction.java | 5 +++++ .../index/seqno/ReplicationTracker.java | 15 +++++++++++++++ .../org/elasticsearch/index/shard/IndexShard.java | 5 ++++- ...ransportVerifyShardBeforeCloseActionTests.java | 5 +++++ .../replication/ReplicationOperationTests.java | 7 +++++++ .../indices/recovery/IndexRecoveryIT.java | 2 +- .../ESIndexLevelReplicationTestCase.java | 5 +++++ 8 files changed, 50 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index ede1726129063..344ea7f321331 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -101,6 +101,7 @@ public void execute() throws Exception { pendingActions.incrementAndGet(); // increase by 1 until we finish all primary coordination primaryResult = primary.perform(request); primary.updateLocalCheckpointForShard(primaryRouting.allocationId().getId(), primary.localCheckpoint()); + primary.updateLocalCheckpointOfSafeCommitForShard(primaryRouting.allocationId().getId(), primary.localCheckpointOfSafeCommit()); final ReplicaRequest replicaRequest = primaryResult.replicaRequest(); if (replicaRequest != null) { if (logger.isTraceEnabled()) { @@ -342,6 +343,13 @@ public interface Primary< */ long globalCheckpoint(); + /** + * Returns the local checkpoint of the safe commit on the primary shard. + * + * @return the local checkpoint of the safe commit + */ + long localCheckpointOfSafeCommit(); + /** * Returns the maximum seq_no of updates (index operations overwrite Lucene) or deletes on the primary. * This value must be captured after the execution of a replication request on the primary is completed. diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 1532b19986695..6296e4220902f 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -1063,6 +1063,11 @@ public long globalCheckpoint() { return indexShard.getGlobalCheckpoint(); } + @Override + public long localCheckpointOfSafeCommit() { + return indexShard.getLocalCheckpointOfSafeCommit(); + } + @Override public long maxSeqNoOfUpdatesOrDeletes() { return indexShard.getMaxSeqNoOfUpdatesOrDeletes(); 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 d4876214a2d6a..4663d009f480f 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -419,6 +419,21 @@ public void persistRetentionLeases(final Path path) throws WriteStateException { } } + public synchronized void updatePeerRecoveryRetentionLeasesFromCheckpointState() { + assert primaryMode; + + for (final Map.Entry entry : checkpoints.entrySet()) { + final ShardRouting shardRouting = routingTable.getByAllocationId(entry.getKey()); + final CheckpointState cps = entry.getValue(); + if (cps.tracked) { + renewRetentionLease( + getPeerRecoveryRetentionLeaseId(shardRouting), + cps.localCheckpointOfSafeCommit + 1, + PEER_RECOVERY_RETENTION_LEASE_SOURCE); + } + } + } + public static class CheckpointState implements Writeable { /** 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 c40827a96c459..478f9d6fa47da 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2475,7 +2475,10 @@ public Future renewPeerRecoveryRetentionLeases() { assert assertPrimaryMode(); final PlainActionFuture plainActionFuture = new PlainActionFuture<>(); if (replicationTracker.peerRetentionLeasesNeedRenewal(getLocalCheckpointOfSafeCommit())) { - peerRecoveryRetentionLeaseRenewer.accept(plainActionFuture); + peerRecoveryRetentionLeaseRenewer.accept(ActionListener.wrap(v -> { + replicationTracker.updatePeerRecoveryRetentionLeasesFromCheckpointState(); + plainActionFuture.onResponse(null); + }, plainActionFuture::onFailure)); } else { plainActionFuture.onResponse(null); } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 81ac1c8d791b5..0463a38c73a70 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -302,6 +302,11 @@ public long globalCheckpoint() { return 0; } + @Override + public long localCheckpointOfSafeCommit() { + return 0; + } + @Override public long maxSeqNoOfUpdatesOrDeletes() { return 0; diff --git a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index a6fca89c996f8..157a974964c23 100644 --- a/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/server/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -443,6 +443,7 @@ static class TestPrimary implements ReplicationOperation.Primary replicationGroupSupplier; final Map knownLocalCheckpoints = new HashMap<>(); @@ -454,6 +455,7 @@ static class TestPrimary implements ReplicationOperation.Primary Date: Wed, 27 Feb 2019 15:29:07 +0000 Subject: [PATCH 26/32] Imports --- .../seqno/PeerRecoveryRetentionLeaseRenewalAction.java | 6 ------ 1 file changed, 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java index 7a81817dea502..69025f591dc5f 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/PeerRecoveryRetentionLeaseRenewalAction.java @@ -20,17 +20,13 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ActionFilters; -import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.seqno.PeerRecoveryRetentionLeaseRenewalAction.Request; import org.elasticsearch.index.shard.IndexShard; @@ -40,8 +36,6 @@ import org.elasticsearch.threadpool.ThreadPool.Names; import org.elasticsearch.transport.TransportService; -import java.io.IOException; - /** * Background action to renew retention leases held to ensure that enough history is retained to perform a peer recovery if needed. This * action renews the leases for each copy of the shard, advancing the corresponding sequence number, and thereby releases any operations From ed18d5c2cf484e0c6ed7a3df496251947565d9c7 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 1 Mar 2019 10:38:15 +0000 Subject: [PATCH 27/32] Weaken retention leases assertion so it only applies in supported versions --- .../index/seqno/ReplicationTracker.java | 29 ++++++++++------- .../elasticsearch/index/shard/IndexShard.java | 3 +- ...ReplicationTrackerRetentionLeaseTests.java | 32 ++++++++++++------- .../seqno/ReplicationTrackerTestCase.java | 4 ++- .../index/seqno/ReplicationTrackerTests.java | 5 +-- .../index/engine/EngineTestCase.java | 3 +- 6 files changed, 49 insertions(+), 27 deletions(-) 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 4663d009f480f..a61c0f1f02d81 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -183,6 +183,11 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L */ private RetentionLeases retentionLeases = RetentionLeases.EMPTY; + /** + * The version in which this index was created + */ + private final Version indexCreatedVersion; + /** * Get all retention leases tracked on this shard. * @@ -698,16 +703,15 @@ private boolean invariant() { assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked"; } - if (checkpoints.get(shardAllocationId).inSync - && (primaryMode || shardAllocationId.equals(routingTable.primaryShard().allocationId().getId()) == false)) { - // a newly-recovered primary creates its own retention lease when entering primaryMode, which is done later, so it doesn't - // exist yet - - // TODO also expect to have a lease for tracked shard copies - // TODO what about relocating shards? - for (ShardRouting shardRouting : routingTable.activeShards()) { - assert retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : - "no retention lease for active shard " + shardRouting + " in " + retentionLeases + " on " + shardAllocationId; + if (primaryMode && indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 after backporting + for (final ShardRouting shardRouting : routingTable.assignedShards()) { + assert checkpoints.get(shardRouting.allocationId().getId()).tracked == false + || retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : + "no retention lease for tracked shard " + shardRouting + " in " + retentionLeases; + assert shardRouting.relocating() == false + || checkpoints.get(shardRouting.allocationId().getRelocationId()).tracked == false + || retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting.getTargetRelocatingShard())) : + "no retention lease for relocation target " + shardRouting + " in " + retentionLeases; } } @@ -748,7 +752,8 @@ public ReplicationTracker( final long globalCheckpoint, final LongConsumer onGlobalCheckpointUpdated, final LongSupplier currentTimeMillisSupplier, - final BiConsumer> onSyncRetentionLeases) { + final BiConsumer> onSyncRetentionLeases, + final Version indexCreatedVersion) { super(shardId, indexSettings); assert globalCheckpoint >= SequenceNumbers.UNASSIGNED_SEQ_NO : "illegal initial global checkpoint: " + globalCheckpoint; this.shardAllocationId = allocationId; @@ -765,6 +770,8 @@ public ReplicationTracker( this.pendingInSync = new HashSet<>(); this.routingTable = null; this.replicationGroup = null; + assert Version.V_EMPTY.equals(indexCreatedVersion) == false; + this.indexCreatedVersion = indexCreatedVersion; assert invariant(); } 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 478f9d6fa47da..99e5b99504ffc 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -334,7 +334,8 @@ public IndexShard( UNASSIGNED_SEQ_NO, globalCheckpointListeners::globalCheckpointUpdated, threadPool::absoluteTimeInMillis, - (retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, retentionLeases, listener)); + (retentionLeases, listener) -> retentionLeaseSyncer.sync(shardId, retentionLeases, listener), + IndexMetaData.SETTING_INDEX_VERSION_CREATED.get(indexSettings.getSettings())); this.replicationTracker = replicationTracker; this.peerRecoveryRetentionLeaseRenewer = peerRecoveryRetentionLeaseRenewer; 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 0718a40e1b808..02702262d2279 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; @@ -69,7 +70,8 @@ public void testAddOrRenewRetentionLease() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -111,7 +113,8 @@ public void testAddDuplicateRetentionLease() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -140,7 +143,8 @@ public void testRenewNotFoundRetentionLease() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -176,7 +180,7 @@ public void testAddRetentionLeaseCausesRetentionLeaseSync() { .stream() .collect(Collectors.toMap(RetentionLease::id, RetentionLease::retainingSequenceNumber)), equalTo(retainingSequenceNumbers)); - }); + }, Version.CURRENT); reference.set(replicationTracker); final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( @@ -214,7 +218,8 @@ public void testRemoveRetentionLease() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -265,7 +270,8 @@ public void testRemoveNotFound() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -301,7 +307,7 @@ public void testRemoveRetentionLeaseCausesRetentionLeaseSync() { .stream() .collect(Collectors.toMap(RetentionLease::id, RetentionLease::retainingSequenceNumber)), equalTo(retainingSequenceNumbers)); - }); + }, Version.CURRENT); reference.set(replicationTracker); final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( @@ -356,7 +362,8 @@ private void runExpirationTest(final boolean primaryMode) { UNASSIGNED_SEQ_NO, value -> {}, currentTimeMillis::get, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -431,7 +438,8 @@ public void testReplicaIgnoresOlderRetentionLeasesVersion() { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -485,7 +493,8 @@ public void testLoadAndPersistRetentionLeases() throws IOException { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), @@ -525,7 +534,8 @@ public void testPersistRetentionLeasesUnderConcurrency() throws IOException { UNASSIGNED_SEQ_NO, value -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), 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..0b98fe79815d6 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; @@ -49,7 +50,8 @@ ReplicationTracker newTracker( UNASSIGNED_SEQ_NO, updatedGlobalCheckpoint, currentTimeMillisSupplier, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); } static IndexShardRoutingTable routingTable(final Set initializingIds, final AllocationId primaryId) { 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 885781a301679..253970496bd01 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.routing.AllocationId; @@ -697,7 +698,7 @@ public void testPrimaryContextHandoff() throws IOException { (leases, listener) -> {}; ReplicationTracker oldPrimary = new ReplicationTracker( shardId, clusterState.routingTable.primaryShard().allocationId().getId(), - indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); + indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); clusterState.apply(oldPrimary); oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10), NO_OPS_PERFORMED); @@ -706,7 +707,7 @@ public void testPrimaryContextHandoff() throws IOException { ReplicationTracker newPrimary = new ReplicationTracker( shardId, clusterState.routingTable.primaryShard().allocationId().getRelocationId(), - indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease); + indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); oldPrimary.addPeerRecoveryRetentionLease(clusterState.routingTable.primaryShard().relocatingNodeId(), 0L, wrap(() -> {})); newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); 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 e7b3f39747124..3a48d99b89f42 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 @@ -663,7 +663,8 @@ public EngineConfig config( SequenceNumbers.NO_OPS_PERFORMED, update -> {}, () -> 0L, - (leases, listener) -> {}); + (leases, listener) -> {}, + Version.CURRENT); globalCheckpointSupplier = replicationTracker; retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { From 6f8f48bb077a5abf4872e4b5638e72a25443b961 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 1 Mar 2019 13:28:20 +0000 Subject: [PATCH 28/32] Create missing leases In a rolling upgrade from a version that pre-dates the introduction of peer-recovery retention leases, eventually the primary lands on an upgraded node. If this was a relocation or a promotion then it inherits the replicas of the previous primary without performing peer recoveries, so there may not be peer recovery retention leases for the other shard copies. This change fixes this by adding leases as-needed on older indices. --- .../TransportReplicationAction.java | 4 +- .../index/seqno/ReplicationTracker.java | 68 ++++++++++++++++--- .../elasticsearch/index/shard/IndexShard.java | 49 +++++++++++++ 3 files changed, 110 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 6296e4220902f..85c0db5539609 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -1115,7 +1115,7 @@ public void readFrom(StreamInput in) throws IOException { } else { globalCheckpoint = SequenceNumbers.PRE_60_NODE_CHECKPOINT; } - if (in.getVersion().onOrAfter(Version.V_8_0_0)) { + if (in.getVersion().onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 for backport localCheckpointOfSafeCommit = in.readZLong(); } else { localCheckpointOfSafeCommit = SequenceNumbers.UNASSIGNED_SEQ_NO; @@ -1131,7 +1131,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_6_0_0_rc1)) { out.writeZLong(globalCheckpoint); } - if (out.getVersion().onOrAfter(Version.V_8_0_0)) { + if (out.getVersion().onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 for backport out.writeZLong(localCheckpointOfSafeCommit); } } 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 a61c0f1f02d81..bfd2935b522ab 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -387,7 +387,19 @@ public synchronized boolean peerRetentionLeasesNeedRenewal(long localCheckpointO assert primaryMode; final RetentionLeases retentionLeases = getRetentionLeases(); return routingTable.activeShards().stream().anyMatch( - sr -> retentionLeases.get(getPeerRecoveryRetentionLeaseId(sr)).retainingSequenceNumber() < localCheckpointOfSafeCommit + 1); + shardRouting -> { + final RetentionLease retentionLease = retentionLeases.get(getPeerRecoveryRetentionLeaseId(shardRouting)); + if (retentionLease == null) { + /* + * We got here via a rolling upgrade from an older version that doesn't create peer recovery retention leases for every + * shard copy. These missing leases are created lazily if they're found to be missing during a + * TransportReplicationAction, such as the peer recovery retention lease sync, so let's trigger a sync. + */ + assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_0_0 in backport + return true; + } + return retentionLease.retainingSequenceNumber() < localCheckpointOfSafeCommit + 1; + }); } /** @@ -439,6 +451,37 @@ public synchronized void updatePeerRecoveryRetentionLeasesFromCheckpointState() } } + public void addMissingPeerRecoveryRetentionLease(String allocationId, long startingSeqNo) { + final RetentionLeases updatedLeases; + synchronized (this) { + final ShardRouting replicaShardRouting = routingTable.getByAllocationId(allocationId); + if (replicaShardRouting == null) { + return; + } + + if (retentionLeases.get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replicaShardRouting)) != null) { + return; + } + + /* + * We got here via a rolling upgrade from an older version that doesn't create peer recovery retention leases for every shard + * copy. But the replica we're dealing with now has been upgraded and is retaining history while we asynchronously make it a + * retention lease. + */ + assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_0_0 in backport + try { + innerAddRetentionLease(getPeerRecoveryRetentionLeaseId(replicaShardRouting.currentNodeId()), startingSeqNo, + PEER_RECOVERY_RETENTION_LEASE_SOURCE); + updatedLeases = retentionLeases; + } catch (RetentionLeaseAlreadyExistsException e) { + assert false : e; + logger.debug("BWC peer recovery retention lease created concurrently", e); + return; + } + } + onSyncRetentionLeases.accept(updatedLeases, ActionListener.wrap(() -> {})); + } + public static class CheckpointState implements Writeable { /** @@ -899,14 +942,21 @@ public synchronized void activatePrimaryMode(final long localCheckpoint, final l final ShardRouting primaryShard = routingTable.primaryShard(); final String leaseId = getPeerRecoveryRetentionLeaseId(primaryShard); if (retentionLeases.get(leaseId) == null) { - // We are starting up the whole replication group from scratch: if we were not (i.e. this is a replica promotion) then - // this copy must already be in-sync and active and therefore holds a retention lease for itself. - assert routingTable.activeShards().equals(singletonList(primaryShard)) : routingTable.activeShards(); - assert primaryShard.allocationId().getId().equals(shardAllocationId) : routingTable.activeShards() + " vs " + shardAllocationId; - assert replicationGroup.getReplicationTargets().equals(singletonList(primaryShard)); - - // Safe to call innerAddRetentionLease() without a subsequent sync because there are no other members of this replication gp. - innerAddRetentionLease(leaseId, max(0L, localCheckpointOfSafeCommit + 1), PEER_RECOVERY_RETENTION_LEASE_SOURCE); + /* + * We might have got here here via a rolling upgrade from an older version that doesn't create peer recovery retention leases + * for every shard copy. The missing leases are created in a more relaxed fashion, and offer weaker guarantees. + */ + if (indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 in backport + // We are starting up the whole replication group from scratch: if we were not (i.e. this is a replica promotion) then + // this copy must already be in-sync and active and therefore holds a retention lease for itself. + assert routingTable.activeShards().equals(singletonList(primaryShard)) : routingTable.activeShards(); + assert primaryShard.allocationId().getId().equals(shardAllocationId) + : routingTable.activeShards() + " vs " + shardAllocationId; + assert replicationGroup.getReplicationTargets().equals(singletonList(primaryShard)); + + // Safe to call innerAddRetentionLease() without a subsequent sync since there are no other members of this replication gp. + innerAddRetentionLease(leaseId, max(0L, localCheckpointOfSafeCommit + 1), PEER_RECOVERY_RETENTION_LEASE_SOURCE); + } } assert invariant(); 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 99e5b99504ffc..cc16a6213a0e9 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -488,6 +488,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(), getLocalCheckpointOfSafeCommit()); + asyncEnsurePeerRecoveryRetentionLeaseForPrimary(); } } else { assert currentRouting.primary() == false : "term is only increased as part of primary promotion"; @@ -530,6 +531,7 @@ public void updateShardState(final ShardRouting newRouting, assert getOperationPrimaryTerm() == newPrimaryTerm; try { replicationTracker.activatePrimaryMode(getLocalCheckpoint(), getLocalCheckpointOfSafeCommit()); + asyncEnsurePeerRecoveryRetentionLeaseForPrimary(); /* * 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 @@ -601,6 +603,28 @@ public void onFailure(Exception e) { } } + private void asyncEnsurePeerRecoveryRetentionLeaseForPrimary() { + if (replicationTracker.getRetentionLeases() + .get(ReplicationTracker.getPeerRecoveryRetentionLeaseId(routingEntry())) == null) { + /* + * We got here via a rolling upgrade from an older version that doesn't create peer recovery retention leases for every shard + * copy, so there isn't yet one for this primary. We offer relaxed guarantees in this case, so create one asynchronously once + * the shard is fully started. + */ + assertIndexCreatedBeforePeerRecoveryRetentionLeases(); + threadPool.generic().execute(() -> { + synchronized (mutex) { + // wait for the shard to be started + } + runUnderPrimaryPermit( + () -> replicationTracker.addPeerRecoveryRetentionLease(shardRouting.currentNodeId(), + getLocalCheckpointOfSafeCommit(), ActionListener.wrap(() -> { })), + e -> logger.debug("failed to lazily create peer recovery retention lease for primary", e), + Names.SAME, ""); + }); + } + } + /** * Marks the shard as recovering based on a recovery state, fails with exception is recovering is not allowed to be set. */ @@ -1900,6 +1924,19 @@ public void updateLocalCheckpointOfSafeCommitForShard(final String allocationId, assert assertPrimaryMode(); verifyNotClosed(); replicationTracker.updateLocalCheckpointOfSafeCommitForShard(allocationId, localCheckpointOfSafeCommit); + + if (indexCreatedBeforePeerRecoveryRetentionLeases()) { + /* + * We might have got here via a rolling upgrade from an older version that doesn't create peer recovery retention leases for + * every shard copy. We create them lazily in this case and offer relaxed guarantees about history retention. If the lease does + * not currently exist, create one using the given LCPoSC as a starting point for retention. This might be too low (other shards + * are no longer retaining this history) or too high (the replica in question already discarded too much history to satisfy the + * other leases) but this will eventually be resolved. + */ + replicationTracker.addMissingPeerRecoveryRetentionLease(allocationId, localCheckpointOfSafeCommit + 1); + } else { + assert localCheckpointOfSafeCommit != UNASSIGNED_SEQ_NO; + } } /** @@ -2227,6 +2264,7 @@ public void activateWithPrimaryContext(final ReplicationTracker.PrimaryContext p getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint(); synchronized (mutex) { replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex + asyncEnsurePeerRecoveryRetentionLeaseForPrimary(); if (getMaxSeqNoOfUpdatesOrDeletes() == UNASSIGNED_SEQ_NO) { // If the old primary was on an old version that did not replicate the msu, // we need to bootstrap it manually from its local history. @@ -2494,6 +2532,17 @@ public long getLocalCheckpointOfSafeCommit() { return engine.getLocalCheckpointOfSafeCommit(); } + private boolean assertIndexCreatedBeforePeerRecoveryRetentionLeases() { + assert indexCreatedBeforePeerRecoveryRetentionLeases() + : IndexMetaData.SETTING_INDEX_VERSION_CREATED.get(indexSettings.getSettings()); + return true; + } + + private boolean indexCreatedBeforePeerRecoveryRetentionLeases() { + return IndexMetaData.SETTING_INDEX_VERSION_CREATED.get(indexSettings.getSettings()).before(Version.V_8_0_0); + // TODO V_7_0_0 on backport + } + class ShardEventListener implements Engine.EventListener { private final CopyOnWriteArrayList> delegates = new CopyOnWriteArrayList<>(); From 7064a6c0ebff130d583292e08663d1d7e23e6853 Mon Sep 17 00:00:00 2001 From: David Turner Date: Fri, 1 Mar 2019 16:30:54 +0000 Subject: [PATCH 29/32] WIP fixing ReplicationTrackerTests --- .../index/seqno/ReplicationTrackerTests.java | 48 +++++++++++++++---- 1 file changed, 39 insertions(+), 9 deletions(-) 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 253970496bd01..cebadfe6d0ec6 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -127,6 +127,8 @@ public void testGlobalCheckpointUpdate() { final IndexShardRoutingTable routingTable = routingTable(initializing, primaryId); tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable, emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + initializing.forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {}))); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1)); initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1 + initializing.size())); @@ -152,8 +154,11 @@ public void testGlobalCheckpointUpdate() { Set newInitializing = new HashSet<>(initializing); newInitializing.add(extraId); + final IndexShardRoutingTable routingTable2 = routingTable(newInitializing, routingTable.primaryShard()); + newInitializing.forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable2.getByAllocationId(aId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {}))); tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), - routingTable(newInitializing, routingTable.primaryShard()), emptySet()); + routingTable2, emptySet()); tracker.initiateTracking(extraId.getId()); @@ -235,8 +240,12 @@ public void testMissingActiveIdsPreventAdvance() { assigned.putAll(initializing); AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); + final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + initializing.keySet() + .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {}))); randomSubsetOf(initializing.keySet()).forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); final AllocationId missingActiveID = randomFrom(active.keySet()); assigned @@ -262,8 +271,12 @@ public void testMissingInSyncIdsPreventAdvance() { AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); + final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + initializing.keySet() + .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {}))); randomSubsetOf(randomIntBetween(1, initializing.size() - 1), initializing.keySet()).forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); @@ -284,8 +297,12 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); final AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); + final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + initializing.keySet() + .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {}))); initializing.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); nonApproved.keySet().forEach(k -> expectThrows(IllegalStateException.class, () -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED))); @@ -366,6 +383,8 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { final IndexShardRoutingTable routingTable = routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId); tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), routingTable, emptySet()); tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); + tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(trackingAllocationId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {})); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -427,9 +446,12 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar final AllocationId inSyncAllocationId = AllocationId.newInitializing(); final AllocationId trackingAllocationId = AllocationId.newInitializing(); final ReplicationTracker tracker = newTracker(inSyncAllocationId); + final IndexShardRoutingTable routingTable = routingTable(singleton(trackingAllocationId), inSyncAllocationId); tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()), - routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet()); + routingTable, emptySet()); tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); + tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(trackingAllocationId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {})); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -569,10 +591,13 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { // using a different length than we have been using above ensures that we can not collide with a previous allocation ID final AllocationId newSyncingAllocationId = AllocationId.newInitializing(); newInitializingAllocationIds.add(newSyncingAllocationId); + routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); + tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(newSyncingAllocationId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {})); tracker.updateFromMaster( initialClusterStateVersion + 3, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, routingTable.primaryShard()), + routingTable, emptySet()); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { @@ -606,12 +631,14 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { /* * The new in-sync allocation ID is in the in-sync set now yet the master does not know this; the allocation ID should still be in * the in-sync set even if we receive a cluster state update that does not reflect this. - * */ + routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); + tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(newSyncingAllocationId.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {})); tracker.updateFromMaster( initialClusterStateVersion + 4, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, routingTable.primaryShard()), + routingTable, emptySet()); assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); @@ -637,12 +664,15 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1); final ReplicationTracker tracker = newTracker(active); + final IndexShardRoutingTable routingTable = routingTable(singleton(initializing), active); tracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(active.getId()), - routingTable(Collections.singleton(initializing), active), + routingTable, emptySet()); tracker.activatePrimaryMode(activeLocalCheckpoint, NO_OPS_PERFORMED); + tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(initializing.getId()).currentNodeId(), + 0, ActionListener.wrap(() -> {})); final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE); final Thread activeThread = new Thread(() -> { try { From 3e0146536f43491ce8edbf475031aeed96fc8d57 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 20 Mar 2019 17:42:21 +0000 Subject: [PATCH 30/32] Test fixes --- .../routing/IndexShardRoutingTable.java | 1 + .../org/elasticsearch/index/IndexService.java | 2 + .../index/seqno/ReplicationTracker.java | 20 ++- .../elasticsearch/index/shard/IndexShard.java | 7 +- ...ReplicationTrackerRetentionLeaseTests.java | 10 +- .../seqno/ReplicationTrackerTestCase.java | 11 +- .../index/seqno/ReplicationTrackerTests.java | 165 ++++++++---------- .../shard/IndexShardRetentionLeaseTests.java | 7 +- .../index/shard/IndexShardTests.java | 2 +- 9 files changed, 108 insertions(+), 117 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index c35b7d810ca0e..a0a3d8acbb054 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -43,6 +43,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index aab6efb50e78f..a8007362aca28 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -1009,6 +1009,8 @@ public String toString() { final class AsyncPeerRecoveryRetentionLeaseRenewalTask extends BaseAsyncTask { + // TODO perhaps we can just piggyback on the synced flush that happens after 5 minutes of inactivity instead? + AsyncPeerRecoveryRetentionLeaseRenewalTask(final IndexService indexService) { super(indexService, RETENTION_LEASE_PEER_RECOVERY_SYNC_INTERVAL_SETTING.get(indexService.getIndexSettings().getSettings())); } 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 5fe90de96950d..5757635e297bd 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -395,7 +395,7 @@ public synchronized boolean peerRetentionLeasesNeedRenewal(long localCheckpointO * shard copy. These missing leases are created lazily if they're found to be missing during a * TransportReplicationAction, such as the peer recovery retention lease sync, so let's trigger a sync. */ - assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_0_0 in backport + assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_1_0 in backport return true; } return retentionLease.retainingSequenceNumber() < localCheckpointOfSafeCommit + 1; @@ -446,10 +446,14 @@ public synchronized void updatePeerRecoveryRetentionLeasesFromCheckpointState() final ShardRouting shardRouting = routingTable.getByAllocationId(entry.getKey()); final CheckpointState cps = entry.getValue(); if (cps.tracked) { - renewRetentionLease( - getPeerRecoveryRetentionLeaseId(shardRouting), - cps.localCheckpointOfSafeCommit + 1, - PEER_RECOVERY_RETENTION_LEASE_SOURCE); + if (getRetentionLeases().contains(getPeerRecoveryRetentionLeaseId(shardRouting))) { + renewRetentionLease( + getPeerRecoveryRetentionLeaseId(shardRouting), + cps.localCheckpointOfSafeCommit + 1, + PEER_RECOVERY_RETENTION_LEASE_SOURCE); + } else { + assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_1_0 in backport + } } } } @@ -471,7 +475,7 @@ public void addMissingPeerRecoveryRetentionLease(String allocationId, long start * copy. But the replica we're dealing with now has been upgraded and is retaining history while we asynchronously make it a * retention lease. */ - assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_0_0 in backport + assert indexCreatedVersion.before(Version.V_8_0_0) : indexCreatedVersion; // TODO V_7_1_0 in backport try { innerAddRetentionLease(getPeerRecoveryRetentionLeaseId(replicaShardRouting.currentNodeId()), startingSeqNo, PEER_RECOVERY_RETENTION_LEASE_SOURCE); @@ -744,7 +748,7 @@ private boolean invariant() { assert checkpoints.get(aId) != null : "aId [" + aId + "] is pending in sync but isn't tracked"; } - if (primaryMode && indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 after backporting + if (primaryMode && indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_1_0 after backporting for (final ShardRouting shardRouting : routingTable.assignedShards()) { assert checkpoints.get(shardRouting.allocationId().getId()).tracked == false || retentionLeases.contains(getPeerRecoveryRetentionLeaseId(shardRouting)) : @@ -944,7 +948,7 @@ public synchronized void activatePrimaryMode(final long localCheckpoint, final l * We might have got here here via a rolling upgrade from an older version that doesn't create peer recovery retention leases * for every shard copy. The missing leases are created in a more relaxed fashion, and offer weaker guarantees. */ - if (indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_0_0 in backport + if (indexCreatedVersion.onOrAfter(Version.V_8_0_0)) { // TODO V_7_1_0 in backport // We are starting up the whole replication group from scratch: if we were not (i.e. this is a replica promotion) then // this copy must already be in-sync and active and therefore holds a retention lease for itself. assert routingTable.activeShards().equals(singletonList(primaryShard)) : routingTable.activeShards(); 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 0a21c6fb24eec..10d9f562759d7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2532,9 +2532,10 @@ public Future renewPeerRecoveryRetentionLeases() { final PlainActionFuture plainActionFuture = new PlainActionFuture<>(); if (replicationTracker.peerRetentionLeasesNeedRenewal(getLocalCheckpointOfSafeCommit())) { peerRecoveryRetentionLeaseRenewer.accept(ActionListener.wrap(v -> { - replicationTracker.updatePeerRecoveryRetentionLeasesFromCheckpointState(); - plainActionFuture.onResponse(null); - }, plainActionFuture::onFailure)); + runUnderPrimaryPermit(replicationTracker::updatePeerRecoveryRetentionLeasesFromCheckpointState, + e -> logger.debug("exception updating peer-recovery retention leases", e), Names.SAME, ""); + plainActionFuture.onResponse(null); + }, plainActionFuture::onFailure)); } else { plainActionFuture.onResponse(null); } 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 81d10916591f7..cfecfb1a6d0ba 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -182,14 +182,13 @@ public void testAddRetentionLeaseCausesRetentionLeaseSync() { equalTo(retainingSequenceNumbers)); }, Version.CURRENT); reference.set(replicationTracker); - final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), - routingTable, + routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); - retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()), 0L); + retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; i++) { @@ -309,14 +308,13 @@ public void testRemoveRetentionLeaseCausesRetentionLeaseSync() { equalTo(retainingSequenceNumbers)); }, Version.CURRENT); reference.set(replicationTracker); - final IndexShardRoutingTable routingTable = routingTable(Collections.emptySet(), allocationId); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(allocationId.getId()), - routingTable, + routingTable(Collections.emptySet(), allocationId), Collections.emptySet()); replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); - retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(routingTable.primaryShard()), 0L); + retainingSequenceNumbers.put(getPeerRecoveryRetentionLeaseId(nodeIdFromAllocationId(allocationId)), 0L); final int length = randomIntBetween(0, 8); for (int i = 0; i < length; 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 0b98fe79815d6..c8e1078834b2c 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java @@ -22,6 +22,7 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.Murmur3HashFunction; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; @@ -54,10 +55,14 @@ ReplicationTracker newTracker( Version.CURRENT); } + 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); } @@ -67,7 +72,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 cebadfe6d0ec6..322cb60467bb8 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.seqno; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.replication.ReplicationResponse; @@ -58,9 +60,6 @@ import java.util.stream.Stream; import static java.util.Collections.emptySet; -import static java.util.Collections.singleton; -import static java.util.Collections.unmodifiableSet; -import static org.elasticsearch.action.ActionListener.wrap; import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; @@ -68,6 +67,8 @@ import static org.hamcrest.Matchers.not; public class ReplicationTrackerTests extends ReplicationTrackerTestCase { + + private static final Logger logger = LogManager.getLogger(ReplicationTrackerTests.class); public void testEmptyShards() { final ReplicationTracker tracker = newTracker(AllocationId.newInitializing()); @@ -124,11 +125,8 @@ public void testGlobalCheckpointUpdate() { logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); }); - final IndexShardRoutingTable routingTable = routingTable(initializing, primaryId); - tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable, emptySet()); + tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - initializing.forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {}))); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1)); initializing.forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); assertThat(tracker.getReplicationGroup().getReplicationTargets().size(), equalTo(1 + initializing.size())); @@ -154,12 +152,9 @@ public void testGlobalCheckpointUpdate() { Set newInitializing = new HashSet<>(initializing); newInitializing.add(extraId); - final IndexShardRoutingTable routingTable2 = routingTable(newInitializing, routingTable.primaryShard()); - newInitializing.forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable2.getByAllocationId(aId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {}))); - tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), - routingTable2, emptySet()); + tracker.updateFromMaster(initialClusterStateVersion + 1, ids(active), routingTable(newInitializing, primaryId), emptySet()); + addPeerRecoveryRetentionLease(tracker, extraId); tracker.initiateTracking(extraId.getId()); // now notify for the new id @@ -201,6 +196,7 @@ public void testMarkAllocationIdAsInSync() throws BrokenBarrierException, Interr tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); final long localCheckpoint = randomLongBetween(0, Long.MAX_VALUE - 1); tracker.activatePrimaryMode(localCheckpoint, NO_OPS_PERFORMED); + addPeerRecoveryRetentionLease(tracker, replicaId); tracker.initiateTracking(replicaId.getId()); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { @@ -240,12 +236,8 @@ public void testMissingActiveIdsPreventAdvance() { assigned.putAll(initializing); AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - initializing.keySet() - .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {}))); randomSubsetOf(initializing.keySet()).forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); final AllocationId missingActiveID = randomFrom(active.keySet()); assigned @@ -271,12 +263,8 @@ public void testMissingInSyncIdsPreventAdvance() { AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - initializing.keySet() - .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {}))); randomSubsetOf(randomIntBetween(1, initializing.size() - 1), initializing.keySet()).forEach(aId -> markAsTrackingAndInSyncQuietly(tracker, aId.getId(), NO_OPS_PERFORMED)); @@ -297,12 +285,8 @@ public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); final AllocationId primaryId = active.keySet().iterator().next(); final ReplicationTracker tracker = newTracker(primaryId); - final IndexShardRoutingTable routingTable = routingTable(initializing.keySet(), primaryId); - tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable, emptySet()); + tracker.updateFromMaster(randomNonNegativeLong(), ids(active.keySet()), routingTable(initializing.keySet(), primaryId), emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); - initializing.keySet() - .forEach(aId -> tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(aId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {}))); initializing.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); nonApproved.keySet().forEach(k -> expectThrows(IllegalStateException.class, () -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED))); @@ -336,8 +320,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { allocations.putAll(initializingToBeRemoved); } final ReplicationTracker tracker = newTracker(primaryId); - final IndexShardRoutingTable routingTable = routingTable(initializing, primaryId); - tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable, emptySet()); + tracker.updateFromMaster(initialClusterStateVersion, ids(active), routingTable(initializing, primaryId), emptySet()); tracker.activatePrimaryMode(NO_OPS_PERFORMED, NO_OPS_PERFORMED); if (randomBoolean()) { initializingToStay.keySet().forEach(k -> markAsTrackingAndInSyncQuietly(tracker, k.getId(), NO_OPS_PERFORMED)); @@ -353,7 +336,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { tracker.updateFromMaster( initialClusterStateVersion + 1, ids(activeToStay.keySet()), - routingTable(initializingToStay.keySet(), routingTable.primaryShard()), + routingTable(initializingToStay.keySet(), primaryId), emptySet()); allocations.forEach((aid, ckp) -> updateLocalCheckpoint(tracker, aid.getId(), ckp + 10L)); } else { @@ -361,7 +344,7 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { tracker.updateFromMaster( initialClusterStateVersion + 2, ids(activeToStay.keySet()), - routingTable(initializingToStay.keySet(), routingTable.primaryShard()), + routingTable(initializingToStay.keySet(), primaryId), emptySet()); } @@ -380,11 +363,10 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { final AllocationId trackingAllocationId = AllocationId.newInitializing(); final ReplicationTracker tracker = newTracker(inSyncAllocationId); final long clusterStateVersion = randomNonNegativeLong(); - final IndexShardRoutingTable routingTable = routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId); - tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), routingTable, emptySet()); + tracker.updateFromMaster(clusterStateVersion, Collections.singleton(inSyncAllocationId.getId()), + routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet()); tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); - tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(trackingAllocationId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {})); + addPeerRecoveryRetentionLease(tracker, trackingAllocationId); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -423,7 +405,7 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { } else { // master changes its mind and cancels the allocation tracker.updateFromMaster(clusterStateVersion + 1, Collections.singleton(inSyncAllocationId.getId()), - routingTable(emptySet(), routingTable.primaryShard()), emptySet()); + routingTable(emptySet(), inSyncAllocationId), emptySet()); barrier.await(); assertTrue(complete.get()); assertNull(tracker.getTrackedLocalCheckpointForShard(trackingAllocationId.getId())); @@ -446,12 +428,10 @@ public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBar final AllocationId inSyncAllocationId = AllocationId.newInitializing(); final AllocationId trackingAllocationId = AllocationId.newInitializing(); final ReplicationTracker tracker = newTracker(inSyncAllocationId); - final IndexShardRoutingTable routingTable = routingTable(singleton(trackingAllocationId), inSyncAllocationId); tracker.updateFromMaster(randomNonNegativeLong(), Collections.singleton(inSyncAllocationId.getId()), - routingTable, emptySet()); + routingTable(Collections.singleton(trackingAllocationId), inSyncAllocationId), emptySet()); tracker.activatePrimaryMode(globalCheckpoint, NO_OPS_PERFORMED); - tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(trackingAllocationId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {})); + addPeerRecoveryRetentionLease(tracker, trackingAllocationId); final Thread thread = new Thread(() -> { try { // synchronize starting with the test thread @@ -528,7 +508,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { final List removingInitializingAllocationIds = randomSubsetOf(initializingIds); final Set newInitializingAllocationIds = initializingIds.stream().filter(a -> !removingInitializingAllocationIds.contains(a)).collect(Collectors.toSet()); - routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); + routingTable = routingTable(newInitializingAllocationIds, primaryId); tracker.updateFromMaster(initialClusterStateVersion + 1, ids(newActiveAllocationIds), routingTable, emptySet()); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); assertTrue(removingActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()) == null)); @@ -546,7 +526,7 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { tracker.updateFromMaster( initialClusterStateVersion + 2, ids(newActiveAllocationIds), - routingTable(newInitializingAllocationIds, routingTable.primaryShard()), + routingTable(newInitializingAllocationIds, primaryId), emptySet()); assertTrue(newActiveAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); assertTrue( @@ -591,14 +571,12 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { // using a different length than we have been using above ensures that we can not collide with a previous allocation ID final AllocationId newSyncingAllocationId = AllocationId.newInitializing(); newInitializingAllocationIds.add(newSyncingAllocationId); - routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); - tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(newSyncingAllocationId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {})); tracker.updateFromMaster( initialClusterStateVersion + 3, ids(newActiveAllocationIds), - routingTable, + routingTable(newInitializingAllocationIds, primaryId), emptySet()); + addPeerRecoveryRetentionLease(tracker, newSyncingAllocationId); final CyclicBarrier barrier = new CyclicBarrier(2); final Thread thread = new Thread(() -> { try { @@ -631,14 +609,12 @@ public void testUpdateAllocationIdsFromMaster() throws Exception { /* * The new in-sync allocation ID is in the in-sync set now yet the master does not know this; the allocation ID should still be in * the in-sync set even if we receive a cluster state update that does not reflect this. + * */ - routingTable = routingTable(newInitializingAllocationIds, routingTable.primaryShard()); - tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(newSyncingAllocationId.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {})); tracker.updateFromMaster( initialClusterStateVersion + 4, ids(newActiveAllocationIds), - routingTable, + routingTable(newInitializingAllocationIds, primaryId), emptySet()); assertTrue(tracker.getTrackedLocalCheckpointForShard(newSyncingAllocationId.getId()).inSync); assertFalse(tracker.pendingInSync.contains(newSyncingAllocationId.getId())); @@ -649,7 +625,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)}). * @@ -664,15 +640,13 @@ public void testRaceUpdatingGlobalCheckpoint() throws InterruptedException, Brok final int activeLocalCheckpoint = randomIntBetween(0, Integer.MAX_VALUE - 1); final ReplicationTracker tracker = newTracker(active); - final IndexShardRoutingTable routingTable = routingTable(singleton(initializing), active); tracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(active.getId()), - routingTable, + routingTable(Collections.singleton(initializing), active), emptySet()); tracker.activatePrimaryMode(activeLocalCheckpoint, NO_OPS_PERFORMED); - tracker.addPeerRecoveryRetentionLease(routingTable.getByAllocationId(initializing.getId()).currentNodeId(), - 0, ActionListener.wrap(() -> {})); + addPeerRecoveryRetentionLease(tracker, initializing); final int nextActiveLocalCheckpoint = randomIntBetween(activeLocalCheckpoint + 1, Integer.MAX_VALUE); final Thread activeThread = new Thread(() -> { try { @@ -726,27 +700,19 @@ public void testPrimaryContextHandoff() throws IOException { final long globalCheckpoint = UNASSIGNED_SEQ_NO; final BiConsumer> onNewRetentionLease = (leases, listener) -> {}; - ReplicationTracker oldPrimary = new ReplicationTracker( - shardId, clusterState.routingTable.primaryShard().allocationId().getId(), - indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); - - clusterState.apply(oldPrimary); - oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10), NO_OPS_PERFORMED); + ReplicationTracker oldPrimary = new ReplicationTracker(shardId, + aId.getId(), indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); + ReplicationTracker newPrimary = new ReplicationTracker(shardId, + aId.getRelocationId(), indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); - clusterState = randomUpdateClusterState(singleton(oldPrimary.shardAllocationId), clusterState); - - ReplicationTracker newPrimary = new ReplicationTracker( - shardId, clusterState.routingTable.primaryShard().allocationId().getRelocationId(), - indexSettings, primaryTerm, globalCheckpoint, onUpdate, () -> 0L, onNewRetentionLease, Version.CURRENT); - - oldPrimary.addPeerRecoveryRetentionLease(clusterState.routingTable.primaryShard().relocatingNodeId(), 0L, wrap(() -> {})); - newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); + Set allocationIds = new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId)); clusterState.apply(oldPrimary); clusterState.apply(newPrimary); - final Set allocationIds - = unmodifiableSet(new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId))); + oldPrimary.activatePrimaryMode(randomIntBetween(Math.toIntExact(NO_OPS_PERFORMED), 10), NO_OPS_PERFORMED); + addPeerRecoveryRetentionLease(oldPrimary, newPrimary.shardAllocationId); + newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); final int numUpdates = randomInt(10); for (int i = 0; i < numUpdates; i++) { @@ -759,7 +725,8 @@ public void testPrimaryContextHandoff() throws IOException { randomLocalCheckpointUpdate(oldPrimary); } if (randomBoolean()) { - randomMarkInSync(oldPrimary); + randomMarkInSync(oldPrimary, newPrimary); + newPrimary.updateRetentionLeasesOnReplica(oldPrimary.getRetentionLeases()); } } @@ -791,7 +758,7 @@ public void testPrimaryContextHandoff() throws IOException { randomLocalCheckpointUpdate(oldPrimary); } if (randomBoolean()) { - randomMarkInSync(oldPrimary); + randomMarkInSync(oldPrimary, newPrimary); } // do another handoff @@ -923,10 +890,17 @@ private static FakeClusterState initialState() { final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); final Set initializingAllocationIds = activeAndInitializingAllocationIds.v2(); final AllocationId primaryId = randomFrom(activeAllocationIds); + final AllocationId relocatingId = AllocationId.newRelocation(primaryId); + activeAllocationIds.remove(primaryId); + activeAllocationIds.add(relocatingId); final ShardId shardId = new ShardId("test", "_na_", 0); final ShardRouting primaryShard = - TestShardRouting.newShardRouting( - shardId, randomAlphaOfLength(10), null, true, ShardRoutingState.STARTED, primaryId); + TestShardRouting.newShardRouting( + shardId, + nodeIdFromAllocationId(relocatingId), + nodeIdFromAllocationId(AllocationId.newInitializing(relocatingId.getRelocationId())), + true, ShardRoutingState.RELOCATING, relocatingId); + return new FakeClusterState( initialClusterStateVersion, activeAllocationIds, @@ -939,10 +913,11 @@ private static void randomLocalCheckpointUpdate(ReplicationTracker gcp) { 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 oldTracker, ReplicationTracker newTracker) { + String allocationId = randomFrom(oldTracker.checkpoints.keySet()); + long newLocalCheckpoint = Math.max(NO_OPS_PERFORMED, oldTracker.getGlobalCheckpoint() + randomInt(5)); + markAsTrackingAndInSyncQuietly(oldTracker, allocationId, newLocalCheckpoint); + newTracker.updateRetentionLeasesOnReplica(oldTracker.getRetentionLeases()); } private static FakeClusterState randomUpdateClusterState(Set allocationIds, FakeClusterState clusterState) { @@ -953,24 +928,15 @@ 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 initializingIds - = Sets.difference(Sets.union(clusterState.initializingIds(), initializingIdsToAdd), initializingIdsToRemove); - final ShardRouting primaryShard = clusterState.routingTable.primaryShard(); - assert !initializingIds.contains(primaryShard.allocationId()); - final ShardId shardId = new ShardId("test", "_na_", 0); - 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)); - } - - builder.addShard(primaryShard.state() == ShardRoutingState.RELOCATING - ? primaryShard : primaryShard.relocate(randomAlphaOfLength(10), 0)); - + 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, - builder.build()); + routingTable( + Sets.difference(Sets.union(initializingIdsExceptRelocationTargets, initializingIdsToAdd), initializingIdsToRemove), + clusterState.routingTable.primaryShard())); } private static Set exclude(Collection allocationIds, Set excludeIds) { @@ -1002,6 +968,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) { @@ -1009,4 +976,14 @@ 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, 0, ActionListener.wrap(() -> { })); + } + } + + private static void addPeerRecoveryRetentionLease(final ReplicationTracker tracker, final String allocationId) { + addPeerRecoveryRetentionLease(tracker, AllocationId.newInitializing(allocationId)); + } } 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 49bae233db4ae..a6a5007bea758 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; @@ -279,8 +280,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.version(), equalTo(1L)); + assertThat(recoveredRetentionLeases.leases(), hasSize(1)); + assertThat(recoveredRetentionLeases.leases().iterator().next().id(), + equalTo(ReplicationTracker.getPeerRecoveryRetentionLeaseId(indexShard.shardRouting))); } finally { closeShards(forceRecoveredShard); } 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 f348c0e172883..62f76b55a3e00 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2819,7 +2819,7 @@ public void testDocStats() throws Exception { } final FlushRequest flushRequest2 = new FlushRequest(); flushRequest2.force(true); - flushRequest2.waitIfOngoing(false); + flushRequest2.waitIfOngoing(true); indexShard.flush(flushRequest2); } From 684e395a115a6759017a43a9c365e3a28afdbc5e Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 21 Mar 2019 14:41:45 +0000 Subject: [PATCH 31/32] Allow retention lease syncs to happen even on closed indices --- .../replication/TransportReplicationAction.java | 9 ++++++++- .../index/seqno/RetentionLeaseSyncAction.java | 10 ++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 8a9d1b2015d03..5c800f3dd261d 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -232,6 +232,13 @@ protected boolean resolveIndex() { return true; } + /** + * True if this action can be replicated even on closed indices. + */ + protected boolean isReplicatedOnClosedIndices() { + return false; + } + protected TransportRequestOptions transportOptions(Settings settings) { return TransportRequestOptions.EMPTY; } @@ -775,7 +782,7 @@ protected void doRun() { retry(new IndexNotFoundException(concreteIndex)); return; } - if (indexMetaData.getState() == IndexMetaData.State.CLOSE) { + if (isReplicatedOnClosedIndices() == false && indexMetaData.getState() == IndexMetaData.State.CLOSE) { throw new IndexClosedException(indexMetaData.getIndex()); } diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java index d4845d92a3a6f..5fb8569dd6052 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLeaseSyncAction.java @@ -122,6 +122,16 @@ public void sync( } } + @Override + protected boolean resolveIndex() { + return false; + } + + @Override + protected boolean isReplicatedOnClosedIndices() { + return true; + } + @Override protected WritePrimaryResult shardOperationOnPrimary( final Request request, From 47ed9559a5b6b196435d98a026a0196ea868297f Mon Sep 17 00:00:00 2001 From: David Turner Date: Thu, 21 Mar 2019 17:56:04 +0000 Subject: [PATCH 32/32] Imports --- .../elasticsearch/cluster/routing/IndexShardRoutingTable.java | 1 - .../index/seqno/ReplicationTrackerRetentionLeaseTests.java | 1 - .../elasticsearch/index/seqno/ReplicationTrackerTestCase.java | 1 - 3 files changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java index a0a3d8acbb054..c35b7d810ca0e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java +++ b/server/src/main/java/org/elasticsearch/cluster/routing/IndexShardRoutingTable.java @@ -43,7 +43,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.stream.Collectors; import static java.util.Collections.emptyMap; 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 cfecfb1a6d0ba..a69df31c6f3ef 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.AllocationId; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; 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 c8e1078834b2c..734041928ccc9 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java @@ -22,7 +22,6 @@ import org.elasticsearch.Version; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.Murmur3HashFunction; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting;