Propagate max_auto_id_timestamp in peer recovery (#33693)

Today we don't store the auto-generated timestamp of append-only
operations in Lucene; and assign -1 to every index operations
constructed from LuceneChangesSnapshot. This looks innocent but it
generates duplicate documents on a replica if a retry append-only
arrives first via peer-recovery; then an original append-only arrives
via replication. Since the retry append-only (delivered via recovery)
does not have timestamp, the replica will happily optimizes the original
request while it should not.

This change transmits the max auto-generated timestamp from the primary
to replicas before translog phase in peer recovery. This timestamp will
prevent replicas from optimizing append-only requests if retry
counterparts have been processed.

Relates #33656 
Relates #33222
This commit is contained in:
Nhat Nguyen 2018-09-20 19:53:30 -04:00 committed by GitHub
parent 6646bcb065
commit 5f7f793f43
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 237 additions and 38 deletions

View File

@ -45,6 +45,7 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables; import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.CheckedRunnable;
import org.elasticsearch.common.FieldMemoryStats; import org.elasticsearch.common.FieldMemoryStats;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -1761,6 +1762,21 @@ public abstract class Engine implements Closeable {
*/ */
public abstract void maybePruneDeletes(); public abstract void maybePruneDeletes();
/**
* Returns the maximum auto_id_timestamp of all append-only index requests have been processed by this engine
* or the auto_id_timestamp received from its primary shard via {@link #updateMaxUnsafeAutoIdTimestamp(long)}.
* Notes this method returns the auto_id_timestamp of all append-only requests, not max_unsafe_auto_id_timestamp.
*/
public long getMaxSeenAutoIdTimestamp() {
return IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
}
/**
* Forces this engine to advance its max_unsafe_auto_id_timestamp marker to at least the given timestamp.
* The engine will disable optimization for all append-only whose timestamp at most {@code newTimestamp}.
*/
public abstract void updateMaxUnsafeAutoIdTimestamp(long newTimestamp);
@FunctionalInterface @FunctionalInterface
public interface TranslogRecoveryRunner { public interface TranslogRecoveryRunner {
int run(Engine engine, Translog.Snapshot snapshot) throws IOException; int run(Engine engine, Translog.Snapshot snapshot) throws IOException;

View File

@ -140,6 +140,7 @@ public class InternalEngine extends Engine {
private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false); private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false);
public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp";
private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1); private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1);
private final AtomicLong maxSeenAutoIdTimestamp = new AtomicLong(-1);
private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
private final CounterMetric numVersionLookups = new CounterMetric(); private final CounterMetric numVersionLookups = new CounterMetric();
private final CounterMetric numIndexVersionsLookups = new CounterMetric(); private final CounterMetric numIndexVersionsLookups = new CounterMetric();
@ -166,7 +167,7 @@ public class InternalEngine extends Engine {
final BiFunction<Long, Long, LocalCheckpointTracker> localCheckpointTrackerSupplier) { final BiFunction<Long, Long, LocalCheckpointTracker> localCheckpointTrackerSupplier) {
super(engineConfig); super(engineConfig);
if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) { if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) {
maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE); updateAutoIdTimestamp(Long.MAX_VALUE, true);
} }
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy( final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(
engineConfig.getIndexSettings().getTranslogRetentionSize().getBytes(), engineConfig.getIndexSettings().getTranslogRetentionSize().getBytes(),
@ -369,7 +370,7 @@ public class InternalEngine extends Engine {
if (key.equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) { if (key.equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) {
assert maxUnsafeAutoIdTimestamp.get() == -1 : assert maxUnsafeAutoIdTimestamp.get() == -1 :
"max unsafe timestamp was assigned already [" + maxUnsafeAutoIdTimestamp.get() + "]"; "max unsafe timestamp was assigned already [" + maxUnsafeAutoIdTimestamp.get() + "]";
maxUnsafeAutoIdTimestamp.set(Long.parseLong(entry.getValue())); updateAutoIdTimestamp(Long.parseLong(entry.getValue()), true);
} }
if (key.equals(SequenceNumbers.MAX_SEQ_NO)) { if (key.equals(SequenceNumbers.MAX_SEQ_NO)) {
assert maxSeqNoOfNonAppendOnlyOperations.get() == -1 : assert maxSeqNoOfNonAppendOnlyOperations.get() == -1 :
@ -1009,11 +1010,12 @@ public class InternalEngine extends Engine {
final boolean mayHaveBeenIndexBefore; final boolean mayHaveBeenIndexBefore;
if (index.isRetry()) { if (index.isRetry()) {
mayHaveBeenIndexBefore = true; mayHaveBeenIndexBefore = true;
maxUnsafeAutoIdTimestamp.updateAndGet(curr -> Math.max(index.getAutoGeneratedIdTimestamp(), curr)); updateAutoIdTimestamp(index.getAutoGeneratedIdTimestamp(), true);
assert maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp(); assert maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp();
} else { } else {
// in this case we force // in this case we force
mayHaveBeenIndexBefore = maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp(); mayHaveBeenIndexBefore = maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp();
updateAutoIdTimestamp(index.getAutoGeneratedIdTimestamp(), false);
} }
return mayHaveBeenIndexBefore; return mayHaveBeenIndexBefore;
} }
@ -2287,7 +2289,7 @@ public class InternalEngine extends Engine {
// this is an anti-viral settings you can only opt out for the entire index // this is an anti-viral settings you can only opt out for the entire index
// only if a shard starts up again due to relocation or if the index is closed // only if a shard starts up again due to relocation or if the index is closed
// the setting will be re-interpreted if it's set to true // the setting will be re-interpreted if it's set to true
this.maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE); updateAutoIdTimestamp(Long.MAX_VALUE, true);
} }
final TranslogDeletionPolicy translogDeletionPolicy = translog.getDeletionPolicy(); final TranslogDeletionPolicy translogDeletionPolicy = translog.getDeletionPolicy();
final IndexSettings indexSettings = engineConfig.getIndexSettings(); final IndexSettings indexSettings = engineConfig.getIndexSettings();
@ -2526,4 +2528,24 @@ public class InternalEngine extends Engine {
assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint; assert refreshedCheckpoint.get() >= checkpoint : refreshedCheckpoint.get() + " < " + checkpoint;
} }
} }
@Override
public final long getMaxSeenAutoIdTimestamp() {
return maxSeenAutoIdTimestamp.get();
}
@Override
public final void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) {
updateAutoIdTimestamp(newTimestamp, true);
}
private void updateAutoIdTimestamp(long newTimestamp, boolean unsafe) {
assert newTimestamp >= -1 : "invalid timestamp [" + newTimestamp + "]";
maxSeenAutoIdTimestamp.updateAndGet(curr -> Math.max(curr, newTimestamp));
if (unsafe) {
maxUnsafeAutoIdTimestamp.updateAndGet(curr -> Math.max(curr, newTimestamp));
}
assert maxUnsafeAutoIdTimestamp.get() <= maxSeenAutoIdTimestamp.get();
}
} }

