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:
parent
6646bcb065
commit
5f7f793f43
|
@ -45,6 +45,7 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.Accountables;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.action.index.IndexRequest;
|
||||
import org.elasticsearch.common.CheckedRunnable;
|
||||
import org.elasticsearch.common.FieldMemoryStats;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
|
@ -1761,6 +1762,21 @@ public abstract class Engine implements Closeable {
|
|||
*/
|
||||
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
|
||||
public interface TranslogRecoveryRunner {
|
||||
int run(Engine engine, Translog.Snapshot snapshot) throws IOException;
|
||||
|
|
|
@ -140,6 +140,7 @@ public class InternalEngine extends Engine {
|
|||
private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false);
|
||||
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 maxSeenAutoIdTimestamp = new AtomicLong(-1);
|
||||
private final AtomicLong maxSeqNoOfNonAppendOnlyOperations = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
||||
private final CounterMetric numVersionLookups = new CounterMetric();
|
||||
private final CounterMetric numIndexVersionsLookups = new CounterMetric();
|
||||
|
@ -166,7 +167,7 @@ public class InternalEngine extends Engine {
|
|||
final BiFunction<Long, Long, LocalCheckpointTracker> localCheckpointTrackerSupplier) {
|
||||
super(engineConfig);
|
||||
if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) {
|
||||
maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE);
|
||||
updateAutoIdTimestamp(Long.MAX_VALUE, true);
|
||||
}
|
||||
final TranslogDeletionPolicy translogDeletionPolicy = new TranslogDeletionPolicy(
|
||||
engineConfig.getIndexSettings().getTranslogRetentionSize().getBytes(),
|
||||
|
@ -369,7 +370,7 @@ public class InternalEngine extends Engine {
|
|||
if (key.equals(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID)) {
|
||||
assert maxUnsafeAutoIdTimestamp.get() == -1 :
|
||||
"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)) {
|
||||
assert maxSeqNoOfNonAppendOnlyOperations.get() == -1 :
|
||||
|
@ -1009,11 +1010,12 @@ public class InternalEngine extends Engine {
|
|||
final boolean mayHaveBeenIndexBefore;
|
||||
if (index.isRetry()) {
|
||||
mayHaveBeenIndexBefore = true;
|
||||
maxUnsafeAutoIdTimestamp.updateAndGet(curr -> Math.max(index.getAutoGeneratedIdTimestamp(), curr));
|
||||
updateAutoIdTimestamp(index.getAutoGeneratedIdTimestamp(), true);
|
||||
assert maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp();
|
||||
} else {
|
||||
// in this case we force
|
||||
mayHaveBeenIndexBefore = maxUnsafeAutoIdTimestamp.get() >= index.getAutoGeneratedIdTimestamp();
|
||||
updateAutoIdTimestamp(index.getAutoGeneratedIdTimestamp(), false);
|
||||
}
|
||||
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
|
||||
// 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
|
||||
this.maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE);
|
||||
updateAutoIdTimestamp(Long.MAX_VALUE, true);
|
||||
}
|
||||
final TranslogDeletionPolicy translogDeletionPolicy = translog.getDeletionPolicy();
|
||||
final IndexSettings indexSettings = engineConfig.getIndexSettings();
|
||||
|
@ -2526,4 +2528,24 @@ public class InternalEngine extends Engine {
|
|||
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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -374,4 +374,9 @@ public final class ReadOnlyEngine extends Engine {
|
|||
public DocsStats docStats() {
|
||||
return docsStats;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateMaxUnsafeAutoIdTimestamp(long newTimestamp) {
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1219,6 +1219,29 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
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 {
|
||||
// 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;
|
||||
|
|
|
@ -455,7 +455,8 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
|
|||
final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger, threadPool.getThreadContext());
|
||||
final RecoveryTarget recoveryTarget = recoveryRef.target();
|
||||
try {
|
||||
recoveryTarget.indexTranslogOperations(request.operations(), request.totalTranslogOps());
|
||||
recoveryTarget.indexTranslogOperations(request.operations(), request.totalTranslogOps(),
|
||||
request.maxSeenAutoIdTimestampOnPrimary());
|
||||
channel.sendResponse(new RecoveryTranslogOperationsResponse(recoveryTarget.indexShard().getLocalCheckpoint()));
|
||||
} catch (MapperException exception) {
|
||||
// in very rare cases a translog replay from primary is processed before a mapping update on this node
|
||||
|
|
|
@ -215,7 +215,10 @@ public class RecoverySourceHandler {
|
|||
}
|
||||
final long targetLocalCheckpoint;
|
||||
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) {
|
||||
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 endingSeqNo the highest sequence number that should be sent
|
||||
* @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
|
||||
*/
|
||||
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 {
|
||||
if (shard.state() == IndexShardState.CLOSED) {
|
||||
throw new IndexShardClosedException(request.shardId());
|
||||
|
@ -462,7 +467,8 @@ public class RecoverySourceHandler {
|
|||
"required [" + requiredSeqNoRangeStart + ":" + endingSeqNo + "]");
|
||||
|
||||
// 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();
|
||||
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 snapshot the translog snapshot to replay operations from @return the local checkpoint on the target and the
|
||||
* 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
|
||||
*/
|
||||
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:
|
||||
"requiredSeqNoRangeStart " + requiredSeqNoRangeStart + " is larger than endingSeqNo " + endingSeqNo;
|
||||
assert startingSeqNo <= requiredSeqNoRangeStart :
|
||||
|
@ -551,8 +558,8 @@ public class RecoverySourceHandler {
|
|||
logger.trace("no translog operations to send");
|
||||
}
|
||||
|
||||
final CancellableThreads.IOInterruptable sendBatch =
|
||||
() -> targetLocalCheckpoint.set(recoveryTarget.indexTranslogOperations(operations, expectedTotalOps));
|
||||
final CancellableThreads.IOInterruptable sendBatch = () ->
|
||||
targetLocalCheckpoint.set(recoveryTarget.indexTranslogOperations(operations, expectedTotalOps, maxSeenAutoIdTimestamp));
|
||||
|
||||
// send operations in batches
|
||||
Translog.Operation operation;
|
||||
|
|
|
@ -386,13 +386,21 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
|
|||
}
|
||||
|
||||
@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();
|
||||
translog.totalOperations(totalTranslogOps);
|
||||
assert indexShard().recoveryState() == state();
|
||||
if (indexShard().state() != IndexShardState.RECOVERING) {
|
||||
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) {
|
||||
Engine.Result result = indexShard().applyTranslogOperation(operation, Engine.Operation.Origin.PEER_RECOVERY);
|
||||
if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) {
|
||||
|
|
|
@ -59,12 +59,13 @@ public interface RecoveryTargetHandler {
|
|||
|
||||
/**
|
||||
* Index a set of translog operations on the target
|
||||
* @param operations operations to index
|
||||
* @param totalTranslogOps current number of total operations expected to be indexed
|
||||
*
|
||||
* @param operations operations to index
|
||||
* @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
|
||||
*/
|
||||
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
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
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.StreamOutput;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
@ -34,15 +36,18 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
|
|||
private ShardId shardId;
|
||||
private List<Translog.Operation> operations;
|
||||
private int totalTranslogOps = RecoveryState.Translog.UNKNOWN;
|
||||
private long maxSeenAutoIdTimestampOnPrimary;
|
||||
|
||||
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.shardId = shardId;
|
||||
this.operations = operations;
|
||||
this.totalTranslogOps = totalTranslogOps;
|
||||
this.maxSeenAutoIdTimestampOnPrimary = maxSeenAutoIdTimestampOnPrimary;
|
||||
}
|
||||
|
||||
public long recoveryId() {
|
||||
|
@ -61,6 +66,10 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
|
|||
return totalTranslogOps;
|
||||
}
|
||||
|
||||
public long maxSeenAutoIdTimestampOnPrimary() {
|
||||
return maxSeenAutoIdTimestampOnPrimary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
|
@ -68,6 +77,11 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
|
|||
shardId = ShardId.readShardId(in);
|
||||
operations = Translog.readOperations(in, "recovery");
|
||||
totalTranslogOps = in.readVInt();
|
||||
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
|
||||
maxSeenAutoIdTimestampOnPrimary = in.readZLong();
|
||||
} else {
|
||||
maxSeenAutoIdTimestampOnPrimary = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -77,5 +91,8 @@ public class RecoveryTranslogOperationsRequest extends TransportRequest {
|
|||
shardId.writeTo(out);
|
||||
Translog.writeOperations(out, operations);
|
||||
out.writeVInt(totalTranslogOps);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
|
||||
out.writeZLong(maxSeenAutoIdTimestampOnPrimary);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -110,9 +110,9 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) {
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps, long maxSeenAutoIdTimestampOnPrimary) {
|
||||
final RecoveryTranslogOperationsRequest translogOperationsRequest =
|
||||
new RecoveryTranslogOperationsRequest(recoveryId, shardId, operations, totalTranslogOps);
|
||||
new RecoveryTranslogOperationsRequest(recoveryId, shardId, operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary);
|
||||
final TransportFuture<RecoveryTranslogOperationsResponse> future = transportService.submitRequest(
|
||||
targetNode,
|
||||
PeerRecoveryTargetService.Actions.TRANSLOG_OPS,
|
||||
|
|
|
@ -3537,6 +3537,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
assertEquals(0, engine.getNumVersionLookups());
|
||||
assertEquals(0, engine.getNumIndexVersionsLookups());
|
||||
assertThat(engine.getMaxSeenAutoIdTimestamp(),
|
||||
equalTo(docs.stream().mapToLong(Engine.Index::getAutoGeneratedIdTimestamp).max().getAsLong()));
|
||||
assertLuceneOperations(engine, numDocs, 0, 0);
|
||||
}
|
||||
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexableField;
|
|||
import org.apache.lucene.index.Term;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.search.TopDocs;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.DocWriteResponse;
|
||||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
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 {
|
||||
//TODO: Enables this test with soft-deletes once we have timestamp
|
||||
Settings settings = Settings.builder().put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), false).build();
|
||||
try (ReplicationGroup shards = createGroup(0, settings)) {
|
||||
try (ReplicationGroup shards = createGroup(0)) {
|
||||
shards.startAll();
|
||||
final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON);
|
||||
indexRequest.onRetry(); // force an update of the timestamp
|
||||
|
|
|
@ -488,9 +488,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
return new RecoveryTarget(indexShard, node, recoveryListener, l -> {
|
||||
}) {
|
||||
@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);
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps, maxSeenAutoIdTimestampOnPrimary);
|
||||
}
|
||||
};
|
||||
});
|
||||
|
@ -557,7 +558,8 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
replica,
|
||||
(indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, l -> {}) {
|
||||
@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 {
|
||||
// index a doc which is not part of the snapshot, but also does not complete on replica
|
||||
replicaEngineFactory.latchIndexers(1);
|
||||
|
@ -585,7 +587,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
} catch (InterruptedException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
|
||||
}
|
||||
});
|
||||
pendingDocActiveWithExtraDocIndexed.await();
|
||||
|
@ -671,11 +673,12 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
}
|
||||
|
||||
@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) {
|
||||
blockIfNeeded(RecoveryState.Stage.TRANSLOG);
|
||||
}
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
return super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -2191,8 +2191,9 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> {
|
||||
}) {
|
||||
@Override
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
|
||||
long maxSeenAutoIdTimestamp) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxSeenAutoIdTimestamp);
|
||||
assertFalse(replica.isSyncNeeded());
|
||||
return localCheckpoint;
|
||||
}
|
||||
|
@ -2298,8 +2299,9 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> {
|
||||
}) {
|
||||
@Override
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
|
||||
long maxAutoIdTimestamp) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
|
||||
// Shard should now be active since we did recover:
|
||||
assertTrue(replica.isActive());
|
||||
return localCheckpoint;
|
||||
|
@ -2345,8 +2347,9 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps);
|
||||
public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps,
|
||||
long maxAutoIdTimestamp) throws IOException {
|
||||
final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps, maxAutoIdTimestamp);
|
||||
assertListenerCalled.accept(replica);
|
||||
return localCheckpoint;
|
||||
}
|
||||
|
|
|
@ -207,11 +207,11 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
public Translog.Operation next() throws IOException {
|
||||
return operations.get(counter++);
|
||||
}
|
||||
});
|
||||
}, randomNonNegativeLong());
|
||||
final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1);
|
||||
assertThat(result.totalOperations, equalTo(expectedOps));
|
||||
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<>();
|
||||
for (List list: shippedOpsCaptor.getAllValues()) {
|
||||
shippedOps.addAll(list);
|
||||
|
@ -249,7 +249,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
} while (op != null && opsToSkip.contains(op));
|
||||
return op;
|
||||
}
|
||||
}));
|
||||
}, randomNonNegativeLong()));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -420,7 +420,8 @@ public class RecoverySourceHandlerTests extends ESTestCase {
|
|||
}
|
||||
|
||||
@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);
|
||||
return SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
}
|
||||
|
|
|
@ -55,6 +55,8 @@ import org.elasticsearch.cluster.routing.ShardRoutingHelper;
|
|||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.TestShardRouting;
|
||||
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.Releasables;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -137,6 +139,17 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
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) {
|
||||
return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(),
|
||||
Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT);
|
||||
|
@ -428,6 +441,12 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
public synchronized void close() throws Exception {
|
||||
if (closed == false) {
|
||||
closed = true;
|
||||
for (IndexShard replica : replicas) {
|
||||
try {
|
||||
assertThat(replica.getMaxSeenAutoIdTimestamp(), equalTo(primary.getMaxSeenAutoIdTimestamp()));
|
||||
} catch (AlreadyClosedException ignored) {
|
||||
}
|
||||
}
|
||||
closeShards(this);
|
||||
} else {
|
||||
throw new AlreadyClosedException("too bad");
|
||||
|
|
Loading…
Reference in New Issue