View File

@ -374,4 +374,9 @@ public final class ReadOnlyEngine extends Engine {
public DocsStats docStats() { public DocsStats docStats() {
return docsStats; return docsStats;
} }
@Override
public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) {
}
} }

View File

@ -1219,6 +1219,29 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
getEngine().trimOperationsFromTranslog(operationPrimaryTerm, aboveSeqNo); getEngine().trimOperationsFromTranslog(operationPrimaryTerm, aboveSeqNo);
} }
/**
* Returns the maximum auto_id_timestamp of all append-only requests have been processed by this shard or the auto_id_timestamp received
* from the primary via {@link #updateMaxUnsafeAutoIdTimestamp(long)} at the beginning of a peer-recovery or a primary-replica resync.
*
* @see #updateMaxUnsafeAutoIdTimestamp(long)
*/
public long getMaxSeenAutoIdTimestamp() {
return getEngine().getMaxSeenAutoIdTimestamp();
}
/**
* Since operations stored in soft-deletes do not have max_auto_id_timestamp, the primary has to propagate its max_auto_id_timestamp
* (via {@link #getMaxSeenAutoIdTimestamp()} of all processed append-only requests to replicas at the beginning of a peer-recovery
* or a primary-replica resync to force a replica to disable optimization for all append-only requests which are replicated via
* replication while its retry variants are replicated via recovery without auto_id_timestamp.
* <p>
* Without this force-update, a replica can generate duplicate documents (for the same id) if it first receives
* a retry append-only (without timestamp) via recovery, then an original append-only (with timestamp) via replication.
*/
public void updateMaxUnsafeAutoIdTimestamp(long maxSeenAutoIdTimestampFromPrimary) {
getEngine().updateMaxUnsafeAutoIdTimestamp(maxSeenAutoIdTimestampFromPrimary);
}
public Engine.Result applyTranslogOperation(Translog.Operation operation, Engine.Operation.Origin origin) throws IOException { public Engine.Result applyTranslogOperation(Translog.Operation operation, Engine.Operation.Origin origin) throws IOException {
// If a translog op is replayed on the primary (eg. ccr), we need to use external instead of null for its version type. // If a translog op is replayed on the primary (eg. ccr), we need to use external instead of null for its version type.
final VersionType versionType = (origin == Engine.Operation.Origin.PRIMARY) ? VersionType.EXTERNAL : null; final VersionType versionType = (origin == Engine.Operation.Origin.PRIMARY) ? VersionType.EXTERNAL : null;

View File

@ -455,7 +455,8 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext()); final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext());
final RecoveryTarget recoveryTarget = recoveryRef.target(); final RecoveryTarget recoveryTarget = recoveryRef.target();
try { try {
recoveryTarget.indexTranslogOperations(request.operations(), request.totalTranslogOps()); recoveryTarget.indexTranslogOperations(request.operations(), request.totalTranslogOps(),
request.maxSeenAutoIdTimestampOnPrimary());
channel.sendResponse(new RecoveryTranslogOperationsResponse(recoveryTarget.indexShard().getLocalCheckpoint())); channel.sendResponse(new RecoveryTranslogOperationsResponse(recoveryTarget.indexShard().getLocalCheckpoint()));
} catch (MapperException exception) { } catch (MapperException exception) {
// in very rare cases a translog replay from primary is processed before a mapping update on this node // in very rare cases a translog replay from primary is processed before a mapping update on this node

View File

@ -215,7 +215,10 @@ public class RecoverySourceHandler {
} }
final long targetLocalCheckpoint; final long targetLocalCheckpoint;
try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) {
targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); // We have to capture the max auto_id_timestamp after taking a snapshot of operations to guarantee
// that the auto_id_timestamp of every operation in the snapshot is at most this timestamp value.
final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp();
targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, maxSeenAutoIdTimestamp);
} catch (Exception e) { } catch (Exception e) {
throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e);
} }
@ -447,9 +450,11 @@ public class RecoverySourceHandler {
* @param requiredSeqNoRangeStart the lower sequence number of the required range (ending with endingSeqNo) * @param requiredSeqNoRangeStart the lower sequence number of the required range (ending with endingSeqNo)
* @param endingSeqNo the highest sequence number that should be sent * @param endingSeqNo the highest sequence number that should be sent
* @param snapshot a snapshot of the translog * @param snapshot a snapshot of the translog
* @param maxSeenAutoIdTimestamp the max auto_id_timestamp of append-only requests on the primary
* @return the local checkpoint on the target * @return the local checkpoint on the target
*/ */
long phase2(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, final Translog.Snapshot snapshot) long phase2(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, final Translog.Snapshot snapshot,
final long maxSeenAutoIdTimestamp)
throws IOException { throws IOException {
if (shard.state() == IndexShardState.CLOSED) { if (shard.state() == IndexShardState.CLOSED) {
throw new IndexShardClosedException(request.shardId()); throw new IndexShardClosedException(request.shardId());
@ -462,7 +467,8 @@ public class RecoverySourceHandler {
"required [" + requiredSeqNoRangeStart + ":" + endingSeqNo + "]"); "required [" + requiredSeqNoRangeStart + ":" + endingSeqNo + "]");
// send all the snapshot's translog operations to the target // send all the snapshot's translog operations to the target
final SendSnapshotResult result = sendSnapshot(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot); final SendSnapshotResult result = sendSnapshot(
startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, maxSeenAutoIdTimestamp);
stopWatch.stop(); stopWatch.stop();
logger.trace("recovery [phase2]: took [{}]", stopWatch.totalTime()); logger.trace("recovery [phase2]: took [{}]", stopWatch.totalTime());
@ -530,10 +536,11 @@ public class RecoverySourceHandler {
* @param endingSeqNo the upper bound of the sequence number range to be sent (inclusive) * @param endingSeqNo the upper bound of the sequence number range to be sent (inclusive)
* @param snapshot the translog snapshot to replay operations from @return the local checkpoint on the target and the * @param snapshot the translog snapshot to replay operations from @return the local checkpoint on the target and the
* total number of operations sent * total number of operations sent
* @param maxSeenAutoIdTimestamp the max auto_id_timestamp of append-only requests on the primary
* @throws IOException if an I/O exception occurred reading the translog snapshot * @throws IOException if an I/O exception occurred reading the translog snapshot
*/ */
protected SendSnapshotResult sendSnapshot(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, protected SendSnapshotResult sendSnapshot(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo,
final Translog.Snapshot snapshot) throws IOException { final Translog.Snapshot snapshot, final long maxSeenAutoIdTimestamp) throws IOException {
assert requiredSeqNoRangeStart <= endingSeqNo + 1: assert requiredSeqNoRangeStart <= endingSeqNo + 1:
"requiredSeqNoRangeStart " + requiredSeqNoRangeStart + " is larger than endingSeqNo " + endingSeqNo; "requiredSeqNoRangeStart " + requiredSeqNoRangeStart + " is larger than endingSeqNo " + endingSeqNo;
assert startingSeqNo <= requiredSeqNoRangeStart : assert startingSeqNo <= requiredSeqNoRangeStart :
@ -551,8 +558,8 @@ public class RecoverySourceHandler {
logger.trace("no translog operations to send"); logger.trace("no translog operations to send");
} }
final CancellableThreads.IOInterruptable sendBatch = final CancellableThreads.IOInterruptable sendBatch = () ->
() -> targetLocalCheckpoint.set(recoveryTarget.indexTranslogOperations(operations, expectedTotalOps)); targetLocalCheckpoint.set(recoveryTarget.indexTranslogOperations(operations, expectedTotalOps, maxSeenAutoIdTimestamp));
// send operations in batches // send operations in batches
Translog.Operation operation; Translog.Operation operation;

View File

@ -386,13 +386,21 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
} }
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
long maxSeenAutoIdTimestampOnPrimary) throws IOException {
final RecoveryState.Translog translog = state().getTranslog(); final RecoveryState.Translog translog = state().getTranslog();
translog.totalOperations(totalTranslogOps); translog.totalOperations(totalTranslogOps);
assert indexShard().recoveryState() == state(); assert indexShard().recoveryState() == state();
if (indexShard().state() != IndexShardState.RECOVERING) { if (indexShard().state() != IndexShardState.RECOVERING) {
throw new IndexShardNotRecoveringException(shardId, indexShard().state()); throw new IndexShardNotRecoveringException(shardId, indexShard().state());
} }
/*
* The maxSeenAutoIdTimestampOnPrimary received from the primary is at least the highest auto_id_timestamp from any operation
* will be replayed. Bootstrapping this timestamp here will disable the optimization for original append-only requests
* (source of these operations) replicated via replication. Without this step, we may have duplicate documents if we
* replay these operations first (without timestamp), then optimize append-only requests (with timestamp).
*/
indexShard().updateMaxUnsafeAutoIdTimestamp(maxSeenAutoIdTimestampOnPrimary);
for (Translog.Operation operation : operations) { for (Translog.Operation operation : operations) {
Engine.Result result = indexShard().applyTranslogOperation(operation, Engine.Operation.Origin.PEER_RECOVERY); Engine.Result result = indexShard().applyTranslogOperation(operation, Engine.Operation.Origin.PEER_RECOVERY);
if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) { if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) {

View File

@ -59,12 +59,13 @@ public interface RecoveryTargetHandler {
/** /**
* Index a set of translog operations on the target * Index a set of translog operations on the target
* @param operations operations to index * @param operations operations to index
* @param totalTranslogOps current number of total operations expected to be indexed * @param totalTranslogOps current number of total operations expected to be indexed
* * @param maxSeenAutoIdTimestampOnPrimary the maximum auto_id_timestamp of all append-only requests processed by the primary shard
* @return the local checkpoint on the target shard * @return the local checkpoint on the target shard
*/ */
long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException; long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
long maxSeenAutoIdTimestampOnPrimary) throws IOException;
/** /**
* Notifies the target of the files it is going to receive * Notifies the target of the files it is going to receive

View File

@ -19,6 +19,8 @@
package org.elasticsearch.indices.recovery; package org.elasticsearch.indices.recovery;
import org.elasticsearch.Version;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
@ -34,15 +36,18 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
private ShardId shardId; private ShardId shardId;
private List<Translog.Operation> operations; private List<Translog.Operation> operations;
private int totalTranslogOps = RecoveryState.Translog.UNKNOWN; private int totalTranslogOps = RecoveryState.Translog.UNKNOWN;
private long maxSeenAutoIdTimestampOnPrimary;
public RecoveryTranslogOperationsRequest() { public RecoveryTranslogOperationsRequest() {
} }
RecoveryTranslogOperationsRequest(long recoveryId, ShardId shardId, List<Translog.Operation> operations, int totalTranslogOps) { RecoveryTranslogOperationsRequest(long recoveryId, ShardId shardId, List<Translog.Operation> operations,
int totalTranslogOps, long maxSeenAutoIdTimestampOnPrimary) {
this.recoveryId = recoveryId; this.recoveryId = recoveryId;
this.shardId = shardId; this.shardId = shardId;
this.operations = operations; this.operations = operations;
this.totalTranslogOps = totalTranslogOps; this.totalTranslogOps = totalTranslogOps;
this.maxSeenAutoIdTimestampOnPrimary = maxSeenAutoIdTimestampOnPrimary;
} }
public long recoveryId() { public long recoveryId() {
@ -61,6 +66,10 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
return totalTranslogOps; return totalTranslogOps;
} }
public long maxSeenAutoIdTimestampOnPrimary() {
return maxSeenAutoIdTimestampOnPrimary;
}
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);
@ -68,6 +77,11 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
shardId = ShardId.readShardId(in); shardId = ShardId.readShardId(in);
operations = Translog.readOperations(in, "recovery"); operations = Translog.readOperations(in, "recovery");
totalTranslogOps = in.readVInt(); totalTranslogOps = in.readVInt();
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
} else {
maxSeenAutoIdTimestampOnPrimary = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
}
} }
@Override @Override
@ -77,5 +91,8 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
shardId.writeTo(out); shardId.writeTo(out);
Translog.writeOperations(out, operations); Translog.writeOperations(out, operations);
out.writeVInt(totalTranslogOps); out.writeVInt(totalTranslogOps);
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
}
} }
} }

View File

@ -110,9 +110,9 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
} }
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps, long maxSeenAutoIdTimestampOnPrimary) {
final RecoveryTranslogOperationsRequest translogOperationsRequest = final RecoveryTranslogOperationsRequest translogOperationsRequest =
new RecoveryTranslogOperationsRequest(recoveryId, shardId, operations, totalTranslogOps); new RecoveryTranslogOperationsRequest(recoveryId, shardId, operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary);
final TransportFuture<RecoveryTranslogOperationsResponse> future = transportService.submitRequest( final TransportFuture<RecoveryTranslogOperationsResponse> future = transportService.submitRequest(
targetNode, targetNode,
PeerRecoveryTargetService.Actions.TRANSLOG_OPS, PeerRecoveryTargetService.Actions.TRANSLOG_OPS,

View File

@ -3537,6 +3537,8 @@ public class InternalEngineTests extends EngineTestCase {
} }
assertEquals(0, engine.getNumVersionLookups()); assertEquals(0, engine.getNumVersionLookups());
assertEquals(0, engine.getNumIndexVersionsLookups()); assertEquals(0, engine.getNumIndexVersionsLookups());
assertThat(engine.getMaxSeenAutoIdTimestamp(),
equalTo(docs.stream().mapToLong(Engine.Index::getAutoGeneratedIdTimestamp).max().getAsLong()));
assertLuceneOperations(engine, numDocs, 0, 0); assertLuceneOperations(engine, numDocs, 0, 0);
} }

View File

@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.elasticsearch.Version;
import org.elasticsearch.action.DocWriteResponse; import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.bulk.BulkShardRequest;
@ -141,10 +142,80 @@ public class IndexLevelReplicationTests extends ESIndexLevelReplicationTestCase
} }
} }
public void testRetryAppendOnlyAfterRecovering() throws Exception {
try (ReplicationGroup shards = createGroup(0)) {
shards.startAll();
final IndexRequest originalRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON);
originalRequest.process(Version.CURRENT, null, index.getName());
final IndexRequest retryRequest = copyIndexRequest(originalRequest);
retryRequest.onRetry();
shards.index(retryRequest);
IndexShard replica = shards.addReplica();
shards.recoverReplica(replica);
shards.assertAllEqual(1);
shards.index(originalRequest); // original append-only arrives after recovery completed
shards.assertAllEqual(1);
assertThat(replica.getMaxSeenAutoIdTimestamp(), equalTo(originalRequest.getAutoGeneratedTimestamp()));
}
}
public void testAppendOnlyRecoveryThenReplication() throws Exception {
CountDownLatch indexedOnPrimary = new CountDownLatch(1);
CountDownLatch recoveryDone = new CountDownLatch(1);
try (ReplicationGroup shards = new ReplicationGroup(buildIndexMetaData(1)) {
@Override
protected EngineFactory getEngineFactory(ShardRouting routing) {
return config -> new InternalEngine(config) {
@Override
public IndexResult index(Index op) throws IOException {
IndexResult result = super.index(op);
if (op.origin() == Operation.Origin.PRIMARY) {
indexedOnPrimary.countDown();
// prevent the indexing on the primary from returning (it was added to Lucene and translog already)
// to make sure that this operation is replicated to the replica via recovery, then via replication.
try {
recoveryDone.await();
} catch (InterruptedException e) {
throw new AssertionError(e);
}
}
return result;
}
};
}
}) {
shards.startAll();
Thread thread = new Thread(() -> {
IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON);
try {
shards.index(indexRequest);
} catch (Exception e) {
throw new AssertionError(e);
}
});
thread.start();
IndexShard replica = shards.addReplica();
Future<Void> fut = shards.asyncRecoverReplica(replica,
(shard, node) -> new RecoveryTarget(shard, node, recoveryListener, v -> {}){
@Override
public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException {
try {
indexedOnPrimary.await();
} catch (InterruptedException e) {
throw new AssertionError(e);
}
super.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps);
}
});
fut.get();
recoveryDone.countDown();
thread.join();
shards.assertAllEqual(1);
}
}
public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception { public void testInheritMaxValidAutoIDTimestampOnRecovery() throws Exception {
//TODO: Enables this test with soft-deletes once we have timestamp try (ReplicationGroup shards = createGroup(0)) {
Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build();
try (ReplicationGroup shards = createGroup(0, settings)) {
shards.startAll(); shards.startAll();
final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON); final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON);
indexRequest.onRetry(); // force an update of the timestamp indexRequest.onRetry(); // force an update of the timestamp

View File

@ -488,9 +488,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
return new RecoveryTarget(indexShard, node, recoveryListener, l -> { return new RecoveryTarget(indexShard, node, recoveryListener, l -> {
}) { }) {
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
long maxSeenAutoIdTimestampOnPrimary) throws IOException {
opsSent.set(true); opsSent.set(true);
return super.indexTranslogOperations(operations, totalTranslogOps); return super.indexTranslogOperations(operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary);
} }
}; };
}); });
@ -557,7 +558,8 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
replica, replica,
(indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, l -> {}) { (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, l -> {}) {
@Override @Override
public long indexTranslogOperations(final List<Translog.Operation> operations, final int totalTranslogOps) public long indexTranslogOperations(final List<Translog.Operation> operations, final int totalTranslogOps,
final long maxAutoIdTimestamp)
throws IOException { throws IOException {
// index a doc which is not part of the snapshot, but also does not complete on replica // index a doc which is not part of the snapshot, but also does not complete on replica
replicaEngineFactory.latchIndexers(1); replicaEngineFactory.latchIndexers(1);
@ -585,7 +587,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
} catch (InterruptedException e) { } catch (InterruptedException e) {
throw new AssertionError(e); throw new AssertionError(e);
} }
return super.indexTranslogOperations(operations, totalTranslogOps); return super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
} }
}); });
pendingDocActiveWithExtraDocIndexed.await(); pendingDocActiveWithExtraDocIndexed.await();
@ -671,11 +673,12 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
} }
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
long maxAutoIdTimestamp) throws IOException {
if (hasBlocked() == false) { if (hasBlocked() == false) {
blockIfNeeded(RecoveryState.Stage.TRANSLOG); blockIfNeeded(RecoveryState.Stage.TRANSLOG);
} }
return super.indexTranslogOperations(operations, totalTranslogOps); return super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
} }
@Override @Override

View File

@ -2191,8 +2191,9 @@ public class IndexShardTests extends IndexShardTestCase {
new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> { new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> {
}) { }) {
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps); long maxSeenAutoIdTimestamp) throws IOException {
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxSeenAutoIdTimestamp);
assertFalse(replica.isSyncNeeded()); assertFalse(replica.isSyncNeeded());
return localCheckpoint; return localCheckpoint;
} }
@ -2298,8 +2299,9 @@ public class IndexShardTests extends IndexShardTestCase {
new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> { new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> {
}) { }) {
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps); long maxAutoIdTimestamp) throws IOException {
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
// Shard should now be active since we did recover: // Shard should now be active since we did recover:
assertTrue(replica.isActive()); assertTrue(replica.isActive());
return localCheckpoint; return localCheckpoint;
@ -2345,8 +2347,9 @@ public class IndexShardTests extends IndexShardTestCase {
} }
@Override @Override
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException { public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps); long maxAutoIdTimestamp) throws IOException {
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
assertListenerCalled.accept(replica); assertListenerCalled.accept(replica);
return localCheckpoint; return localCheckpoint;
} }

View File

@ -207,11 +207,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
public Translog.Operation next() throws IOException { public Translog.Operation next() throws IOException {
return operations.get(counter++); return operations.get(counter++);
} }
}); }, randomNonNegativeLong());
final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1); final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1);
assertThat(result.totalOperations, equalTo(expectedOps)); assertThat(result.totalOperations, equalTo(expectedOps));
final ArgumentCaptor<List> shippedOpsCaptor = ArgumentCaptor.forClass(List.class); final ArgumentCaptor<List> shippedOpsCaptor = ArgumentCaptor.forClass(List.class);
verify(recoveryTarget).indexTranslogOperations(shippedOpsCaptor.capture(), ArgumentCaptor.forClass(Integer.class).capture()); verify(recoveryTarget).indexTranslogOperations(shippedOpsCaptor.capture(), ArgumentCaptor.forClass(Integer.class).capture(), ArgumentCaptor.forClass(Long.class).capture());
List<Translog.Operation> shippedOps = new ArrayList<>(); List<Translog.Operation> shippedOps = new ArrayList<>();
for (List list: shippedOpsCaptor.getAllValues()) { for (List list: shippedOpsCaptor.getAllValues()) {
shippedOps.addAll(list); shippedOps.addAll(list);
@ -249,7 +249,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
} while (op != null && opsToSkip.contains(op)); } while (op != null && opsToSkip.contains(op));
return op; return op;
} }
})); }, randomNonNegativeLong()));
} }
} }
@ -420,7 +420,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
} }
@Override @Override
long phase2(long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, Translog.Snapshot snapshot) throws IOException { long phase2(long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, Translog.Snapshot snapshot,
long maxSeenAutoIdTimestamp) {
phase2Called.set(true); phase2Called.set(true);
return SequenceNumbers.UNASSIGNED_SEQ_NO; return SequenceNumbers.UNASSIGNED_SEQ_NO;
} }

View File

@ -55,6 +55,8 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -137,6 +139,17 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
return metaData.build(); return metaData.build();
} }
protected IndexRequest copyIndexRequest(IndexRequest inRequest) throws IOException {
final IndexRequest outRequest = new IndexRequest();
try (BytesStreamOutput out = new BytesStreamOutput()) {
inRequest.writeTo(out);
try (StreamInput in = out.bytes().streamInput()) {
outRequest.readFrom(in);
}
}
return outRequest;
}
protected DiscoveryNode getDiscoveryNode(String id) { protected DiscoveryNode getDiscoveryNode(String id) {
return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(), return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(),
Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT); Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT);
@ -428,6 +441,12 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
public synchronized void close() throws Exception { public synchronized void close() throws Exception {
if (closed == false) { if (closed == false) {
closed = true; closed = true;
for (IndexShard replica : replicas) {
try {
assertThat(replica.getMaxSeenAutoIdTimestamp(), equalTo(primary.getMaxSeenAutoIdTimestamp()));
} catch (AlreadyClosedException ignored) {
}
}
closeShards(this); closeShards(this);
} else { } else {
throw new AlreadyClosedException("too bad"); throw new AlreadyClosedException("too bad");