Advance checkpoints only after persisting ops (#43205)
Local and global checkpoints currently do not correctly reflect what's persisted to disk. The issue is that the local checkpoint is adapted as soon as an operation is processed (but not fsynced yet). This leaves room for the history below the global checkpoint to still change in case of a crash. As we rely on global checkpoints for CCR as well as operation-based recoveries, this has the risk of shard copies / follower clusters going out of sync. This commit required changing some core classes in the system: - The LocalCheckpointTracker keeps track now not only of the information whether an operation has been processed, but also whether that operation has been persisted to disk. - TranslogWriter now keeps track of the sequence numbers that have not been fsynced yet. Once they are fsynced, TranslogWriter notifies LocalCheckpointTracker of this. - ReplicationTracker now keeps track of the persisted local and persisted global checkpoints of all shard copies when in primary mode. The computed global checkpoint (which represents the minimum of all persisted local checkpoints of all in-sync shard copies), which was previously stored in the checkpoint entry for the local shard copy, has been moved to an extra field. - The periodic global checkpoint sync now also takes async durability into account, where the local checkpoints on shards only advance when the translog is asynchronously fsynced. This means that the previous condition to detect inactivity (max sequence number is equal to global checkpoint) is not sufficient anymore. - The new index closing API does not work when combined with async durability. The shard verification step is now requires an additional pre-flight step to fsync the translog, so that the main verify shard step has the most up-to-date global checkpoint at disposition.
This commit is contained in:
parent
adab7eae71
commit
7f8e1454ab
|
@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.indices.close;
|
|||
|
||||
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.admin.indices.flush.FlushRequest;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
|
@ -94,12 +95,12 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) {
|
||||
protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws IOException {
|
||||
executeShardOperation(shardRequest, replica);
|
||||
return new ReplicaResult();
|
||||
}
|
||||
|
||||
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) {
|
||||
private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) throws IOException {
|
||||
final ShardId shardId = indexShard.shardId();
|
||||
if (indexShard.getActiveOperationsCount() != IndexShard.OPERATIONS_BLOCKED) {
|
||||
throw new IllegalStateException("Index shard " + shardId + " is not blocking all operations during closing");
|
||||
|
@ -109,9 +110,19 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
|||
if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), request.clusterBlock()) == false) {
|
||||
throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + request.clusterBlock() + " before closing");
|
||||
}
|
||||
indexShard.verifyShardBeforeIndexClosing();
|
||||
indexShard.flush(new FlushRequest().force(true).waitIfOngoing(true));
|
||||
logger.trace("{} shard is ready for closing", shardId);
|
||||
if (request.isPhase1()) {
|
||||
// in order to advance the global checkpoint to the maximum sequence number, the (persisted) local checkpoint needs to be
|
||||
// advanced first, which, when using async translog syncing, does not automatically hold at the time where we have acquired
|
||||
// all operation permits. Instead, this requires and explicit sync, which communicates the updated (persisted) local checkpoint
|
||||
// to the primary (we call this phase1), and phase2 can then use the fact that the global checkpoint has moved to the maximum
|
||||
// sequence number to pass the verifyShardBeforeIndexClosing check and create a safe commit where the maximum sequence number
|
||||
// is equal to the global checkpoint.
|
||||
indexShard.sync();
|
||||
} else {
|
||||
indexShard.verifyShardBeforeIndexClosing();
|
||||
indexShard.flush(new FlushRequest().force(true).waitIfOngoing(true));
|
||||
logger.trace("{} shard is ready for closing", shardId);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -136,14 +147,22 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
|||
|
||||
private final ClusterBlock clusterBlock;
|
||||
|
||||
private final boolean phase1;
|
||||
|
||||
ShardRequest(StreamInput in) throws IOException {
|
||||
super(in);
|
||||
clusterBlock = new ClusterBlock(in);
|
||||
if (in.getVersion().onOrAfter(Version.V_7_3_0)) {
|
||||
phase1 = in.readBoolean();
|
||||
} else {
|
||||
phase1 = false;
|
||||
}
|
||||
}
|
||||
|
||||
public ShardRequest(final ShardId shardId, final ClusterBlock clusterBlock, final TaskId parentTaskId) {
|
||||
public ShardRequest(final ShardId shardId, final ClusterBlock clusterBlock, final boolean phase1, final TaskId parentTaskId) {
|
||||
super(shardId);
|
||||
this.clusterBlock = Objects.requireNonNull(clusterBlock);
|
||||
this.phase1 = phase1;
|
||||
setParentTask(parentTaskId);
|
||||
}
|
||||
|
||||
|
@ -161,10 +180,17 @@ public class TransportVerifyShardBeforeCloseAction extends TransportReplicationA
|
|||
public void writeTo(final StreamOutput out) throws IOException {
|
||||
super.writeTo(out);
|
||||
clusterBlock.writeTo(out);
|
||||
if (out.getVersion().onOrAfter(Version.V_7_3_0)) {
|
||||
out.writeBoolean(phase1);
|
||||
}
|
||||
}
|
||||
|
||||
public ClusterBlock clusterBlock() {
|
||||
return clusterBlock;
|
||||
}
|
||||
|
||||
public boolean isPhase1() {
|
||||
return phase1;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -111,6 +111,7 @@ public class ReplicationOperation<
|
|||
private void handlePrimaryResult(final PrimaryResultT primaryResult) {
|
||||
this.primaryResult = primaryResult;
|
||||
primary.updateLocalCheckpointForShard(primary.routingEntry().allocationId().getId(), primary.localCheckpoint());
|
||||
primary.updateGlobalCheckpointForShard(primary.routingEntry().allocationId().getId(), primary.globalCheckpoint());
|
||||
final ReplicaRequest replicaRequest = primaryResult.replicaRequest();
|
||||
if (replicaRequest != null) {
|
||||
if (logger.isTraceEnabled()) {
|
||||
|
@ -123,7 +124,7 @@ public class ReplicationOperation<
|
|||
// is valid for this replication group. If we would sample in the reverse, the global checkpoint might be based on a subset
|
||||
// of the sampled replication group, and advanced further than what the given replication group would allow it to.
|
||||
// This would entail that some shards could learn about a global checkpoint that would be higher than its local checkpoint.
|
||||
final long globalCheckpoint = primary.globalCheckpoint();
|
||||
final long globalCheckpoint = primary.computedGlobalCheckpoint();
|
||||
// we have to capture the max_seq_no_of_updates after this request was completed on the primary to make sure the value of
|
||||
// max_seq_no_of_updates on replica when this request is executed is at least the value on the primary when it was executed
|
||||
// on.
|
||||
|
@ -341,16 +342,23 @@ public class ReplicationOperation<
|
|||
void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint);
|
||||
|
||||
/**
|
||||
* Returns the local checkpoint on the primary shard.
|
||||
* Returns the persisted local checkpoint on the primary shard.
|
||||
*
|
||||
* @return the local checkpoint
|
||||
*/
|
||||
long localCheckpoint();
|
||||
|
||||
/**
|
||||
* Returns the global checkpoint on the primary shard.
|
||||
* Returns the global checkpoint computed on the primary shard.
|
||||
*
|
||||
* @return the global checkpoint
|
||||
* @return the computed global checkpoint
|
||||
*/
|
||||
long computedGlobalCheckpoint();
|
||||
|
||||
/**
|
||||
* Returns the persisted global checkpoint on the primary shard.
|
||||
*
|
||||
* @return the persisted global checkpoint
|
||||
*/
|
||||
long globalCheckpoint();
|
||||
|
||||
|
@ -419,16 +427,16 @@ public class ReplicationOperation<
|
|||
public interface ReplicaResponse {
|
||||
|
||||
/**
|
||||
* The local checkpoint for the shard.
|
||||
* The persisted local checkpoint for the shard.
|
||||
*
|
||||
* @return the local checkpoint
|
||||
* @return the persisted local checkpoint
|
||||
**/
|
||||
long localCheckpoint();
|
||||
|
||||
/**
|
||||
* The global checkpoint for the shard.
|
||||
* The persisted global checkpoint for the shard.
|
||||
*
|
||||
* @return the global checkpoint
|
||||
* @return the persisted global checkpoint
|
||||
**/
|
||||
long globalCheckpoint();
|
||||
|
||||
|
|
|
@ -527,7 +527,7 @@ public abstract class TransportReplicationAction<
|
|||
final ReplicaResult replicaResult = shardOperationOnReplica(replicaRequest.getRequest(), replica);
|
||||
releasable.close(); // release shard operation lock before responding to caller
|
||||
final TransportReplicationAction.ReplicaResponse response =
|
||||
new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint());
|
||||
new ReplicaResponse(replica.getLocalCheckpoint(), replica.getLastSyncedGlobalCheckpoint());
|
||||
replicaResult.respond(new ResponseListener(response));
|
||||
} catch (final Exception e) {
|
||||
Releasables.closeWhileHandlingException(releasable); // release shard operation lock before responding to caller
|
||||
|
@ -893,10 +893,6 @@ public abstract class TransportReplicationAction<
|
|||
operationLock.close();
|
||||
}
|
||||
|
||||
public long getLocalCheckpoint() {
|
||||
return indexShard.getLocalCheckpoint();
|
||||
}
|
||||
|
||||
public ShardRouting routingEntry() {
|
||||
return indexShard.routingEntry();
|
||||
}
|
||||
|
@ -944,7 +940,12 @@ public abstract class TransportReplicationAction<
|
|||
|
||||
@Override
|
||||
public long globalCheckpoint() {
|
||||
return indexShard.getGlobalCheckpoint();
|
||||
return indexShard.getLastSyncedGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long computedGlobalCheckpoint() {
|
||||
return indexShard.getLastKnownGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -389,11 +389,26 @@ public class MetaDataIndexStateService {
|
|||
}
|
||||
final TaskId parentTaskId = new TaskId(clusterService.localNode().getId(), request.taskId());
|
||||
final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest =
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, parentTaskId);
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, true, parentTaskId);
|
||||
if (request.ackTimeout() != null) {
|
||||
shardRequest.timeout(request.ackTimeout());
|
||||
}
|
||||
transportVerifyShardBeforeCloseAction.execute(shardRequest, listener);
|
||||
transportVerifyShardBeforeCloseAction.execute(shardRequest, new ActionListener<ReplicationResponse>() {
|
||||
@Override
|
||||
public void onResponse(ReplicationResponse replicationResponse) {
|
||||
final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest =
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, false, parentTaskId);
|
||||
if (request.ackTimeout() != null) {
|
||||
shardRequest.timeout(request.ackTimeout());
|
||||
}
|
||||
transportVerifyShardBeforeCloseAction.execute(shardRequest, listener);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
listener.onFailure(e);
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -793,9 +793,9 @@ public abstract class Engine implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* @return the local checkpoint for this Engine
|
||||
* @return the persisted local checkpoint for this Engine
|
||||
*/
|
||||
public abstract long getLocalCheckpoint();
|
||||
public abstract long getPersistedLocalCheckpoint();
|
||||
|
||||
/**
|
||||
* @return a {@link SeqNoStats} object, using local state and the supplied global checkpoint
|
||||
|
|
|
@ -117,6 +117,7 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.function.BiFunction;
|
||||
import java.util.function.LongConsumer;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
@ -208,7 +209,14 @@ public class InternalEngine extends Engine {
|
|||
throttle = new IndexThrottle();
|
||||
try {
|
||||
trimUnsafeCommits(engineConfig);
|
||||
translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier());
|
||||
translog = openTranslog(engineConfig, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier(),
|
||||
seqNo -> {
|
||||
final LocalCheckpointTracker tracker = getLocalCheckpointTracker();
|
||||
assert tracker != null || getTranslog().isOpen() == false;
|
||||
if (tracker != null) {
|
||||
tracker.markSeqNoAsPersisted(seqNo);
|
||||
}
|
||||
});
|
||||
assert translog.getGeneration() != null;
|
||||
this.translog = translog;
|
||||
this.softDeleteEnabled = engineConfig.getIndexSettings().isSoftDeleteEnabled();
|
||||
|
@ -245,10 +253,10 @@ public class InternalEngine extends Engine {
|
|||
for (ReferenceManager.RefreshListener listener: engineConfig.getInternalRefreshListener()) {
|
||||
this.internalSearcherManager.addListener(listener);
|
||||
}
|
||||
this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getCheckpoint());
|
||||
this.lastRefreshedCheckpointListener = new LastRefreshedCheckpointListener(localCheckpointTracker.getProcessedCheckpoint());
|
||||
this.internalSearcherManager.addListener(lastRefreshedCheckpointListener);
|
||||
maxSeqNoOfUpdatesOrDeletes = new AtomicLong(SequenceNumbers.max(localCheckpointTracker.getMaxSeqNo(), translog.getMaxSeqNo()));
|
||||
if (softDeleteEnabled && localCheckpointTracker.getCheckpoint() < localCheckpointTracker.getMaxSeqNo()) {
|
||||
if (softDeleteEnabled && localCheckpointTracker.getPersistedCheckpoint() < localCheckpointTracker.getMaxSeqNo()) {
|
||||
try (Searcher searcher = acquireSearcher("restore_version_map_and_checkpoint_tracker", SearcherScope.INTERNAL)) {
|
||||
restoreVersionMapAndCheckpointTracker(Lucene.wrapAllDocsLive(searcher.getDirectoryReader()));
|
||||
} catch (IOException e) {
|
||||
|
@ -370,7 +378,7 @@ public class InternalEngine extends Engine {
|
|||
public int restoreLocalHistoryFromTranslog(TranslogRecoveryRunner translogRecoveryRunner) throws IOException {
|
||||
try (ReleasableLock ignored = readLock.acquire()) {
|
||||
ensureOpen();
|
||||
final long localCheckpoint = localCheckpointTracker.getCheckpoint();
|
||||
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||
try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(localCheckpoint + 1)) {
|
||||
return translogRecoveryRunner.run(this, snapshot);
|
||||
}
|
||||
|
@ -381,19 +389,23 @@ public class InternalEngine extends Engine {
|
|||
public int fillSeqNoGaps(long primaryTerm) throws IOException {
|
||||
try (ReleasableLock ignored = writeLock.acquire()) {
|
||||
ensureOpen();
|
||||
final long localCheckpoint = localCheckpointTracker.getCheckpoint();
|
||||
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||
final long maxSeqNo = localCheckpointTracker.getMaxSeqNo();
|
||||
int numNoOpsAdded = 0;
|
||||
for (
|
||||
long seqNo = localCheckpoint + 1;
|
||||
seqNo <= maxSeqNo;
|
||||
seqNo = localCheckpointTracker.getCheckpoint() + 1 /* the local checkpoint might have advanced so we leap-frog */) {
|
||||
seqNo = localCheckpointTracker.getProcessedCheckpoint() + 1 /* leap-frog the local checkpoint */) {
|
||||
innerNoOp(new NoOp(seqNo, primaryTerm, Operation.Origin.PRIMARY, System.nanoTime(), "filling gaps"));
|
||||
numNoOpsAdded++;
|
||||
assert seqNo <= localCheckpointTracker.getCheckpoint()
|
||||
: "local checkpoint did not advance; was [" + seqNo + "], now [" + localCheckpointTracker.getCheckpoint() + "]";
|
||||
assert seqNo <= localCheckpointTracker.getProcessedCheckpoint() :
|
||||
"local checkpoint did not advance; was [" + seqNo + "], now [" + localCheckpointTracker.getProcessedCheckpoint() + "]";
|
||||
|
||||
}
|
||||
syncTranslog(); // to persist noops associated with the advancement of the local checkpoint
|
||||
assert localCheckpointTracker.getPersistedCheckpoint() == maxSeqNo
|
||||
: "persisted local checkpoint did not advance to max seq no; is [" + localCheckpointTracker.getPersistedCheckpoint() +
|
||||
"], max seq no [" + maxSeqNo + "]";
|
||||
return numNoOpsAdded;
|
||||
}
|
||||
}
|
||||
|
@ -471,13 +483,13 @@ public class InternalEngine extends Engine {
|
|||
}
|
||||
|
||||
private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy,
|
||||
LongSupplier globalCheckpointSupplier) throws IOException {
|
||||
LongSupplier globalCheckpointSupplier, LongConsumer persistedSequenceNumberConsumer) throws IOException {
|
||||
|
||||
final TranslogConfig translogConfig = engineConfig.getTranslogConfig();
|
||||
final String translogUUID = loadTranslogUUIDFromLastCommit();
|
||||
// We expect that this shard already exists, so it must already have an existing translog else something is badly wrong!
|
||||
return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier,
|
||||
engineConfig.getPrimaryTermSupplier());
|
||||
engineConfig.getPrimaryTermSupplier(), persistedSequenceNumberConsumer);
|
||||
}
|
||||
|
||||
// Package private for testing purposes only
|
||||
|
@ -711,7 +723,7 @@ public class InternalEngine extends Engine {
|
|||
} else if (op.seqNo() > docAndSeqNo.seqNo) {
|
||||
status = OpVsLuceneDocStatus.OP_NEWER;
|
||||
} else if (op.seqNo() == docAndSeqNo.seqNo) {
|
||||
assert localCheckpointTracker.contains(op.seqNo()) || softDeleteEnabled == false :
|
||||
assert localCheckpointTracker.hasProcessed(op.seqNo()) || softDeleteEnabled == false :
|
||||
"local checkpoint tracker is not updated seq_no=" + op.seqNo() + " id=" + op.id();
|
||||
status = OpVsLuceneDocStatus.OP_STALE_OR_EQUAL;
|
||||
} else {
|
||||
|
@ -914,7 +926,12 @@ public class InternalEngine extends Engine {
|
|||
versionMap.maybePutIndexUnderLock(index.uid().bytes(),
|
||||
new IndexVersionValue(translogLocation, plan.versionForIndexing, index.seqNo(), index.primaryTerm()));
|
||||
}
|
||||
localCheckpointTracker.markSeqNoAsCompleted(indexResult.getSeqNo());
|
||||
localCheckpointTracker.markSeqNoAsProcessed(indexResult.getSeqNo());
|
||||
if (indexResult.getTranslogLocation() == null) {
|
||||
// the op is coming from the translog (and is hence persisted already) or it does not have a sequence number
|
||||
assert index.origin().isFromTranslog() || indexResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
localCheckpointTracker.markSeqNoAsPersisted(indexResult.getSeqNo());
|
||||
}
|
||||
indexResult.setTook(System.nanoTime() - index.startTime());
|
||||
indexResult.freeze();
|
||||
return indexResult;
|
||||
|
@ -954,7 +971,7 @@ public class InternalEngine extends Engine {
|
|||
// unlike the primary, replicas don't really care to about creation status of documents
|
||||
// this allows to ignore the case where a document was found in the live version maps in
|
||||
// a delete state and return false for the created flag in favor of code simplicity
|
||||
if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){
|
||||
if (index.seqNo() <= localCheckpointTracker.getProcessedCheckpoint()){
|
||||
// the operation seq# is lower then the current local checkpoint and thus was already put into lucene
|
||||
// this can happen during recovery where older operations are sent from the translog that are already
|
||||
// part of the lucene commit (either from a peer recovery or a local translog)
|
||||
|
@ -1267,7 +1284,12 @@ public class InternalEngine extends Engine {
|
|||
final Translog.Location location = translog.add(new Translog.Delete(delete, deleteResult));
|
||||
deleteResult.setTranslogLocation(location);
|
||||
}
|
||||
localCheckpointTracker.markSeqNoAsCompleted(deleteResult.getSeqNo());
|
||||
localCheckpointTracker.markSeqNoAsProcessed(deleteResult.getSeqNo());
|
||||
if (deleteResult.getTranslogLocation() == null) {
|
||||
// the op is coming from the translog (and is hence persisted already) or does not have a sequence number (version conflict)
|
||||
assert delete.origin().isFromTranslog() || deleteResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
localCheckpointTracker.markSeqNoAsPersisted(deleteResult.getSeqNo());
|
||||
}
|
||||
deleteResult.setTook(System.nanoTime() - delete.startTime());
|
||||
deleteResult.freeze();
|
||||
} catch (RuntimeException | IOException e) {
|
||||
|
@ -1300,7 +1322,7 @@ public class InternalEngine extends Engine {
|
|||
// this allows to ignore the case where a document was found in the live version maps in
|
||||
// a delete state and return true for the found flag in favor of code simplicity
|
||||
final DeletionStrategy plan;
|
||||
if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) {
|
||||
if (delete.seqNo() <= localCheckpointTracker.getProcessedCheckpoint()) {
|
||||
// the operation seq# is lower then the current local checkpoint and thus was already put into lucene
|
||||
// this can happen during recovery where older operations are sent from the translog that are already
|
||||
// part of the lucene commit (either from a peer recovery or a local translog)
|
||||
|
@ -1474,10 +1496,10 @@ public class InternalEngine extends Engine {
|
|||
try (Releasable ignored = noOpKeyedLock.acquire(seqNo)) {
|
||||
final NoOpResult noOpResult;
|
||||
final Optional<Exception> preFlightError = preFlightCheckForNoOp(noOp);
|
||||
Exception failure = null;
|
||||
if (preFlightError.isPresent()) {
|
||||
noOpResult = new NoOpResult(getPrimaryTerm(), noOp.seqNo(), preFlightError.get());
|
||||
noOpResult = new NoOpResult(getPrimaryTerm(), SequenceNumbers.UNASSIGNED_SEQ_NO, preFlightError.get());
|
||||
} else {
|
||||
Exception failure = null;
|
||||
markSeqNoAsSeen(noOp.seqNo());
|
||||
if (softDeleteEnabled) {
|
||||
try {
|
||||
|
@ -1510,7 +1532,14 @@ public class InternalEngine extends Engine {
|
|||
noOpResult.setTranslogLocation(location);
|
||||
}
|
||||
}
|
||||
localCheckpointTracker.markSeqNoAsCompleted(seqNo);
|
||||
localCheckpointTracker.markSeqNoAsProcessed(noOpResult.getSeqNo());
|
||||
if (noOpResult.getTranslogLocation() == null) {
|
||||
// the op is coming from the translog (and is hence persisted already) or it does not have a sequence number, or we failed
|
||||
// to add a tombstone doc to Lucene with a non-fatal error, which would be very surprising
|
||||
// TODO: always fail the engine in the last case, as this creates gaps in the history
|
||||
assert noOp.origin().isFromTranslog() || noOpResult.getSeqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO || failure != null;
|
||||
localCheckpointTracker.markSeqNoAsPersisted(noOpResult.getSeqNo());
|
||||
}
|
||||
noOpResult.setTook(System.nanoTime() - noOp.startTime());
|
||||
noOpResult.freeze();
|
||||
return noOpResult;
|
||||
|
@ -1540,7 +1569,7 @@ public class InternalEngine extends Engine {
|
|||
// since it flushes the index as well (though, in terms of concurrency, we are allowed to do it)
|
||||
// both refresh types will result in an internal refresh but only the external will also
|
||||
// pass the new reader reference to the external reader manager.
|
||||
final long localCheckpointBeforeRefresh = getLocalCheckpoint();
|
||||
final long localCheckpointBeforeRefresh = localCheckpointTracker.getProcessedCheckpoint();
|
||||
boolean refreshed;
|
||||
try (ReleasableLock lock = readLock.acquire()) {
|
||||
ensureOpen();
|
||||
|
@ -1682,9 +1711,9 @@ public class InternalEngine extends Engine {
|
|||
* This method is to maintain translog only, thus IndexWriter#hasUncommittedChanges condition is not considered.
|
||||
*/
|
||||
final long translogGenerationOfNewCommit =
|
||||
translog.getMinGenerationForSeqNo(localCheckpointTracker.getCheckpoint() + 1).translogFileGeneration;
|
||||
translog.getMinGenerationForSeqNo(localCheckpointTracker.getProcessedCheckpoint() + 1).translogFileGeneration;
|
||||
return translogGenerationOfLastCommit < translogGenerationOfNewCommit
|
||||
|| localCheckpointTracker.getCheckpoint() == localCheckpointTracker.getMaxSeqNo();
|
||||
|| localCheckpointTracker.getProcessedCheckpoint() == localCheckpointTracker.getMaxSeqNo();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1871,7 +1900,7 @@ public class InternalEngine extends Engine {
|
|||
*/
|
||||
final long timeMSec = engineConfig.getThreadPool().relativeTimeInMillis();
|
||||
final long maxTimestampToPrune = timeMSec - engineConfig.getIndexSettings().getGcDeletesInMillis();
|
||||
versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getCheckpoint());
|
||||
versionMap.pruneTombstones(maxTimestampToPrune, localCheckpointTracker.getProcessedCheckpoint());
|
||||
lastDeleteVersionPruneTimeMSec = timeMSec;
|
||||
}
|
||||
|
||||
|
@ -2361,7 +2390,7 @@ public class InternalEngine extends Engine {
|
|||
protected void commitIndexWriter(final IndexWriter writer, final Translog translog, @Nullable final String syncId) throws IOException {
|
||||
ensureCanFlush();
|
||||
try {
|
||||
final long localCheckpoint = localCheckpointTracker.getCheckpoint();
|
||||
final long localCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||
final Translog.TranslogGeneration translogGeneration = translog.getMinGenerationForSeqNo(localCheckpoint + 1);
|
||||
final String translogFileGeneration = Long.toString(translogGeneration.translogFileGeneration);
|
||||
final String translogUUID = translogGeneration.translogUUID;
|
||||
|
@ -2452,7 +2481,6 @@ public class InternalEngine extends Engine {
|
|||
return mergeScheduler.stats();
|
||||
}
|
||||
|
||||
// Used only for testing! Package private to prevent anyone else from using it
|
||||
LocalCheckpointTracker getLocalCheckpointTracker() {
|
||||
return localCheckpointTracker;
|
||||
}
|
||||
|
@ -2462,9 +2490,13 @@ public class InternalEngine extends Engine {
|
|||
return getTranslog().getLastSyncedGlobalCheckpoint();
|
||||
}
|
||||
|
||||
public long getProcessedLocalCheckpoint() {
|
||||
return localCheckpointTracker.getProcessedCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLocalCheckpoint() {
|
||||
return localCheckpointTracker.getCheckpoint();
|
||||
public long getPersistedLocalCheckpoint() {
|
||||
return localCheckpointTracker.getPersistedCheckpoint();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2487,7 +2519,7 @@ public class InternalEngine extends Engine {
|
|||
assert versionMap.assertKeyedLockHeldByCurrentThread(op.uid().bytes());
|
||||
}
|
||||
}
|
||||
return localCheckpointTracker.contains(op.seqNo());
|
||||
return localCheckpointTracker.hasProcessed(op.seqNo());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2577,7 +2609,7 @@ public class InternalEngine extends Engine {
|
|||
|
||||
@Override
|
||||
public boolean hasCompleteOperationHistory(String source, MapperService mapperService, long startingSeqNo) throws IOException {
|
||||
final long currentLocalCheckpoint = getLocalCheckpointTracker().getCheckpoint();
|
||||
final long currentLocalCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||
// avoid scanning translog if not necessary
|
||||
if (startingSeqNo > currentLocalCheckpoint) {
|
||||
return true;
|
||||
|
@ -2587,11 +2619,11 @@ public class InternalEngine extends Engine {
|
|||
Translog.Operation operation;
|
||||
while ((operation = snapshot.next()) != null) {
|
||||
if (operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) {
|
||||
tracker.markSeqNoAsCompleted(operation.seqNo());
|
||||
tracker.markSeqNoAsProcessed(operation.seqNo());
|
||||
}
|
||||
}
|
||||
}
|
||||
return tracker.getCheckpoint() >= currentLocalCheckpoint;
|
||||
return tracker.getProcessedCheckpoint() >= currentLocalCheckpoint;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2707,7 +2739,7 @@ public class InternalEngine extends Engine {
|
|||
@Override
|
||||
public void beforeRefresh() {
|
||||
// all changes until this point should be visible after refresh
|
||||
pendingCheckpoint = localCheckpointTracker.getCheckpoint();
|
||||
pendingCheckpoint = localCheckpointTracker.getProcessedCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -2768,7 +2800,7 @@ public class InternalEngine extends Engine {
|
|||
// Operations can be processed on a replica in a different order than on the primary. If the order on the primary is index-1,
|
||||
// delete-2, index-3, and the order on a replica is index-1, index-3, delete-2, then the msu of index-3 on the replica is 2
|
||||
// even though it is an update (overwrites index-1). We should relax this assertion if there is a pending gap in the seq_no.
|
||||
if (relaxIfGapInSeqNo && getLocalCheckpoint() < maxSeqNoOfUpdates) {
|
||||
if (relaxIfGapInSeqNo && localCheckpointTracker.getProcessedCheckpoint() < maxSeqNoOfUpdates) {
|
||||
return true;
|
||||
}
|
||||
assert seqNo <= maxSeqNoOfUpdates : "id=" + id + " seq_no=" + seqNo + " msu=" + maxSeqNoOfUpdates;
|
||||
|
@ -2792,7 +2824,7 @@ public class InternalEngine extends Engine {
|
|||
private void restoreVersionMapAndCheckpointTracker(DirectoryReader directoryReader) throws IOException {
|
||||
final IndexSearcher searcher = new IndexSearcher(directoryReader);
|
||||
searcher.setQueryCache(null);
|
||||
final Query query = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getLocalCheckpoint() + 1, Long.MAX_VALUE);
|
||||
final Query query = LongPoint.newRangeQuery(SeqNoFieldMapper.NAME, getPersistedLocalCheckpoint() + 1, Long.MAX_VALUE);
|
||||
final Weight weight = searcher.createWeight(query, ScoreMode.COMPLETE_NO_SCORES, 1.0f);
|
||||
for (LeafReaderContext leaf : directoryReader.leaves()) {
|
||||
final Scorer scorer = weight.scorer(leaf);
|
||||
|
@ -2809,7 +2841,8 @@ public class InternalEngine extends Engine {
|
|||
continue; // skip children docs which do not have primary term
|
||||
}
|
||||
final long seqNo = dv.docSeqNo(docId);
|
||||
localCheckpointTracker.markSeqNoAsCompleted(seqNo);
|
||||
localCheckpointTracker.markSeqNoAsProcessed(seqNo);
|
||||
localCheckpointTracker.markSeqNoAsPersisted(seqNo);
|
||||
idFieldVisitor.reset();
|
||||
leaf.reader().document(docId, idFieldVisitor);
|
||||
if (idFieldVisitor.getId() == null) {
|
||||
|
|
|
@ -329,7 +329,7 @@ public class ReadOnlyEngine extends Engine {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getLocalCheckpoint() {
|
||||
public long getPersistedLocalCheckpoint() {
|
||||
return seqNoStats.getLocalCheckpoint();
|
||||
}
|
||||
|
||||
|
|
|
@ -118,7 +118,7 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<
|
|||
|
||||
private void maybeSyncTranslog(final IndexShard indexShard) throws IOException {
|
||||
if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST &&
|
||||
indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getGlobalCheckpoint()) {
|
||||
indexShard.getLastSyncedGlobalCheckpoint() < indexShard.getLastKnownGlobalCheckpoint()) {
|
||||
indexShard.sync();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,8 @@ package org.elasticsearch.index.seqno;
|
|||
import com.carrotsearch.hppc.LongObjectHashMap;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
/**
|
||||
* This class generates sequences numbers and keeps track of the so-called "local checkpoint" which is the highest number for which all
|
||||
* previous sequence numbers have been processed (inclusive).
|
||||
|
@ -35,20 +37,31 @@ public class LocalCheckpointTracker {
|
|||
static final short BIT_SET_SIZE = 1024;
|
||||
|
||||
/**
|
||||
* A collection of bit sets representing pending sequence numbers. Each sequence number is mapped to a bit set by dividing by the
|
||||
* A collection of bit sets representing processed sequence numbers. Each sequence number is mapped to a bit set by dividing by the
|
||||
* bit set size.
|
||||
*/
|
||||
final LongObjectHashMap<CountedBitSet> processedSeqNo = new LongObjectHashMap<>();
|
||||
|
||||
/**
|
||||
* The current local checkpoint, i.e., all sequence numbers no more than this number have been completed.
|
||||
* A collection of bit sets representing durably persisted sequence numbers. Each sequence number is mapped to a bit set by dividing by
|
||||
* the bit set size.
|
||||
*/
|
||||
volatile long checkpoint;
|
||||
final LongObjectHashMap<CountedBitSet> persistedSeqNo = new LongObjectHashMap<>();
|
||||
|
||||
/**
|
||||
* The current local checkpoint, i.e., all sequence numbers no more than this number have been processed.
|
||||
*/
|
||||
final AtomicLong processedCheckpoint = new AtomicLong();
|
||||
|
||||
/**
|
||||
* The current persisted local checkpoint, i.e., all sequence numbers no more than this number have been durably persisted.
|
||||
*/
|
||||
final AtomicLong persistedCheckpoint = new AtomicLong();
|
||||
|
||||
/**
|
||||
* The next available sequence number.
|
||||
*/
|
||||
private volatile long nextSeqNo;
|
||||
final AtomicLong nextSeqNo = new AtomicLong();
|
||||
|
||||
/**
|
||||
* Initialize the local checkpoint service. The {@code maxSeqNo} should be set to the last sequence number assigned, or
|
||||
|
@ -68,8 +81,9 @@ public class LocalCheckpointTracker {
|
|||
throw new IllegalArgumentException(
|
||||
"max seq. no. must be non-negative or [" + SequenceNumbers.NO_OPS_PERFORMED + "] but was [" + maxSeqNo + "]");
|
||||
}
|
||||
nextSeqNo = maxSeqNo == SequenceNumbers.NO_OPS_PERFORMED ? 0 : maxSeqNo + 1;
|
||||
checkpoint = localCheckpoint;
|
||||
nextSeqNo.set(maxSeqNo + 1);
|
||||
processedCheckpoint.set(localCheckpoint);
|
||||
persistedCheckpoint.set(localCheckpoint);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -77,48 +91,67 @@ public class LocalCheckpointTracker {
|
|||
*
|
||||
* @return the next assigned sequence number
|
||||
*/
|
||||
public synchronized long generateSeqNo() {
|
||||
return nextSeqNo++;
|
||||
public long generateSeqNo() {
|
||||
return nextSeqNo.getAndIncrement();
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks the provided sequence number as seen and updates the max_seq_no if needed.
|
||||
*/
|
||||
public synchronized void advanceMaxSeqNo(long seqNo) {
|
||||
if (seqNo >= nextSeqNo) {
|
||||
nextSeqNo = seqNo + 1;
|
||||
}
|
||||
public void advanceMaxSeqNo(final long seqNo) {
|
||||
nextSeqNo.accumulateAndGet(seqNo + 1, Math::max);
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks the processing of the provided sequence number as completed as updates the checkpoint if possible.
|
||||
* Marks the provided sequence number as processed and updates the processed checkpoint if possible.
|
||||
*
|
||||
* @param seqNo the sequence number to mark as completed
|
||||
* @param seqNo the sequence number to mark as processed
|
||||
*/
|
||||
public synchronized void markSeqNoAsCompleted(final long seqNo) {
|
||||
public synchronized void markSeqNoAsProcessed(final long seqNo) {
|
||||
markSeqNo(seqNo, processedCheckpoint, processedSeqNo);
|
||||
}
|
||||
|
||||
/**
|
||||
* Marks the provided sequence number as persisted and updates the checkpoint if possible.
|
||||
*
|
||||
* @param seqNo the sequence number to mark as persisted
|
||||
*/
|
||||
public synchronized void markSeqNoAsPersisted(final long seqNo) {
|
||||
markSeqNo(seqNo, persistedCheckpoint, persistedSeqNo);
|
||||
}
|
||||
|
||||
private void markSeqNo(final long seqNo, final AtomicLong checkPoint, final LongObjectHashMap<CountedBitSet> bitSetMap) {
|
||||
assert Thread.holdsLock(this);
|
||||
// make sure we track highest seen sequence number
|
||||
if (seqNo >= nextSeqNo) {
|
||||
nextSeqNo = seqNo + 1;
|
||||
}
|
||||
if (seqNo <= checkpoint) {
|
||||
advanceMaxSeqNo(seqNo);
|
||||
if (seqNo <= checkPoint.get()) {
|
||||
// this is possible during recovery where we might replay an operation that was also replicated
|
||||
return;
|
||||
}
|
||||
final CountedBitSet bitSet = getBitSetForSeqNo(seqNo);
|
||||
final CountedBitSet bitSet = getBitSetForSeqNo(bitSetMap, seqNo);
|
||||
final int offset = seqNoToBitSetOffset(seqNo);
|
||||
bitSet.set(offset);
|
||||
if (seqNo == checkpoint + 1) {
|
||||
updateCheckpoint();
|
||||
if (seqNo == checkPoint.get() + 1) {
|
||||
updateCheckpoint(checkPoint, bitSetMap);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The current checkpoint which can be advanced by {@link #markSeqNoAsCompleted(long)}.
|
||||
* The current checkpoint which can be advanced by {@link #markSeqNoAsProcessed(long)}.
|
||||
*
|
||||
* @return the current checkpoint
|
||||
*/
|
||||
public long getCheckpoint() {
|
||||
return checkpoint;
|
||||
public long getProcessedCheckpoint() {
|
||||
return processedCheckpoint.get();
|
||||
}
|
||||
|
||||
/**
|
||||
* The current persisted checkpoint which can be advanced by {@link #markSeqNoAsPersisted(long)}.
|
||||
*
|
||||
* @return the current persisted checkpoint
|
||||
*/
|
||||
public long getPersistedCheckpoint() {
|
||||
return persistedCheckpoint.get();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -127,17 +160,17 @@ public class LocalCheckpointTracker {
|
|||
* @return the maximum sequence number
|
||||
*/
|
||||
public long getMaxSeqNo() {
|
||||
return nextSeqNo - 1;
|
||||
return nextSeqNo.get() - 1;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* constructs a {@link SeqNoStats} object, using local state and the supplied global checkpoint
|
||||
*
|
||||
* This is needed to make sure the local checkpoint and max seq no are consistent
|
||||
* This is needed to make sure the persisted local checkpoint and max seq no are consistent
|
||||
*/
|
||||
public synchronized SeqNoStats getStats(final long globalCheckpoint) {
|
||||
return new SeqNoStats(getMaxSeqNo(), getCheckpoint(), globalCheckpoint);
|
||||
return new SeqNoStats(getMaxSeqNo(), getPersistedCheckpoint(), globalCheckpoint);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -147,70 +180,74 @@ public class LocalCheckpointTracker {
|
|||
* @throws InterruptedException if the thread was interrupted while blocking on the condition
|
||||
*/
|
||||
@SuppressForbidden(reason = "Object#wait")
|
||||
public synchronized void waitForOpsToComplete(final long seqNo) throws InterruptedException {
|
||||
while (checkpoint < seqNo) {
|
||||
public synchronized void waitForProcessedOpsToComplete(final long seqNo) throws InterruptedException {
|
||||
while (processedCheckpoint.get() < seqNo) {
|
||||
// notified by updateCheckpoint
|
||||
this.wait();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks if the given sequence number was marked as completed in this tracker.
|
||||
* Checks if the given sequence number was marked as processed in this tracker.
|
||||
*/
|
||||
public boolean contains(final long seqNo) {
|
||||
public boolean hasProcessed(final long seqNo) {
|
||||
assert seqNo >= 0 : "invalid seq_no=" + seqNo;
|
||||
if (seqNo >= nextSeqNo) {
|
||||
if (seqNo >= nextSeqNo.get()) {
|
||||
return false;
|
||||
}
|
||||
if (seqNo <= checkpoint) {
|
||||
if (seqNo <= processedCheckpoint.get()) {
|
||||
return true;
|
||||
}
|
||||
final long bitSetKey = getBitSetKey(seqNo);
|
||||
final int bitSetOffset = seqNoToBitSetOffset(seqNo);
|
||||
synchronized (this) {
|
||||
// check again under lock
|
||||
if (seqNo <= processedCheckpoint.get()) {
|
||||
return true;
|
||||
}
|
||||
final CountedBitSet bitSet = processedSeqNo.get(bitSetKey);
|
||||
return bitSet != null && bitSet.get(bitSetOffset);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Moves the checkpoint to the last consecutively processed sequence number. This method assumes that the sequence number following the
|
||||
* current checkpoint is processed.
|
||||
* Moves the checkpoint to the last consecutively processed sequence number. This method assumes that the sequence number
|
||||
* following the current checkpoint is processed.
|
||||
*/
|
||||
@SuppressForbidden(reason = "Object#notifyAll")
|
||||
private void updateCheckpoint() {
|
||||
private void updateCheckpoint(AtomicLong checkPoint, LongObjectHashMap<CountedBitSet> bitSetMap) {
|
||||
assert Thread.holdsLock(this);
|
||||
assert getBitSetForSeqNo(checkpoint + 1).get(seqNoToBitSetOffset(checkpoint + 1)) :
|
||||
assert getBitSetForSeqNo(bitSetMap, checkPoint.get() + 1).get(seqNoToBitSetOffset(checkPoint.get() + 1)) :
|
||||
"updateCheckpoint is called but the bit following the checkpoint is not set";
|
||||
try {
|
||||
// keep it simple for now, get the checkpoint one by one; in the future we can optimize and read words
|
||||
long bitSetKey = getBitSetKey(checkpoint);
|
||||
CountedBitSet current = processedSeqNo.get(bitSetKey);
|
||||
long bitSetKey = getBitSetKey(checkPoint.get());
|
||||
CountedBitSet current = bitSetMap.get(bitSetKey);
|
||||
if (current == null) {
|
||||
// the bit set corresponding to the checkpoint has already been removed, set ourselves up for the next bit set
|
||||
assert checkpoint % BIT_SET_SIZE == BIT_SET_SIZE - 1;
|
||||
current = processedSeqNo.get(++bitSetKey);
|
||||
assert checkPoint.get() % BIT_SET_SIZE == BIT_SET_SIZE - 1;
|
||||
current = bitSetMap.get(++bitSetKey);
|
||||
}
|
||||
do {
|
||||
checkpoint++;
|
||||
checkPoint.incrementAndGet();
|
||||
/*
|
||||
* The checkpoint always falls in the current bit set or we have already cleaned it; if it falls on the last bit of the
|
||||
* current bit set, we can clean it.
|
||||
*/
|
||||
if (checkpoint == lastSeqNoInBitSet(bitSetKey)) {
|
||||
if (checkPoint.get() == lastSeqNoInBitSet(bitSetKey)) {
|
||||
assert current != null;
|
||||
final CountedBitSet removed = processedSeqNo.remove(bitSetKey);
|
||||
final CountedBitSet removed = bitSetMap.remove(bitSetKey);
|
||||
assert removed == current;
|
||||
current = processedSeqNo.get(++bitSetKey);
|
||||
current = bitSetMap.get(++bitSetKey);
|
||||
}
|
||||
} while (current != null && current.get(seqNoToBitSetOffset(checkpoint + 1)));
|
||||
} while (current != null && current.get(seqNoToBitSetOffset(checkPoint.get() + 1)));
|
||||
} finally {
|
||||
// notifies waiters in waitForOpsToComplete
|
||||
// notifies waiters in waitForProcessedOpsToComplete
|
||||
this.notifyAll();
|
||||
}
|
||||
}
|
||||
|
||||
private long lastSeqNoInBitSet(final long bitSetKey) {
|
||||
private static long lastSeqNoInBitSet(final long bitSetKey) {
|
||||
return (1 + bitSetKey) * BIT_SET_SIZE - 1;
|
||||
}
|
||||
|
||||
|
@ -220,32 +257,32 @@ public class LocalCheckpointTracker {
|
|||
* @param seqNo the sequence number to obtain the bit set for
|
||||
* @return the bit set corresponding to the provided sequence number
|
||||
*/
|
||||
private long getBitSetKey(final long seqNo) {
|
||||
private static long getBitSetKey(final long seqNo) {
|
||||
return seqNo / BIT_SET_SIZE;
|
||||
}
|
||||
|
||||
private CountedBitSet getBitSetForSeqNo(final long seqNo) {
|
||||
private CountedBitSet getBitSetForSeqNo(final LongObjectHashMap<CountedBitSet> bitSetMap, final long seqNo) {
|
||||
assert Thread.holdsLock(this);
|
||||
final long bitSetKey = getBitSetKey(seqNo);
|
||||
final int index = processedSeqNo.indexOf(bitSetKey);
|
||||
final int index = bitSetMap.indexOf(bitSetKey);
|
||||
final CountedBitSet bitSet;
|
||||
if (processedSeqNo.indexExists(index)) {
|
||||
bitSet = processedSeqNo.indexGet(index);
|
||||
if (bitSetMap.indexExists(index)) {
|
||||
bitSet = bitSetMap.indexGet(index);
|
||||
} else {
|
||||
bitSet = new CountedBitSet(BIT_SET_SIZE);
|
||||
processedSeqNo.indexInsert(index, bitSetKey, bitSet);
|
||||
bitSetMap.indexInsert(index, bitSetKey, bitSet);
|
||||
}
|
||||
return bitSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain the position in the bit set corresponding to the provided sequence number. The bit set corresponding to the sequence number
|
||||
* can be obtained via {@link #getBitSetForSeqNo(long)}.
|
||||
* can be obtained via {@link #getBitSetForSeqNo(LongObjectHashMap, long)}.
|
||||
*
|
||||
* @param seqNo the sequence number to obtain the position for
|
||||
* @return the position in the bit set corresponding to the provided sequence number
|
||||
*/
|
||||
private int seqNoToBitSetOffset(final long seqNo) {
|
||||
private static int seqNoToBitSetOffset(final long seqNo) {
|
||||
return Math.toIntExact(seqNo % BIT_SET_SIZE);
|
||||
}
|
||||
|
||||
|
|
|
@ -146,9 +146,15 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
final Map<String, CheckpointState> checkpoints;
|
||||
|
||||
/**
|
||||
* A callback invoked when the global checkpoint is updated. For primary mode this occurs if the computed global checkpoint advances on
|
||||
* the basis of state changes tracked here. For non-primary mode this occurs if the local knowledge of the global checkpoint advances
|
||||
* due to an update from the primary.
|
||||
* The current in-memory global checkpoint. In primary mode, this is a cached version of the checkpoint computed from the local
|
||||
* checkpoints. In replica mode, this is the in-memory global checkpoint that's communicated by the primary.
|
||||
*/
|
||||
volatile long globalCheckpoint;
|
||||
|
||||
/**
|
||||
* A callback invoked when the in-memory global checkpoint is updated. For primary mode this occurs if the computed global checkpoint
|
||||
* advances on the basis of state changes tracked here. For non-primary mode this occurs if the local knowledge of the global checkpoint
|
||||
* advances due to an update from the primary.
|
||||
*/
|
||||
private final LongConsumer onGlobalCheckpointUpdated;
|
||||
|
||||
|
@ -393,13 +399,13 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
public static class CheckpointState implements Writeable {
|
||||
|
||||
/**
|
||||
* the last local checkpoint information that we have for this shard
|
||||
* the last local checkpoint information that we have for this shard. All operations up to this point are properly fsynced to disk.
|
||||
*/
|
||||
long localCheckpoint;
|
||||
|
||||
/**
|
||||
* the last global checkpoint information that we have for this shard. This information is computed for the primary if
|
||||
* the tracker is in primary mode and received from the primary if in replica mode.
|
||||
* the last global checkpoint information that we have for this shard. This is the global checkpoint that's fsynced to disk on the
|
||||
* respective shard, and all operations up to this point are properly fsynced to disk as well.
|
||||
*/
|
||||
long globalCheckpoint;
|
||||
/**
|
||||
|
@ -494,9 +500,9 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the local knowledge of the global checkpoints for all in-sync allocation IDs.
|
||||
* Get the local knowledge of the persisted global checkpoints for all in-sync allocation IDs.
|
||||
*
|
||||
* @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID
|
||||
* @return a map from allocation ID to the local knowledge of the persisted global checkpoint for that allocation ID
|
||||
*/
|
||||
public synchronized ObjectLongMap<String> getInSyncGlobalCheckpoints() {
|
||||
assert primaryMode;
|
||||
|
@ -549,20 +555,11 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
* as a logical operator, many of the invariants are written under the form (!A || B), they should be read as (A implies B) however.
|
||||
*/
|
||||
private boolean invariant() {
|
||||
assert checkpoints.get(shardAllocationId) != null :
|
||||
"checkpoints map should always have an entry for the current shard";
|
||||
|
||||
// local checkpoints only set during primary mode
|
||||
assert primaryMode || checkpoints.values().stream().allMatch(lcps -> lcps.localCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
|
||||
// global checkpoints for other shards only set during primary mode
|
||||
assert primaryMode
|
||||
|| checkpoints
|
||||
.entrySet()
|
||||
.stream()
|
||||
.filter(e -> e.getKey().equals(shardAllocationId) == false)
|
||||
.map(Map.Entry::getValue)
|
||||
.allMatch(cps -> cps.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
// global checkpoints only set during primary mode
|
||||
assert primaryMode || checkpoints.values().stream().allMatch(cps -> cps.globalCheckpoint == SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
|
||||
// relocation handoff can only occur in primary mode
|
||||
assert !handoffInProgress || primaryMode;
|
||||
|
@ -591,14 +588,14 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
|
||||
// the computed global checkpoint is always up-to-date
|
||||
assert !primaryMode
|
||||
|| getGlobalCheckpoint() == computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint())
|
||||
|| globalCheckpoint == computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint)
|
||||
: "global checkpoint is not up-to-date, expected: " +
|
||||
computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint()) + " but was: " + getGlobalCheckpoint();
|
||||
computeGlobalCheckpoint(pendingInSync, checkpoints.values(), globalCheckpoint) + " but was: " + globalCheckpoint;
|
||||
|
||||
// when in primary mode, the global checkpoint is at most the minimum local checkpoint on all in-sync shard copies
|
||||
assert !primaryMode
|
||||
|| getGlobalCheckpoint() <= inSyncCheckpointStates(checkpoints, CheckpointState::getLocalCheckpoint, LongStream::min)
|
||||
: "global checkpoint [" + getGlobalCheckpoint() + "] "
|
||||
|| globalCheckpoint <= inSyncCheckpointStates(checkpoints, CheckpointState::getLocalCheckpoint, LongStream::min)
|
||||
: "global checkpoint [" + globalCheckpoint + "] "
|
||||
+ "for primary mode allocation ID [" + shardAllocationId + "] "
|
||||
+ "more than in-sync local checkpoints [" + checkpoints + "]";
|
||||
|
||||
|
@ -672,8 +669,8 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
this.operationPrimaryTerm = operationPrimaryTerm;
|
||||
this.handoffInProgress = false;
|
||||
this.appliedClusterStateVersion = -1L;
|
||||
this.globalCheckpoint = globalCheckpoint;
|
||||
this.checkpoints = new HashMap<>(1 + indexSettings.getNumberOfReplicas());
|
||||
checkpoints.put(allocationId, new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, false, false));
|
||||
this.onGlobalCheckpointUpdated = Objects.requireNonNull(onGlobalCheckpointUpdated);
|
||||
this.currentTimeMillisSupplier = Objects.requireNonNull(currentTimeMillisSupplier);
|
||||
this.onSyncRetentionLeases = Objects.requireNonNull(onSyncRetentionLeases);
|
||||
|
@ -700,28 +697,26 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the global checkpoint for the shard.
|
||||
* Returns the in-memory global checkpoint for the shard.
|
||||
*
|
||||
* @return the global checkpoint
|
||||
*/
|
||||
public synchronized long getGlobalCheckpoint() {
|
||||
final CheckpointState cps = checkpoints.get(shardAllocationId);
|
||||
assert cps != null;
|
||||
return cps.globalCheckpoint;
|
||||
public long getGlobalCheckpoint() {
|
||||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getAsLong() {
|
||||
return getGlobalCheckpoint();
|
||||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
/**
|
||||
* Updates the global checkpoint on a replica shard after it has been updated by the primary.
|
||||
*
|
||||
* @param globalCheckpoint the global checkpoint
|
||||
* @param reason the reason the global checkpoint was updated
|
||||
* @param newGlobalCheckpoint the new global checkpoint
|
||||
* @param reason the reason the global checkpoint was updated
|
||||
*/
|
||||
public synchronized void updateGlobalCheckpointOnReplica(final long globalCheckpoint, final String reason) {
|
||||
public synchronized void updateGlobalCheckpointOnReplica(final long newGlobalCheckpoint, final String reason) {
|
||||
assert invariant();
|
||||
assert primaryMode == false;
|
||||
/*
|
||||
|
@ -730,18 +725,17 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
* replica shards). In these cases, the local knowledge of the global checkpoint could be higher than the sync from the lagging
|
||||
* primary.
|
||||
*/
|
||||
updateGlobalCheckpoint(
|
||||
shardAllocationId,
|
||||
globalCheckpoint,
|
||||
current -> {
|
||||
logger.trace("updated global checkpoint from [{}] to [{}] due to [{}]", current, globalCheckpoint, reason);
|
||||
onGlobalCheckpointUpdated.accept(globalCheckpoint);
|
||||
});
|
||||
final long previousGlobalCheckpoint = globalCheckpoint;
|
||||
if (newGlobalCheckpoint > previousGlobalCheckpoint) {
|
||||
globalCheckpoint = newGlobalCheckpoint;
|
||||
logger.trace("updated global checkpoint from [{}] to [{}] due to [{}]", previousGlobalCheckpoint, globalCheckpoint, reason);
|
||||
onGlobalCheckpointUpdated.accept(globalCheckpoint);
|
||||
}
|
||||
assert invariant();
|
||||
}
|
||||
|
||||
/**
|
||||
* Update the local knowledge of the global checkpoint for the specified allocation ID.
|
||||
* Update the local knowledge of the persisted global checkpoint for the specified allocation ID.
|
||||
*
|
||||
* @param allocationId the allocation ID to update the global checkpoint for
|
||||
* @param globalCheckpoint the global checkpoint
|
||||
|
@ -750,24 +744,15 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
assert primaryMode;
|
||||
assert handoffInProgress == false;
|
||||
assert invariant();
|
||||
updateGlobalCheckpoint(
|
||||
allocationId,
|
||||
globalCheckpoint,
|
||||
current -> logger.trace(
|
||||
"updated local knowledge for [{}] on the primary of the global checkpoint from [{}] to [{}]",
|
||||
allocationId,
|
||||
current,
|
||||
globalCheckpoint));
|
||||
assert invariant();
|
||||
}
|
||||
|
||||
private void updateGlobalCheckpoint(final String allocationId, final long globalCheckpoint, LongConsumer ifUpdated) {
|
||||
final CheckpointState cps = checkpoints.get(allocationId);
|
||||
assert !this.shardAllocationId.equals(allocationId) || cps != null;
|
||||
if (cps != null && globalCheckpoint > cps.globalCheckpoint) {
|
||||
final long previousGlobalCheckpoint = cps.globalCheckpoint;
|
||||
cps.globalCheckpoint = globalCheckpoint;
|
||||
ifUpdated.accept(cps.globalCheckpoint);
|
||||
logger.trace("updated local knowledge for [{}] on the primary of the global checkpoint from [{}] to [{}]",
|
||||
allocationId, previousGlobalCheckpoint, globalCheckpoint);
|
||||
}
|
||||
assert invariant();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -824,23 +809,14 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
}
|
||||
} else {
|
||||
for (String initializingId : initializingAllocationIds) {
|
||||
if (shardAllocationId.equals(initializingId) == false) {
|
||||
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
final long globalCheckpoint = localCheckpoint;
|
||||
checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false));
|
||||
}
|
||||
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
final long globalCheckpoint = localCheckpoint;
|
||||
checkpoints.put(initializingId, new CheckpointState(localCheckpoint, globalCheckpoint, false, false));
|
||||
}
|
||||
for (String inSyncId : inSyncAllocationIds) {
|
||||
if (shardAllocationId.equals(inSyncId)) {
|
||||
// current shard is initially marked as not in-sync because we don't know better at that point
|
||||
CheckpointState checkpointState = checkpoints.get(shardAllocationId);
|
||||
checkpointState.inSync = true;
|
||||
checkpointState.tracked = true;
|
||||
} else {
|
||||
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
final long globalCheckpoint = localCheckpoint;
|
||||
checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true));
|
||||
}
|
||||
final long localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
final long globalCheckpoint = localCheckpoint;
|
||||
checkpoints.put(inSyncId, new CheckpointState(localCheckpoint, globalCheckpoint, true, true));
|
||||
}
|
||||
}
|
||||
appliedClusterStateVersion = applyingClusterStateVersion;
|
||||
|
@ -1000,13 +976,11 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
*/
|
||||
private synchronized void updateGlobalCheckpointOnPrimary() {
|
||||
assert primaryMode;
|
||||
final CheckpointState cps = checkpoints.get(shardAllocationId);
|
||||
final long globalCheckpoint = cps.globalCheckpoint;
|
||||
final long computedGlobalCheckpoint = computeGlobalCheckpoint(pendingInSync, checkpoints.values(), getGlobalCheckpoint());
|
||||
assert computedGlobalCheckpoint >= globalCheckpoint : "new global checkpoint [" + computedGlobalCheckpoint +
|
||||
"] is lower than previous one [" + globalCheckpoint + "]";
|
||||
if (globalCheckpoint != computedGlobalCheckpoint) {
|
||||
cps.globalCheckpoint = computedGlobalCheckpoint;
|
||||
globalCheckpoint = computedGlobalCheckpoint;
|
||||
logger.trace("updated global checkpoint to [{}]", computedGlobalCheckpoint);
|
||||
onGlobalCheckpointUpdated.accept(computedGlobalCheckpoint);
|
||||
}
|
||||
|
@ -1056,13 +1030,10 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L
|
|||
primaryMode = false;
|
||||
handoffInProgress = false;
|
||||
relocated = true;
|
||||
// forget all checkpoint information except for global checkpoint of current shard
|
||||
// forget all checkpoint information
|
||||
checkpoints.forEach((key, cps) -> {
|
||||
cps.localCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
if (key.equals(shardAllocationId) == false) {
|
||||
// don't throw global checkpoint information of current shard away
|
||||
cps.globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
}
|
||||
cps.globalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO;
|
||||
});
|
||||
assert invariant();
|
||||
}
|
||||
|
|
|
@ -58,7 +58,7 @@ public class SeqNoStats implements ToXContentFragment, Writeable {
|
|||
return maxSeqNo;
|
||||
}
|
||||
|
||||
/** the maximum sequence number for which all previous operations (including) have been completed */
|
||||
/** the maximum sequence number for which all previous operations (including) have been persisted */
|
||||
public long getLocalCheckpoint() {
|
||||
return localCheckpoint;
|
||||
}
|
||||
|
|
|
@ -540,7 +540,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
*/
|
||||
engine.rollTranslogGeneration();
|
||||
engine.fillSeqNoGaps(newPrimaryTerm);
|
||||
replicationTracker.updateLocalCheckpoint(currentRouting.allocationId().getId(), getLocalCheckpoint());
|
||||
replicationTracker.updateLocalCheckpoint(currentRouting.allocationId().getId(),
|
||||
getLocalCheckpoint());
|
||||
primaryReplicaSyncer.accept(this, new ActionListener<ResyncTask>() {
|
||||
@Override
|
||||
public void onResponse(ResyncTask resyncTask) {
|
||||
|
@ -1865,7 +1866,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
}
|
||||
|
||||
/**
|
||||
* Update the local knowledge of the global checkpoint for the specified allocation ID.
|
||||
* Update the local knowledge of the persisted global checkpoint for the specified allocation ID.
|
||||
*
|
||||
* @param allocationId the allocation ID to update the global checkpoint for
|
||||
* @param globalCheckpoint the global checkpoint
|
||||
|
@ -2079,12 +2080,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the local checkpoint for the shard.
|
||||
* Returns the persisted local checkpoint for the shard.
|
||||
*
|
||||
* @return the local checkpoint
|
||||
*/
|
||||
public long getLocalCheckpoint() {
|
||||
return getEngine().getLocalCheckpoint();
|
||||
return getEngine().getPersistedLocalCheckpoint();
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -2092,7 +2093,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
*
|
||||
* @return the global checkpoint
|
||||
*/
|
||||
public long getGlobalCheckpoint() {
|
||||
public long getLastKnownGlobalCheckpoint() {
|
||||
return replicationTracker.getGlobalCheckpoint();
|
||||
}
|
||||
|
||||
|
@ -2125,15 +2126,19 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
return;
|
||||
}
|
||||
assert assertPrimaryMode();
|
||||
// only sync if there are not operations in flight
|
||||
// only sync if there are no operations in flight, or when using async durability
|
||||
final SeqNoStats stats = getEngine().getSeqNoStats(replicationTracker.getGlobalCheckpoint());
|
||||
if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint()) {
|
||||
final boolean asyncDurability = indexSettings().getTranslogDurability() == Translog.Durability.ASYNC;
|
||||
if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint() || asyncDurability) {
|
||||
final ObjectLongMap<String> globalCheckpoints = getInSyncGlobalCheckpoints();
|
||||
final String allocationId = routingEntry().allocationId().getId();
|
||||
assert globalCheckpoints.containsKey(allocationId);
|
||||
final long globalCheckpoint = globalCheckpoints.get(allocationId);
|
||||
final long globalCheckpoint = replicationTracker.getGlobalCheckpoint();
|
||||
// async durability means that the local checkpoint might lag (as it is only advanced on fsync)
|
||||
// periodically ask for the newest local checkpoint by syncing the global checkpoint, so that ultimately the global
|
||||
// checkpoint can be synced
|
||||
final boolean syncNeeded =
|
||||
StreamSupport
|
||||
(asyncDurability && stats.getGlobalCheckpoint() < stats.getMaxSeqNo())
|
||||
// check if the persisted global checkpoint
|
||||
|| StreamSupport
|
||||
.stream(globalCheckpoints.values().spliterator(), false)
|
||||
.anyMatch(v -> v.value < globalCheckpoint);
|
||||
// only sync if index is not closed and there is a shard lagging the primary
|
||||
|
@ -2192,7 +2197,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
assert shardRouting.primary() && shardRouting.isRelocationTarget() :
|
||||
"only primary relocation target can update allocation IDs from primary context: " + shardRouting;
|
||||
assert primaryContext.getCheckpointStates().containsKey(routingEntry().allocationId().getId()) &&
|
||||
getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint();
|
||||
getLocalCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId())
|
||||
.getLocalCheckpoint();
|
||||
synchronized (mutex) {
|
||||
replicationTracker.activateWithPrimaryContext(primaryContext); // make changes to primaryMode flag only under mutex
|
||||
}
|
||||
|
@ -2733,7 +2739,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
|
||||
bumpPrimaryTerm(opPrimaryTerm, () -> {
|
||||
updateGlobalCheckpointOnReplica(globalCheckpoint, "primary term transition");
|
||||
final long currentGlobalCheckpoint = getGlobalCheckpoint();
|
||||
final long currentGlobalCheckpoint = getLastKnownGlobalCheckpoint();
|
||||
final long maxSeqNo = seqNoStats().getMaxSeqNo();
|
||||
logger.info("detected new primary with primary term [{}], global checkpoint [{}], max_seq_no [{}]",
|
||||
opPrimaryTerm, currentGlobalCheckpoint, maxSeqNo);
|
||||
|
@ -3103,7 +3109,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
flush(new FlushRequest().waitIfOngoing(true));
|
||||
|
||||
SetOnce<Engine> newEngineReference = new SetOnce<>();
|
||||
final long globalCheckpoint = getGlobalCheckpoint();
|
||||
final long globalCheckpoint = getLastKnownGlobalCheckpoint();
|
||||
assert globalCheckpoint == getLastSyncedGlobalCheckpoint();
|
||||
synchronized (mutex) {
|
||||
verifyNotClosed();
|
||||
// we must create both new read-only engine and new read-write engine under mutex to ensure snapshotStoreMetadata,
|
||||
|
|
|
@ -84,7 +84,7 @@ public class PrimaryReplicaSyncer {
|
|||
public void resync(final IndexShard indexShard, final ActionListener<ResyncTask> listener) {
|
||||
Translog.Snapshot snapshot = null;
|
||||
try {
|
||||
final long startingSeqNo = indexShard.getGlobalCheckpoint() + 1;
|
||||
final long startingSeqNo = indexShard.getLastKnownGlobalCheckpoint() + 1;
|
||||
final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo();
|
||||
final ShardId shardId = indexShard.shardId();
|
||||
// Wrap translog snapshot to make it synchronized as it is accessed by different threads through SnapshotSender.
|
||||
|
|
|
@ -64,6 +64,7 @@ import java.util.OptionalLong;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.locks.ReadWriteLock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
import java.util.function.LongConsumer;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
@ -130,6 +131,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
private final LongSupplier primaryTermSupplier;
|
||||
private final String translogUUID;
|
||||
private final TranslogDeletionPolicy deletionPolicy;
|
||||
private final LongConsumer persistedSequenceNumberConsumer;
|
||||
|
||||
/**
|
||||
* Creates a new Translog instance. This method will create a new transaction log unless the given {@link TranslogGeneration} is
|
||||
|
@ -147,14 +149,18 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
* examined and stored in the header whenever a new generation is rolled. It's guaranteed from outside
|
||||
* that a new generation is rolled when the term is increased. This guarantee allows to us to validate
|
||||
* and reject operation whose term is higher than the primary term stored in the translog header.
|
||||
* @param persistedSequenceNumberConsumer a callback that's called whenever an operation with a given sequence number is successfully
|
||||
* persisted.
|
||||
*/
|
||||
public Translog(
|
||||
final TranslogConfig config, final String translogUUID, TranslogDeletionPolicy deletionPolicy,
|
||||
final LongSupplier globalCheckpointSupplier, final LongSupplier primaryTermSupplier) throws IOException {
|
||||
final LongSupplier globalCheckpointSupplier, final LongSupplier primaryTermSupplier,
|
||||
final LongConsumer persistedSequenceNumberConsumer) throws IOException {
|
||||
super(config.getShardId(), config.getIndexSettings());
|
||||
this.config = config;
|
||||
this.globalCheckpointSupplier = globalCheckpointSupplier;
|
||||
this.primaryTermSupplier = primaryTermSupplier;
|
||||
this.persistedSequenceNumberConsumer = persistedSequenceNumberConsumer;
|
||||
this.deletionPolicy = deletionPolicy;
|
||||
this.translogUUID = translogUUID;
|
||||
bigArrays = config.getBigArrays();
|
||||
|
@ -191,7 +197,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
boolean success = false;
|
||||
current = null;
|
||||
try {
|
||||
current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint);
|
||||
current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint,
|
||||
persistedSequenceNumberConsumer);
|
||||
success = true;
|
||||
} finally {
|
||||
// we have to close all the recovered ones otherwise we leak file handles here
|
||||
|
@ -479,7 +486,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
* @throws IOException if creating the translog failed
|
||||
*/
|
||||
TranslogWriter createWriter(long fileGeneration) throws IOException {
|
||||
final TranslogWriter writer = createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong());
|
||||
final TranslogWriter writer = createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong(),
|
||||
persistedSequenceNumberConsumer);
|
||||
assert writer.sizeInBytes() == DEFAULT_HEADER_SIZE_IN_BYTES : "Mismatch translog header size; " +
|
||||
"empty translog size [" + writer.sizeInBytes() + ", header size [" + DEFAULT_HEADER_SIZE_IN_BYTES + "]";
|
||||
return writer;
|
||||
|
@ -494,7 +502,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
* With no readers and no current, a call to {@link #getMinFileGeneration()} would not work.
|
||||
* @param initialGlobalCheckpoint the global checkpoint to be written in the first checkpoint.
|
||||
*/
|
||||
TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException {
|
||||
TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint,
|
||||
LongConsumer persistedSequenceNumberConsumer) throws IOException {
|
||||
final TranslogWriter newFile;
|
||||
try {
|
||||
newFile = TranslogWriter.create(
|
||||
|
@ -505,7 +514,8 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
getChannelFactory(),
|
||||
config.getBufferSize(),
|
||||
initialMinTranslogGen, initialGlobalCheckpoint,
|
||||
globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy);
|
||||
globalCheckpointSupplier, this::getMinFileGeneration, primaryTermSupplier.getAsLong(), tragedy,
|
||||
persistedSequenceNumberConsumer);
|
||||
} catch (final IOException e) {
|
||||
throw new TranslogException(shardId, "failed to create new translog file", e);
|
||||
}
|
||||
|
@ -1876,7 +1886,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
|
|||
location.resolve(getFilename(1)), channelFactory,
|
||||
new ByteSizeValue(10), 1, initialGlobalCheckpoint,
|
||||
() -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); }, primaryTerm,
|
||||
new TragicExceptionHolder());
|
||||
new TragicExceptionHolder(), seqNo -> { throw new UnsupportedOperationException(); });
|
||||
writer.close();
|
||||
return translogUUID;
|
||||
}
|
||||
|
|
|
@ -19,6 +19,8 @@
|
|||
|
||||
package org.elasticsearch.index.translog;
|
||||
|
||||
import com.carrotsearch.hppc.LongArrayList;
|
||||
import com.carrotsearch.hppc.procedures.LongProcedure;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.elasticsearch.core.internal.io.IOUtils;
|
||||
import org.elasticsearch.Assertions;
|
||||
|
@ -42,6 +44,7 @@ import java.util.HashMap;
|
|||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.function.LongConsumer;
|
||||
import java.util.function.LongSupplier;
|
||||
|
||||
public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
||||
|
@ -64,10 +67,15 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
private final LongSupplier globalCheckpointSupplier;
|
||||
private final LongSupplier minTranslogGenerationSupplier;
|
||||
|
||||
// callback that's called whenever an operation with a given sequence number is successfully persisted.
|
||||
private final LongConsumer persistedSequenceNumberConsumer;
|
||||
|
||||
protected final AtomicBoolean closed = new AtomicBoolean(false);
|
||||
// lock order synchronized(syncLock) -> synchronized(this)
|
||||
private final Object syncLock = new Object();
|
||||
|
||||
private LongArrayList nonFsyncedSequenceNumbers;
|
||||
|
||||
private final Map<Long, Tuple<BytesReference, Exception>> seenSequenceNumbers;
|
||||
|
||||
private TranslogWriter(
|
||||
|
@ -78,7 +86,8 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
final Path path,
|
||||
final ByteSizeValue bufferSize,
|
||||
final LongSupplier globalCheckpointSupplier, LongSupplier minTranslogGenerationSupplier, TranslogHeader header,
|
||||
TragicExceptionHolder tragedy)
|
||||
TragicExceptionHolder tragedy,
|
||||
final LongConsumer persistedSequenceNumberConsumer)
|
||||
throws
|
||||
IOException {
|
||||
super(initialCheckpoint.generation, channel, path, header);
|
||||
|
@ -97,6 +106,8 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
this.maxSeqNo = initialCheckpoint.maxSeqNo;
|
||||
assert initialCheckpoint.trimmedAboveSeqNo == SequenceNumbers.UNASSIGNED_SEQ_NO : initialCheckpoint.trimmedAboveSeqNo;
|
||||
this.globalCheckpointSupplier = globalCheckpointSupplier;
|
||||
this.nonFsyncedSequenceNumbers = new LongArrayList(64);
|
||||
this.persistedSequenceNumberConsumer = persistedSequenceNumberConsumer;
|
||||
this.seenSequenceNumbers = Assertions.ENABLED ? new HashMap<>() : null;
|
||||
this.tragedy = tragedy;
|
||||
}
|
||||
|
@ -104,7 +115,7 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, ChannelFactory channelFactory,
|
||||
ByteSizeValue bufferSize, final long initialMinTranslogGen, long initialGlobalCheckpoint,
|
||||
final LongSupplier globalCheckpointSupplier, final LongSupplier minTranslogGenerationSupplier,
|
||||
final long primaryTerm, TragicExceptionHolder tragedy)
|
||||
final long primaryTerm, TragicExceptionHolder tragedy, LongConsumer persistedSequenceNumberConsumer)
|
||||
throws IOException {
|
||||
final FileChannel channel = channelFactory.open(file);
|
||||
try {
|
||||
|
@ -125,7 +136,7 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
writerGlobalCheckpointSupplier = globalCheckpointSupplier;
|
||||
}
|
||||
return new TranslogWriter(channelFactory, shardId, checkpoint, channel, file, bufferSize,
|
||||
writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header, tragedy);
|
||||
writerGlobalCheckpointSupplier, minTranslogGenerationSupplier, header, tragedy, persistedSequenceNumberConsumer);
|
||||
} catch (Exception exception) {
|
||||
// if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that
|
||||
// file exists we remove it. We only apply this logic to the checkpoint.generation+1 any other file with a higher generation
|
||||
|
@ -177,6 +188,8 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
minSeqNo = SequenceNumbers.min(minSeqNo, seqNo);
|
||||
maxSeqNo = SequenceNumbers.max(maxSeqNo, seqNo);
|
||||
|
||||
nonFsyncedSequenceNumbers.add(seqNo);
|
||||
|
||||
operationCounter++;
|
||||
|
||||
assert assertNoSeqNumberConflict(seqNo, data);
|
||||
|
@ -338,7 +351,9 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
* @return <code>true</code> if this call caused an actual sync operation
|
||||
*/
|
||||
public boolean syncUpTo(long offset) throws IOException {
|
||||
boolean synced = false;
|
||||
if (lastSyncedCheckpoint.offset < offset && syncNeeded()) {
|
||||
LongArrayList flushedSequenceNumbers = null;
|
||||
synchronized (syncLock) { // only one sync/checkpoint should happen concurrently but we wait
|
||||
if (lastSyncedCheckpoint.offset < offset && syncNeeded()) {
|
||||
// double checked locking - we don't want to fsync unless we have to and now that we have
|
||||
|
@ -349,6 +364,8 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
try {
|
||||
outputStream.flush();
|
||||
checkpointToSync = getCheckpoint();
|
||||
flushedSequenceNumbers = nonFsyncedSequenceNumbers;
|
||||
nonFsyncedSequenceNumbers = new LongArrayList(64);
|
||||
} catch (final Exception ex) {
|
||||
closeWithTragicEvent(ex);
|
||||
throw ex;
|
||||
|
@ -366,11 +383,14 @@ public class TranslogWriter extends BaseTranslogReader implements Closeable {
|
|||
assert lastSyncedCheckpoint.offset <= checkpointToSync.offset :
|
||||
"illegal state: " + lastSyncedCheckpoint.offset + " <= " + checkpointToSync.offset;
|
||||
lastSyncedCheckpoint = checkpointToSync; // write protected by syncLock
|
||||
return true;
|
||||
synced = true;
|
||||
}
|
||||
}
|
||||
if (flushedSequenceNumbers != null) {
|
||||
flushedSequenceNumbers.forEach((LongProcedure) persistedSequenceNumberConsumer::accept);
|
||||
}
|
||||
}
|
||||
return false;
|
||||
return synced;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -181,7 +181,7 @@ public class TruncateTranslogAction {
|
|||
new TranslogDeletionPolicy(indexSettings.getTranslogRetentionSize().getBytes(),
|
||||
indexSettings.getTranslogRetentionAge().getMillis());
|
||||
try (Translog translog = new Translog(translogConfig, translogUUID,
|
||||
translogDeletionPolicy, () -> translogGlobalCheckpoint, () -> primaryTerm);
|
||||
translogDeletionPolicy, () -> translogGlobalCheckpoint, () -> primaryTerm, seqNo -> {});
|
||||
Translog.Snapshot snapshot = translog.newSnapshot()) {
|
||||
//noinspection StatementWithEmptyBody we are just checking that we can iterate through the whole snapshot
|
||||
while (snapshot.next() != null) {
|
||||
|
|
|
@ -177,7 +177,7 @@ public class RecoverySourceHandler {
|
|||
startingSeqNo = 0;
|
||||
try {
|
||||
final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo);
|
||||
sendFileResult = phase1(phase1Snapshot.getIndexCommit(), shard.getGlobalCheckpoint(), () -> estimateNumOps);
|
||||
sendFileResult = phase1(phase1Snapshot.getIndexCommit(), shard.getLastKnownGlobalCheckpoint(), () -> estimateNumOps);
|
||||
} catch (final Exception e) {
|
||||
throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
|
||||
} finally {
|
||||
|
@ -644,7 +644,7 @@ public class RecoverySourceHandler {
|
|||
*/
|
||||
runUnderPrimaryPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint),
|
||||
shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger);
|
||||
final long globalCheckpoint = shard.getGlobalCheckpoint();
|
||||
final long globalCheckpoint = shard.getLastKnownGlobalCheckpoint(); // this global checkpoint is persisted in finalizeRecovery
|
||||
final StepListener<Void> finalizeListener = new StepListener<>();
|
||||
cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint, finalizeListener));
|
||||
finalizeListener.whenComplete(r -> {
|
||||
|
@ -712,7 +712,8 @@ public class RecoverySourceHandler {
|
|||
final BytesArray content = new BytesArray(buffer, 0, bytesRead);
|
||||
final boolean lastChunk = position + content.length() == md.length();
|
||||
final long requestSeqId = requestSeqIdTracker.generateSeqNo();
|
||||
cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqId - maxConcurrentFileChunks));
|
||||
cancellableThreads.execute(
|
||||
() -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - maxConcurrentFileChunks));
|
||||
cancellableThreads.checkForCancel();
|
||||
if (error.get() != null) {
|
||||
break;
|
||||
|
@ -721,10 +722,10 @@ public class RecoverySourceHandler {
|
|||
cancellableThreads.executeIO(() ->
|
||||
recoveryTarget.writeFileChunk(md, requestFilePosition, content, lastChunk, translogOps.get(),
|
||||
ActionListener.wrap(
|
||||
r -> requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId),
|
||||
r -> requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId),
|
||||
e -> {
|
||||
error.compareAndSet(null, Tuple.tuple(md, e));
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
}
|
||||
)));
|
||||
position += content.length();
|
||||
|
@ -737,7 +738,7 @@ public class RecoverySourceHandler {
|
|||
// When we terminate exceptionally, we don't wait for the outstanding requests as we don't use their results anyway.
|
||||
// This allows us to end quickly and eliminate the complexity of handling requestSeqIds in case of error.
|
||||
if (error.get() == null) {
|
||||
cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqIdTracker.getMaxSeqNo()));
|
||||
cancellableThreads.execute(() -> requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo()));
|
||||
}
|
||||
if (error.get() != null) {
|
||||
handleErrorOnSendFiles(store, error.get().v1(), error.get().v2());
|
||||
|
|
|
@ -136,9 +136,13 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
private void executeOnPrimaryOrReplica() throws Throwable {
|
||||
executeOnPrimaryOrReplica(false);
|
||||
}
|
||||
|
||||
private void executeOnPrimaryOrReplica(boolean phase1) throws Throwable {
|
||||
final TaskId taskId = new TaskId("_node_id", randomNonNegativeLong());
|
||||
final TransportVerifyShardBeforeCloseAction.ShardRequest request =
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), clusterBlock, taskId);
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), clusterBlock, phase1, taskId);
|
||||
final PlainActionFuture<Void> res = PlainActionFuture.newFuture();
|
||||
action.shardOperationOnPrimary(request, indexShard, ActionListener.wrap(
|
||||
r -> {
|
||||
|
@ -165,6 +169,11 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase {
|
|||
assertThat(flushRequest.getValue().force(), is(true));
|
||||
}
|
||||
|
||||
public void testShardIsSynced() throws Throwable {
|
||||
executeOnPrimaryOrReplica(true);
|
||||
verify(indexShard, times(1)).sync();
|
||||
}
|
||||
|
||||
public void testOperationFailsWhenNotBlocked() {
|
||||
when(indexShard.getActiveOperationsCount()).thenReturn(randomIntBetween(0, 10));
|
||||
|
||||
|
@ -227,7 +236,7 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase {
|
|||
final PlainActionFuture<PrimaryResult> listener = new PlainActionFuture<>();
|
||||
TaskId taskId = new TaskId(clusterService.localNode().getId(), 0L);
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest request =
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, clusterBlock, taskId);
|
||||
new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, clusterBlock, false, taskId);
|
||||
ReplicationOperation.Replicas<TransportVerifyShardBeforeCloseAction.ShardRequest> proxy = action.newReplicasProxy();
|
||||
ReplicationOperation<TransportVerifyShardBeforeCloseAction.ShardRequest,
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest, PrimaryResult> operation = new ReplicationOperation<>(
|
||||
|
@ -268,53 +277,56 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase {
|
|||
TransportVerifyShardBeforeCloseAction.ShardRequest,
|
||||
PrimaryResult>
|
||||
createPrimary(final ShardRouting primary, final ReplicationGroup replicationGroup) {
|
||||
return new ReplicationOperation.Primary<
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest,
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest,
|
||||
PrimaryResult>() {
|
||||
@Override
|
||||
public ShardRouting routingEntry() {
|
||||
return primary;
|
||||
}
|
||||
return new ReplicationOperation.Primary<TransportVerifyShardBeforeCloseAction.ShardRequest,
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest, PrimaryResult>() {
|
||||
@Override
|
||||
public ShardRouting routingEntry() {
|
||||
return primary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
return replicationGroup;
|
||||
}
|
||||
@Override
|
||||
public ReplicationGroup getReplicationGroup() {
|
||||
return replicationGroup;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void perform(
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest request, ActionListener<PrimaryResult> listener) {
|
||||
listener.onResponse(new PrimaryResult(request));
|
||||
}
|
||||
@Override
|
||||
public void perform(
|
||||
TransportVerifyShardBeforeCloseAction.ShardRequest request, ActionListener<PrimaryResult> listener) {
|
||||
listener.onResponse(new PrimaryResult(request));
|
||||
}
|
||||
|
||||
@Override
|
||||
public void failShard(String message, Exception exception) {
|
||||
@Override
|
||||
public void failShard(String message, Exception exception) {
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateLocalCheckpointForShard(String allocationId, long checkpoint) {
|
||||
}
|
||||
@Override
|
||||
public void updateLocalCheckpointForShard(String allocationId, long checkpoint) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) {
|
||||
}
|
||||
@Override
|
||||
public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public long localCheckpoint() {
|
||||
return 0;
|
||||
}
|
||||
@Override
|
||||
public long localCheckpoint() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long globalCheckpoint() {
|
||||
return 0;
|
||||
}
|
||||
@Override
|
||||
public long computedGlobalCheckpoint() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long maxSeqNoOfUpdatesOrDeletes() {
|
||||
return 0;
|
||||
}
|
||||
@Override
|
||||
public long globalCheckpoint() {
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long maxSeqNoOfUpdatesOrDeletes() {
|
||||
return 0;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -23,8 +23,13 @@ import org.elasticsearch.action.index.IndexRequest;
|
|||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
|
@ -41,6 +46,18 @@ public class BulkRejectionIT extends ESIntegTestCase {
|
|||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<Class<? extends Plugin>> nodePlugins() {
|
||||
return Arrays.asList(InternalSettingsPlugin.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Settings indexSettings() {
|
||||
return Settings.builder().put(super.indexSettings())
|
||||
// sync global checkpoint quickly so we can verify seq_no_stats aligned between all copies after tests.
|
||||
.put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s").build();
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int numberOfReplicas() {
|
||||
return 1;
|
||||
|
|
|
@ -137,6 +137,7 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
|
||||
assertThat(primary.knownLocalCheckpoints.remove(primaryShard.allocationId().getId()), equalTo(primary.localCheckpoint));
|
||||
assertThat(primary.knownLocalCheckpoints, equalTo(replicasProxy.generatedLocalCheckpoints));
|
||||
assertThat(primary.knownGlobalCheckpoints.remove(primaryShard.allocationId().getId()), equalTo(primary.globalCheckpoint));
|
||||
assertThat(primary.knownGlobalCheckpoints, equalTo(replicasProxy.generatedGlobalCheckpoints));
|
||||
}
|
||||
|
||||
|
@ -533,6 +534,11 @@ public class ReplicationOperationTests extends ESTestCase {
|
|||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long computedGlobalCheckpoint() {
|
||||
return globalCheckpoint;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long maxSeqNoOfUpdatesOrDeletes() {
|
||||
return maxSeqNoOfUpdatesOrDeletes;
|
||||
|
|
|
@ -0,0 +1,180 @@
|
|||
/*
|
||||
* 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.discovery;
|
||||
|
||||
import com.carrotsearch.randomizedtesting.RandomizedTest;
|
||||
import org.apache.lucene.mockfile.FilterFileSystemProvider;
|
||||
import org.elasticsearch.action.admin.indices.stats.ShardStats;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.io.PathUtils;
|
||||
import org.elasticsearch.common.io.PathUtilsForTesting;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.seqno.SequenceNumbers;
|
||||
import org.elasticsearch.test.BackgroundIndexer;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.InternalTestCluster;
|
||||
import org.elasticsearch.test.junit.annotations.TestLogging;
|
||||
import org.junit.After;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.BeforeClass;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.nio.file.FileSystem;
|
||||
import java.nio.file.OpenOption;
|
||||
import java.nio.file.Path;
|
||||
import java.nio.file.attribute.FileAttribute;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
|
||||
|
||||
@TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE")
|
||||
@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0)
|
||||
public class DiskDisruptionIT extends AbstractDisruptionTestCase {
|
||||
|
||||
private static DisruptTranslogFileSystemProvider disruptTranslogFileSystemProvider;
|
||||
|
||||
@BeforeClass
|
||||
public static void installDisruptTranslogFS() {
|
||||
FileSystem current = PathUtils.getDefaultFileSystem();
|
||||
disruptTranslogFileSystemProvider = new DisruptTranslogFileSystemProvider(current);
|
||||
PathUtilsForTesting.installMock(disruptTranslogFileSystemProvider.getFileSystem(null));
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void removeDisruptTranslogFS() {
|
||||
PathUtilsForTesting.teardown();
|
||||
}
|
||||
|
||||
void injectTranslogFailures() {
|
||||
disruptTranslogFileSystemProvider.injectFailures.set(true);
|
||||
}
|
||||
|
||||
@After
|
||||
void stopTranslogFailures() {
|
||||
disruptTranslogFileSystemProvider.injectFailures.set(false);
|
||||
}
|
||||
|
||||
static class DisruptTranslogFileSystemProvider extends FilterFileSystemProvider {
|
||||
|
||||
AtomicBoolean injectFailures = new AtomicBoolean();
|
||||
|
||||
DisruptTranslogFileSystemProvider(FileSystem inner) {
|
||||
super("disrupttranslog://", inner);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChannel newFileChannel(Path path, Set<? extends OpenOption> options, FileAttribute<?>... attrs) throws IOException {
|
||||
if (injectFailures.get() && path.toString().endsWith(".ckp")) {
|
||||
// prevents checkpoint file to be updated
|
||||
throw new IOException("fake IOException");
|
||||
}
|
||||
return super.newFileChannel(path, options, attrs);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* This test checks that all operations below the global checkpoint are properly persisted.
|
||||
* It simulates a full power outage by preventing translog checkpoint files to be written and restart the cluster. This means that
|
||||
* all un-fsynced data will be lost.
|
||||
*/
|
||||
public void testGlobalCheckpointIsSafe() throws Exception {
|
||||
startCluster(rarely() ? 5 : 3);
|
||||
|
||||
final int numberOfShards = 1 + randomInt(2);
|
||||
assertAcked(prepareCreate("test")
|
||||
.setSettings(Settings.builder()
|
||||
.put(indexSettings())
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards)
|
||||
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomInt(2))
|
||||
));
|
||||
ensureGreen();
|
||||
|
||||
AtomicBoolean stopGlobalCheckpointFetcher = new AtomicBoolean();
|
||||
|
||||
Map<Integer, Long> shardToGcp = new ConcurrentHashMap<>();
|
||||
for (int i = 0; i < numberOfShards; i++) {
|
||||
shardToGcp.put(i, SequenceNumbers.NO_OPS_PERFORMED);
|
||||
}
|
||||
final Thread globalCheckpointSampler = new Thread(() -> {
|
||||
while (stopGlobalCheckpointFetcher.get() == false) {
|
||||
try {
|
||||
for (ShardStats shardStats : client().admin().indices().prepareStats("test").clear().get().getShards()) {
|
||||
final int shardId = shardStats.getShardRouting().id();
|
||||
final long globalCheckpoint = shardStats.getSeqNoStats().getGlobalCheckpoint();
|
||||
shardToGcp.compute(shardId, (i, v) -> Math.max(v, globalCheckpoint));
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// ignore
|
||||
logger.debug("failed to fetch shard stats", e);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
globalCheckpointSampler.start();
|
||||
|
||||
try (BackgroundIndexer indexer = new BackgroundIndexer("test", "_doc", client(), -1, RandomizedTest.scaledRandomIntBetween(2, 5),
|
||||
false, random())) {
|
||||
indexer.setRequestTimeout(TimeValue.ZERO);
|
||||
indexer.setIgnoreIndexingFailures(true);
|
||||
indexer.setAssertNoFailuresOnStop(false);
|
||||
indexer.start(-1);
|
||||
|
||||
waitForDocs(randomIntBetween(1, 100), indexer);
|
||||
|
||||
logger.info("injecting failures");
|
||||
injectTranslogFailures();
|
||||
logger.info("stopping indexing");
|
||||
}
|
||||
|
||||
logger.info("full cluster restart");
|
||||
internalCluster().fullRestart(new InternalTestCluster.RestartCallback() {
|
||||
|
||||
@Override
|
||||
public void onAllNodesStopped() {
|
||||
logger.info("stopping failures");
|
||||
stopTranslogFailures();
|
||||
}
|
||||
|
||||
});
|
||||
|
||||
stopGlobalCheckpointFetcher.set(true);
|
||||
|
||||
logger.info("waiting for global checkpoint sampler");
|
||||
globalCheckpointSampler.join();
|
||||
|
||||
logger.info("waiting for green");
|
||||
ensureGreen("test");
|
||||
|
||||
for (ShardStats shardStats : client().admin().indices().prepareStats("test").clear().get().getShards()) {
|
||||
final int shardId = shardStats.getShardRouting().id();
|
||||
final long maxSeqNo = shardStats.getSeqNoStats().getMaxSeqNo();
|
||||
if (shardStats.getShardRouting().active()) {
|
||||
assertThat(maxSeqNo, greaterThanOrEqualTo(shardToGcp.get(shardId)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -677,7 +677,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
|
||||
@Override
|
||||
public long getCheckpoint() {
|
||||
public long getProcessedCheckpoint() {
|
||||
return localCheckpoint.get();
|
||||
}
|
||||
}
|
||||
|
@ -891,18 +891,18 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
}
|
||||
maxSeqNo = engine.getLocalCheckpointTracker().getMaxSeqNo();
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getProcessedLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
}
|
||||
try (InternalEngine engine = new InternalEngine(config)) {
|
||||
engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
assertThat(engine.getLocalCheckpoint(), equalTo(maxSeqNo));
|
||||
assertThat(engine.getProcessedLocalCheckpoint(), equalTo(maxSeqNo));
|
||||
assertThat(engine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(maxSeqNo));
|
||||
}
|
||||
try (InternalEngine engine = new InternalEngine(config)) {
|
||||
long upToSeqNo = randomLongBetween(globalCheckpoint.get(), maxSeqNo);
|
||||
engine.recoverFromTranslog(translogHandler, upToSeqNo);
|
||||
assertThat(engine.getLocalCheckpoint(), equalTo(upToSeqNo));
|
||||
assertThat(engine.getProcessedLocalCheckpoint(), equalTo(upToSeqNo));
|
||||
assertThat(engine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(upToSeqNo));
|
||||
}
|
||||
}
|
||||
|
@ -1172,7 +1172,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
engine.index(indexForDoc(doc));
|
||||
boolean inSync = randomBoolean();
|
||||
if (inSync) {
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // to advance persisted local checkpoint
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
}
|
||||
|
||||
engine.flush();
|
||||
|
@ -1190,7 +1191,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 4L : 1L));
|
||||
assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(4L));
|
||||
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
engine.flush(true, true);
|
||||
assertThat(engine.getTranslog().currentFileGeneration(), equalTo(5L));
|
||||
assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(5L));
|
||||
|
@ -1614,7 +1615,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
engine.flush();
|
||||
|
||||
long localCheckpoint = engine.getLocalCheckpoint();
|
||||
long localCheckpoint = engine.getProcessedLocalCheckpoint();
|
||||
globalCheckpoint.set(randomLongBetween(0, localCheckpoint));
|
||||
engine.syncTranslog();
|
||||
final long safeCommitCheckpoint;
|
||||
|
@ -1702,7 +1703,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
}
|
||||
engine.flush();
|
||||
globalCheckpoint.set(randomLongBetween(0, engine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(0, engine.getPersistedLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
final long minSeqNoToRetain;
|
||||
try (Engine.IndexCommitRef safeCommit = engine.acquireSafeIndexCommit()) {
|
||||
|
@ -1714,7 +1715,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService);
|
||||
Map<Long, Translog.Operation> ops = readAllOperationsInLucene(engine, mapperService)
|
||||
.stream().collect(Collectors.toMap(Translog.Operation::seqNo, Function.identity()));
|
||||
for (long seqno = 0; seqno <= engine.getLocalCheckpoint(); seqno++) {
|
||||
for (long seqno = 0; seqno <= engine.getPersistedLocalCheckpoint(); seqno++) {
|
||||
String msg = "seq# [" + seqno + "], global checkpoint [" + globalCheckpoint + "], retained-ops [" + retainedExtraOps + "]";
|
||||
if (seqno < minSeqNoToRetain) {
|
||||
Translog.Operation op = ops.get(seqno);
|
||||
|
@ -1736,14 +1737,14 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
// If the global checkpoint equals to the local checkpoint, the next force-merge will be a noop
|
||||
// because all deleted documents are expunged in the previous force-merge already. We need to flush
|
||||
// a new segment to make merge happen so that we can verify that all _recovery_source are pruned.
|
||||
if (globalCheckpoint.get() == engine.getLocalCheckpoint() && liveDocs.isEmpty() == false) {
|
||||
if (globalCheckpoint.get() == engine.getPersistedLocalCheckpoint() && liveDocs.isEmpty() == false) {
|
||||
String deleteId = randomFrom(liveDocs);
|
||||
engine.delete(new Engine.Delete("test", deleteId, newUid(deleteId), primaryTerm.get()));
|
||||
liveDocsWithSource.remove(deleteId);
|
||||
liveDocs.remove(deleteId);
|
||||
engine.flush();
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
engine.syncTranslog();
|
||||
engine.forceMerge(true, 1, false, false, false);
|
||||
assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService);
|
||||
|
@ -2449,12 +2450,14 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
initialEngine.syncTranslog(); // to advance persisted local checkpoint
|
||||
|
||||
if (randomInt(10) < 3) {
|
||||
// only update rarely as we do it every doc
|
||||
replicaLocalCheckpoint = randomIntBetween(Math.toIntExact(replicaLocalCheckpoint), Math.toIntExact(primarySeqNo));
|
||||
}
|
||||
gcpTracker.updateLocalCheckpoint(primary.allocationId().getId(),
|
||||
initialEngine.getLocalCheckpoint());
|
||||
initialEngine.getPersistedLocalCheckpoint());
|
||||
gcpTracker.updateLocalCheckpoint(replica.allocationId().getId(), replicaLocalCheckpoint);
|
||||
|
||||
if (rarely()) {
|
||||
|
@ -2468,7 +2471,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
globalCheckpoint = gcpTracker.getGlobalCheckpoint();
|
||||
|
||||
assertEquals(primarySeqNo, initialEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(primarySeqNo, initialEngine.getLocalCheckpoint());
|
||||
assertEquals(primarySeqNo, initialEngine.getPersistedLocalCheckpoint());
|
||||
assertThat(globalCheckpoint, equalTo(replicaLocalCheckpoint));
|
||||
|
||||
assertThat(
|
||||
|
@ -2502,7 +2505,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
// that the committed max seq no is equivalent to what the current primary seq no is, as all data
|
||||
// we have assigned sequence numbers to should be in the commit
|
||||
equalTo(primarySeqNo));
|
||||
assertThat(recoveringEngine.getLocalCheckpoint(), equalTo(primarySeqNo));
|
||||
assertThat(recoveringEngine.getProcessedLocalCheckpoint(), equalTo(primarySeqNo));
|
||||
assertThat(recoveringEngine.getPersistedLocalCheckpoint(), equalTo(primarySeqNo));
|
||||
assertThat(recoveringEngine.getSeqNoStats(-1).getMaxSeqNo(), equalTo(primarySeqNo));
|
||||
assertThat(generateNewSeqNo(recoveringEngine), equalTo(primarySeqNo + 1));
|
||||
}
|
||||
|
@ -2819,7 +2823,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
|
||||
try (InternalEngine engine = createEngine(config)) {
|
||||
engine.index(firstIndexRequest);
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // to advance persisted local checkpoint
|
||||
assertEquals(engine.getProcessedLocalCheckpoint(), engine.getPersistedLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE));
|
||||
Map<String, String> userData = engine.getLastCommittedSegmentInfos().getUserData();
|
||||
assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY));
|
||||
|
@ -2981,7 +2987,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
final ParsedDocument doc1 = testParsedDocument("1", null,
|
||||
testDocumentWithTextField(), SOURCE, null);
|
||||
engine.index(indexForDoc(doc1));
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // to advance local checkpoint
|
||||
assertEquals(engine.getProcessedLocalCheckpoint(), engine.getPersistedLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
throwErrorOnCommit.set(true);
|
||||
FlushFailedEngineException e = expectThrows(FlushFailedEngineException.class, engine::flush);
|
||||
assertThat(e.getCause().getMessage(), equalTo("power's out"));
|
||||
|
@ -3041,7 +3049,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
|
||||
public void testTranslogReplay() throws IOException {
|
||||
final LongSupplier inSyncGlobalCheckpointSupplier = () -> this.engine.getLocalCheckpoint();
|
||||
final LongSupplier inSyncGlobalCheckpointSupplier = () -> this.engine.getProcessedLocalCheckpoint();
|
||||
final int numDocs = randomIntBetween(1, 10);
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null);
|
||||
|
@ -3135,7 +3143,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
final String badUUID = Translog.createEmptyTranslog(badTranslogLog, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get());
|
||||
Translog translog = new Translog(
|
||||
new TranslogConfig(shardId, badTranslogLog, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE),
|
||||
badUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
badUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {});
|
||||
translog.add(new Translog.Index("test", "SomeBogusId", 0, primaryTerm.get(),
|
||||
"{}".getBytes(Charset.forName("UTF-8"))));
|
||||
assertEquals(generation.translogFileGeneration, translog.currentFileGeneration());
|
||||
|
@ -4177,9 +4185,10 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
assertThat(initialEngine.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint.get()));
|
||||
assertThat(initialEngine.getProcessedLocalCheckpoint(), equalTo(expectedLocalCheckpoint.get()));
|
||||
assertThat(initialEngine.getSeqNoStats(-1).getMaxSeqNo(), equalTo((long) (docs - 1)));
|
||||
initialEngine.flush(true, true);
|
||||
assertEquals(initialEngine.getProcessedLocalCheckpoint(), initialEngine.getPersistedLocalCheckpoint());
|
||||
|
||||
latchReference.get().countDown();
|
||||
for (final Thread thread : threads) {
|
||||
|
@ -4188,10 +4197,11 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
} finally {
|
||||
IOUtils.close(initialEngine);
|
||||
}
|
||||
try (Engine recoveringEngine = new InternalEngine(initialEngine.config())) {
|
||||
try (InternalEngine recoveringEngine = new InternalEngine(initialEngine.config())) {
|
||||
recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
recoveringEngine.fillSeqNoGaps(2);
|
||||
assertThat(recoveringEngine.getLocalCheckpoint(), greaterThanOrEqualTo((long) (docs - 1)));
|
||||
assertEquals(recoveringEngine.getProcessedLocalCheckpoint(), recoveringEngine.getPersistedLocalCheckpoint());
|
||||
assertThat(recoveringEngine.getProcessedLocalCheckpoint(), greaterThanOrEqualTo((long) (docs - 1)));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4272,7 +4282,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
expectedLocalCheckpoint = numberOfOperations - 1;
|
||||
}
|
||||
|
||||
assertThat(engine.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint));
|
||||
assertThat(engine.getProcessedLocalCheckpoint(), equalTo(expectedLocalCheckpoint));
|
||||
try (Engine.GetResult result = engine.get(new Engine.Get(true, false,
|
||||
"type", "2", uid), searcherFactory)) {
|
||||
assertThat(result.exists(), equalTo(exists));
|
||||
|
@ -4304,12 +4314,12 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
final int gapsFilled = noOpEngine.fillSeqNoGaps(primaryTerm.get());
|
||||
final String reason = "filling gaps";
|
||||
noOpEngine.noOp(new Engine.NoOp(maxSeqNo + 1, primaryTerm.get(), LOCAL_TRANSLOG_RECOVERY, System.nanoTime(), reason));
|
||||
assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1)));
|
||||
assertThat(noOpEngine.getProcessedLocalCheckpoint(), equalTo((long) (maxSeqNo + 1)));
|
||||
assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled));
|
||||
noOpEngine.noOp(
|
||||
new Engine.NoOp(maxSeqNo + 2, primaryTerm.get(),
|
||||
randomFrom(PRIMARY, REPLICA, PEER_RECOVERY), System.nanoTime(), reason));
|
||||
assertThat(noOpEngine.getLocalCheckpoint(), equalTo((long) (maxSeqNo + 2)));
|
||||
assertThat(noOpEngine.getProcessedLocalCheckpoint(), equalTo((long) (maxSeqNo + 2)));
|
||||
assertThat(noOpEngine.getTranslog().stats().getUncommittedOperations(), equalTo(gapsFilled + 1));
|
||||
// skip to the op that we added to the translog
|
||||
Translog.Operation op;
|
||||
|
@ -4528,7 +4538,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
engine.flush();
|
||||
}
|
||||
}
|
||||
globalCheckpoint.set(randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, engine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, engine.getPersistedLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
prevSeqNoStats = engine.getSeqNoStats(globalCheckpoint.get());
|
||||
prevDocs = getDocIds(engine, true);
|
||||
|
@ -4565,7 +4575,9 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
replicaEngine.index(replicaIndexForDoc(doc, 1, indexResult.getSeqNo(), false));
|
||||
}
|
||||
}
|
||||
checkpointOnReplica = replicaEngine.getLocalCheckpoint();
|
||||
engine.syncTranslog(); // to advance local checkpoint
|
||||
replicaEngine.syncTranslog(); // to advance local checkpoint
|
||||
checkpointOnReplica = replicaEngine.getProcessedLocalCheckpoint();
|
||||
} finally {
|
||||
IOUtils.close(replicaEngine);
|
||||
}
|
||||
|
@ -4573,17 +4585,17 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
|
||||
boolean flushed = false;
|
||||
AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED);
|
||||
Engine recoveringEngine = null;
|
||||
InternalEngine recoveringEngine = null;
|
||||
try {
|
||||
assertEquals(docs - 1, engine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(docs - 1, engine.getLocalCheckpoint());
|
||||
assertEquals(docs - 1, engine.getProcessedLocalCheckpoint());
|
||||
assertEquals(maxSeqIDOnReplica, replicaEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(checkpointOnReplica, replicaEngine.getLocalCheckpoint());
|
||||
assertEquals(checkpointOnReplica, replicaEngine.getProcessedLocalCheckpoint());
|
||||
recoveringEngine = new InternalEngine(copy(replicaEngine.config(), globalCheckpoint::get));
|
||||
assertEquals(numDocsOnReplica, getTranslog(recoveringEngine).stats().getUncommittedOperations());
|
||||
recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(checkpointOnReplica, recoveringEngine.getLocalCheckpoint());
|
||||
assertEquals(checkpointOnReplica, recoveringEngine.getProcessedLocalCheckpoint());
|
||||
assertEquals((maxSeqIDOnReplica + 1) - numDocsOnReplica, recoveringEngine.fillSeqNoGaps(2));
|
||||
|
||||
// now snapshot the tlog and ensure the primary term is updated
|
||||
|
@ -4599,7 +4611,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
|
||||
}
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpoint());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getProcessedLocalCheckpoint());
|
||||
if ((flushed = randomBoolean())) {
|
||||
globalCheckpoint.set(recoveringEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
getTranslog(recoveringEngine).sync();
|
||||
|
@ -4618,10 +4630,10 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpoint());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getProcessedLocalCheckpoint());
|
||||
assertEquals(0, recoveringEngine.fillSeqNoGaps(3));
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getSeqNoStats(-1).getMaxSeqNo());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpoint());
|
||||
assertEquals(maxSeqIDOnReplica, recoveringEngine.getProcessedLocalCheckpoint());
|
||||
} finally {
|
||||
IOUtils.close(recoveringEngine);
|
||||
}
|
||||
|
@ -4805,7 +4817,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
// Advance the global checkpoint during the flush to create a lag between a persisted global checkpoint in the translog
|
||||
// (this value is visible to the deletion policy) and an in memory global checkpoint in the SequenceNumbersService.
|
||||
if (rarely()) {
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), getPersistedLocalCheckpoint()));
|
||||
}
|
||||
super.commitIndexWriter(writer, translog, syncId);
|
||||
}
|
||||
|
@ -4817,7 +4829,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE));
|
||||
engine.index(indexForDoc(testParsedDocument(Integer.toString(docId), null, document, B_1, null)));
|
||||
if (frequently()) {
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
}
|
||||
if (frequently()) {
|
||||
|
@ -4957,11 +4969,11 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
engine.flush(false, randomBoolean());
|
||||
List<IndexCommit> commits = DirectoryReader.listCommits(store.directory());
|
||||
// Global checkpoint advanced but not enough - all commits are kept.
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint() - 1));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint() - 1));
|
||||
engine.syncTranslog();
|
||||
assertThat(DirectoryReader.listCommits(store.directory()), equalTo(commits));
|
||||
// Global checkpoint advanced enough - only the last commit is kept.
|
||||
globalCheckpoint.set(randomLongBetween(engine.getLocalCheckpoint(), Long.MAX_VALUE));
|
||||
globalCheckpoint.set(randomLongBetween(engine.getPersistedLocalCheckpoint(), Long.MAX_VALUE));
|
||||
engine.syncTranslog();
|
||||
assertThat(DirectoryReader.listCommits(store.directory()), contains(commits.get(commits.size() - 1)));
|
||||
assertThat(engine.getTranslog().totalOperations(), equalTo(0));
|
||||
|
@ -4986,7 +4998,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
for (int i = 0; i < numSnapshots; i++) {
|
||||
snapshots.add(engine.acquireSafeIndexCommit()); // taking snapshots from the safe commit.
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
engine.syncTranslog();
|
||||
final List<IndexCommit> commits = DirectoryReader.listCommits(store.directory());
|
||||
for (int i = 0; i < numSnapshots - 1; i++) {
|
||||
|
@ -5067,7 +5079,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
engine.onSettingsChanged();
|
||||
final int numOps = scaledRandomIntBetween(100, 10_000);
|
||||
for (int i = 0; i < numOps; i++) {
|
||||
final long localCheckPoint = engine.getLocalCheckpoint();
|
||||
final long localCheckPoint = engine.getProcessedLocalCheckpoint();
|
||||
final long seqno = randomLongBetween(Math.max(0, localCheckPoint), localCheckPoint + 5);
|
||||
final ParsedDocument doc =
|
||||
testParsedDocument(Long.toString(seqno), null, testDocumentWithTextField(), SOURCE, null);
|
||||
|
@ -5250,8 +5262,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
appendOnlyIndexer.join(120_000);
|
||||
assertThat(engine.getMaxSeqNoOfNonAppendOnlyOperations(), equalTo(maxSeqNoOfNonAppendOnly));
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog();
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
engine.flush();
|
||||
}
|
||||
try (InternalEngine engine = createEngine(store, translogPath, globalCheckpoint::get)) {
|
||||
|
@ -5435,7 +5447,10 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
}
|
||||
existingSeqNos.add(result.getSeqNo());
|
||||
if (randomBoolean()) {
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint()));
|
||||
engine.syncTranslog(); // advance persisted local checkpoint
|
||||
assertEquals(engine.getProcessedLocalCheckpoint(), engine.getPersistedLocalCheckpoint());
|
||||
globalCheckpoint.set(
|
||||
randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getPersistedCheckpoint()));
|
||||
}
|
||||
if (randomBoolean()) {
|
||||
retentionLeasesVersion.incrementAndGet();
|
||||
|
@ -5499,7 +5514,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
latch.countDown();
|
||||
refreshThreads[i] = new Thread(() -> {
|
||||
while (done.get() == false) {
|
||||
long checkPointBeforeRefresh = engine.getLocalCheckpoint();
|
||||
long checkPointBeforeRefresh = engine.getProcessedLocalCheckpoint();
|
||||
engine.refresh("test", randomFrom(Engine.SearcherScope.values()), true);
|
||||
assertThat(engine.lastRefreshedCheckpoint(), greaterThanOrEqualTo(checkPointBeforeRefresh));
|
||||
}
|
||||
|
@ -5515,7 +5530,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
thread.join();
|
||||
}
|
||||
engine.refresh("test");
|
||||
assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getLocalCheckpoint()));
|
||||
assertThat(engine.lastRefreshedCheckpoint(), equalTo(engine.getProcessedLocalCheckpoint()));
|
||||
}
|
||||
|
||||
public void testLuceneSnapshotRefreshesOnlyOnce() throws Exception {
|
||||
|
@ -5628,8 +5643,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
flushedOperations.add(op);
|
||||
applyOperation(engine, op);
|
||||
if (randomBoolean()) {
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
}
|
||||
if (randomInt(100) < 10) {
|
||||
engine.refresh("test");
|
||||
|
@ -5653,7 +5668,7 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
try (InternalEngine engine = new InternalEngine(config)) { // do not recover from translog
|
||||
final Map<BytesRef, Engine.Operation> deletesAfterCheckpoint = new HashMap<>();
|
||||
for (Engine.Operation op : operationsInSafeCommit) {
|
||||
if (op instanceof Engine.NoOp == false && op.seqNo() > engine.getLocalCheckpoint()) {
|
||||
if (op instanceof Engine.NoOp == false && op.seqNo() > engine.getPersistedLocalCheckpoint()) {
|
||||
deletesAfterCheckpoint.put(new Term(IdFieldMapper.NAME, Uid.encodeId(op.id())).bytes(), op);
|
||||
}
|
||||
}
|
||||
|
@ -5674,8 +5689,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
final Set<Long> seqNosInSafeCommit = operationsInSafeCommit.stream().map(op -> op.seqNo()).collect(Collectors.toSet());
|
||||
for (Engine.Operation op : operations) {
|
||||
assertThat(
|
||||
"seq_no=" + op.seqNo() + " max_seq_no=" + tracker.getMaxSeqNo() + " checkpoint=" + tracker.getCheckpoint(),
|
||||
tracker.contains(op.seqNo()), equalTo(seqNosInSafeCommit.contains(op.seqNo())));
|
||||
"seq_no=" + op.seqNo() + " max_seq_no=" + tracker.getMaxSeqNo() + "checkpoint=" + tracker.getProcessedCheckpoint(),
|
||||
tracker.hasProcessed(op.seqNo()), equalTo(seqNosInSafeCommit.contains(op.seqNo())));
|
||||
}
|
||||
engine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
assertThat(getDocIds(engine, true), equalTo(docs));
|
||||
|
@ -5695,8 +5710,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
config(softDeletesEnabled, store, translogPath, newMergePolicy(), null, null, globalCheckpoint::get))) {
|
||||
List<Engine.Operation> ops = generateHistoryOnReplica(between(1, 100), randomBoolean(), randomBoolean(), randomBoolean());
|
||||
applyOperations(engine, ops);
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
engine.syncTranslog(); // to advance persisted checkpoint
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
engine.flush();
|
||||
docs = getDocIds(engine, true);
|
||||
}
|
||||
|
@ -5955,8 +5970,8 @@ public class InternalEngineTests extends EngineTestCase {
|
|||
for (Engine.Operation op : operations) {
|
||||
applyOperation(engine, op);
|
||||
if (randomBoolean()) {
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
engine.syncTranslog();
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
}
|
||||
if (randomInt(100) < 10) {
|
||||
engine.refresh("test");
|
||||
|
|
|
@ -161,7 +161,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
|
|||
int totalOps = 0;
|
||||
for (Engine.Operation op : operations) {
|
||||
// Engine skips deletes or indexes below the local checkpoint
|
||||
if (engine.getLocalCheckpoint() < op.seqNo() || op instanceof Engine.NoOp) {
|
||||
if (engine.getProcessedLocalCheckpoint() < op.seqNo() || op instanceof Engine.NoOp) {
|
||||
seqNoToTerm.put(op.seqNo(), op.primaryTerm());
|
||||
if (op instanceof Engine.Index) {
|
||||
totalOps += ((Engine.Index) op).docs().size();
|
||||
|
@ -228,7 +228,7 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
|
|||
readyLatch.countDown();
|
||||
readyLatch.await();
|
||||
concurrentlyApplyOps(operations, engine);
|
||||
assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(operations.size() - 1L));
|
||||
assertThat(engine.getLocalCheckpointTracker().getProcessedCheckpoint(), equalTo(operations.size() - 1L));
|
||||
isDone.set(true);
|
||||
for (Follower follower : followers) {
|
||||
follower.join();
|
||||
|
@ -237,13 +237,13 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
|
|||
}
|
||||
|
||||
class Follower extends Thread {
|
||||
private final Engine leader;
|
||||
private final InternalEngine leader;
|
||||
private final InternalEngine engine;
|
||||
private final TranslogHandler translogHandler;
|
||||
private final AtomicBoolean isDone;
|
||||
private final CountDownLatch readLatch;
|
||||
|
||||
Follower(Engine leader, AtomicBoolean isDone, CountDownLatch readLatch) throws IOException {
|
||||
Follower(InternalEngine leader, AtomicBoolean isDone, CountDownLatch readLatch) throws IOException {
|
||||
this.leader = leader;
|
||||
this.isDone = isDone;
|
||||
this.readLatch = readLatch;
|
||||
|
@ -252,9 +252,9 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
|
|||
this.engine = createEngine(createStore(), createTempDir());
|
||||
}
|
||||
|
||||
void pullOperations(Engine follower) throws IOException {
|
||||
long leaderCheckpoint = leader.getLocalCheckpoint();
|
||||
long followerCheckpoint = follower.getLocalCheckpoint();
|
||||
void pullOperations(InternalEngine follower) throws IOException {
|
||||
long leaderCheckpoint = leader.getLocalCheckpointTracker().getProcessedCheckpoint();
|
||||
long followerCheckpoint = follower.getLocalCheckpointTracker().getProcessedCheckpoint();
|
||||
if (followerCheckpoint < leaderCheckpoint) {
|
||||
long fromSeqNo = followerCheckpoint + 1;
|
||||
long batchSize = randomLongBetween(0, 100);
|
||||
|
@ -271,7 +271,8 @@ public class LuceneChangesSnapshotTests extends EngineTestCase {
|
|||
readLatch.countDown();
|
||||
readLatch.await();
|
||||
while (isDone.get() == false ||
|
||||
engine.getLocalCheckpointTracker().getCheckpoint() < leader.getLocalCheckpoint()) {
|
||||
engine.getLocalCheckpointTracker().getProcessedCheckpoint() <
|
||||
leader.getLocalCheckpointTracker().getProcessedCheckpoint()) {
|
||||
pullOperations(engine);
|
||||
}
|
||||
assertConsistentHistoryBetweenTranslogAndLuceneIndex(engine, mapperService);
|
||||
|
|
|
@ -85,12 +85,12 @@ public class NoOpEngineTests extends EngineTestCase {
|
|||
|
||||
flushAndTrimTranslog(engine);
|
||||
|
||||
long localCheckpoint = engine.getLocalCheckpoint();
|
||||
long localCheckpoint = engine.getPersistedLocalCheckpoint();
|
||||
long maxSeqNo = engine.getSeqNoStats(100L).getMaxSeqNo();
|
||||
engine.close();
|
||||
|
||||
final NoOpEngine noOpEngine = new NoOpEngine(noOpConfig(INDEX_SETTINGS, store, primaryTranslogDir, tracker));
|
||||
assertThat(noOpEngine.getLocalCheckpoint(), equalTo(localCheckpoint));
|
||||
assertThat(noOpEngine.getPersistedLocalCheckpoint(), equalTo(localCheckpoint));
|
||||
assertThat(noOpEngine.getSeqNoStats(100L).getMaxSeqNo(), equalTo(maxSeqNo));
|
||||
try (Engine.IndexCommitRef ref = noOpEngine.acquireLastIndexCommit(false)) {
|
||||
try (IndexReader reader = DirectoryReader.open(ref.getIndexCommit())) {
|
||||
|
@ -114,7 +114,8 @@ public class NoOpEngineTests extends EngineTestCase {
|
|||
if (rarely()) {
|
||||
engine.flush();
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // advance persisted local checkpoint
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
}
|
||||
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
|
@ -122,11 +123,12 @@ public class NoOpEngineTests extends EngineTestCase {
|
|||
String delId = Integer.toString(i);
|
||||
Engine.DeleteResult result = engine.delete(new Engine.Delete("test", delId, newUid(delId), primaryTerm.get()));
|
||||
assertTrue(result.isFound());
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // advance persisted local checkpoint
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
deletions += 1;
|
||||
}
|
||||
}
|
||||
engine.getLocalCheckpointTracker().waitForOpsToComplete(numDocs + deletions - 1);
|
||||
engine.getLocalCheckpointTracker().waitForProcessedOpsToComplete(numDocs + deletions - 1);
|
||||
flushAndTrimTranslog(engine);
|
||||
}
|
||||
|
||||
|
|
|
@ -62,15 +62,16 @@ public class ReadOnlyEngineTests extends EngineTestCase {
|
|||
if (rarely()) {
|
||||
engine.flush();
|
||||
}
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
}
|
||||
engine.syncTranslog();
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getPersistedLocalCheckpoint()));
|
||||
engine.flush();
|
||||
readOnlyEngine = new ReadOnlyEngine(engine.engineConfig, engine.getSeqNoStats(globalCheckpoint.get()),
|
||||
engine.getTranslogStats(), false, Function.identity());
|
||||
lastSeqNoStats = engine.getSeqNoStats(globalCheckpoint.get());
|
||||
lastDocIds = getDocIds(engine, true);
|
||||
assertThat(readOnlyEngine.getLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getPersistedLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getSeqNoStats(globalCheckpoint.get()).getMaxSeqNo(), equalTo(lastSeqNoStats.getMaxSeqNo()));
|
||||
assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds));
|
||||
for (int i = 0; i < numDocs; i++) {
|
||||
|
@ -94,7 +95,7 @@ public class ReadOnlyEngineTests extends EngineTestCase {
|
|||
|
||||
IOUtils.close(external, internal);
|
||||
// the locked down engine should still point to the previous commit
|
||||
assertThat(readOnlyEngine.getLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getPersistedLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getSeqNoStats(globalCheckpoint.get()).getMaxSeqNo(), equalTo(lastSeqNoStats.getMaxSeqNo()));
|
||||
assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds));
|
||||
try (Engine.GetResult getResult = readOnlyEngine.get(get, readOnlyEngine::acquireSearcher)) {
|
||||
|
@ -105,7 +106,7 @@ public class ReadOnlyEngineTests extends EngineTestCase {
|
|||
try (InternalEngine recoveringEngine = new InternalEngine(config)) {
|
||||
recoveringEngine.recoverFromTranslog(translogHandler, Long.MAX_VALUE);
|
||||
// the locked down engine should still point to the previous commit
|
||||
assertThat(readOnlyEngine.getLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getPersistedLocalCheckpoint(), equalTo(lastSeqNoStats.getLocalCheckpoint()));
|
||||
assertThat(readOnlyEngine.getSeqNoStats(globalCheckpoint.get()).getMaxSeqNo(), equalTo(lastSeqNoStats.getMaxSeqNo()));
|
||||
assertThat(getDocIds(readOnlyEngine, false), equalTo(lastDocIds));
|
||||
}
|
||||
|
@ -129,9 +130,10 @@ public class ReadOnlyEngineTests extends EngineTestCase {
|
|||
if (rarely()) {
|
||||
engine.flush();
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
engine.syncTranslog(); // advance persisted local checkpoint
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint());
|
||||
engine.syncTranslog();
|
||||
engine.flushAndClose();
|
||||
readOnlyEngine = new ReadOnlyEngine(engine.engineConfig, null , null, true, Function.identity());
|
||||
|
@ -155,10 +157,10 @@ public class ReadOnlyEngineTests extends EngineTestCase {
|
|||
ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null);
|
||||
engine.index(new Engine.Index(newUid(doc), doc, i, primaryTerm.get(), 1, null, Engine.Operation.Origin.REPLICA,
|
||||
System.nanoTime(), -1, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0));
|
||||
maxSeqNo = engine.getLocalCheckpoint();
|
||||
maxSeqNo = engine.getProcessedLocalCheckpoint();
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint() - 1);
|
||||
engine.syncTranslog();
|
||||
globalCheckpoint.set(engine.getPersistedLocalCheckpoint() - 1);
|
||||
engine.flushAndClose();
|
||||
|
||||
IllegalStateException exception = expectThrows(IllegalStateException.class,
|
||||
|
|
|
@ -596,10 +596,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
final long expectedDocs = docs + 2L;
|
||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
// recovery has not completed, therefore the global checkpoint can have advanced on the primary
|
||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
assertThat(shards.getPrimary().getLastKnownGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
// the pending document is not done, the checkpoints can not have advanced on the replica
|
||||
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 1));
|
||||
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 1));
|
||||
assertThat(replica.getLastKnownGlobalCheckpoint(), lessThan(expectedDocs - 1));
|
||||
}
|
||||
|
||||
// wait for recovery to enter the translog phase
|
||||
|
@ -612,9 +612,9 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
final long expectedDocs = docs + 3L;
|
||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
// recovery is now in the process of being completed, therefore the global checkpoint can not have advanced on the primary
|
||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 2));
|
||||
assertThat(shards.getPrimary().getLastKnownGlobalCheckpoint(), equalTo(expectedDocs - 2));
|
||||
assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 2));
|
||||
assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 2));
|
||||
assertThat(replica.getLastKnownGlobalCheckpoint(), lessThan(expectedDocs - 2));
|
||||
}
|
||||
|
||||
replicaEngineFactory.releaseLatchedIndexers();
|
||||
|
@ -624,10 +624,10 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
final long expectedDocs = docs + 3L;
|
||||
assertBusy(() -> {
|
||||
assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
assertThat(shards.getPrimary().getLastKnownGlobalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
assertThat(replica.getLocalCheckpoint(), equalTo(expectedDocs - 1));
|
||||
// the global checkpoint advances can only advance here if a background global checkpoint sync fires
|
||||
assertThat(replica.getGlobalCheckpoint(), anyOf(equalTo(expectedDocs - 1), equalTo(expectedDocs - 2)));
|
||||
assertThat(replica.getLastKnownGlobalCheckpoint(), anyOf(equalTo(expectedDocs - 1), equalTo(expectedDocs - 2)));
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -762,7 +762,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
|
|||
}
|
||||
shards.refresh("test");
|
||||
List<DocIdSeqNoAndSource> docsBelowGlobalCheckpoint = EngineTestCase.getDocIds(getEngine(newPrimary), randomBoolean())
|
||||
.stream().filter(doc -> doc.getSeqNo() <= newPrimary.getGlobalCheckpoint()).collect(Collectors.toList());
|
||||
.stream().filter(doc -> doc.getSeqNo() <= newPrimary.getLastKnownGlobalCheckpoint()).collect(Collectors.toList());
|
||||
CountDownLatch latch = new CountDownLatch(1);
|
||||
final AtomicBoolean done = new AtomicBoolean();
|
||||
Thread thread = new Thread(() -> {
|
||||
|
|
|
@ -100,7 +100,7 @@ public class GlobalCheckpointSyncActionTests extends ESTestCase {
|
|||
lastSyncedGlobalCheckpoint = globalCheckpoint;
|
||||
}
|
||||
|
||||
when(indexShard.getGlobalCheckpoint()).thenReturn(globalCheckpoint);
|
||||
when(indexShard.getLastKnownGlobalCheckpoint()).thenReturn(globalCheckpoint);
|
||||
when(indexShard.getLastSyncedGlobalCheckpoint()).thenReturn(lastSyncedGlobalCheckpoint);
|
||||
|
||||
final GlobalCheckpointSyncAction action = new GlobalCheckpointSyncAction(
|
||||
|
|
|
@ -30,6 +30,8 @@ import org.elasticsearch.common.settings.Settings;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.translog.Translog;
|
||||
import org.elasticsearch.plugins.Plugin;
|
||||
import org.elasticsearch.test.ESIntegTestCase;
|
||||
import org.elasticsearch.test.InternalSettingsPlugin;
|
||||
|
@ -58,6 +60,28 @@ public class GlobalCheckpointSyncIT extends ESIntegTestCase {
|
|||
.collect(Collectors.toList());
|
||||
}
|
||||
|
||||
public void testGlobalCheckpointSyncWithAsyncDurability() throws Exception {
|
||||
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||
prepareCreate(
|
||||
"test",
|
||||
Settings.builder()
|
||||
.put(IndexService.GLOBAL_CHECKPOINT_SYNC_INTERVAL_SETTING.getKey(), "1s")
|
||||
.put(IndexSettings.INDEX_TRANSLOG_DURABILITY_SETTING.getKey(), Translog.Durability.ASYNC)
|
||||
.put(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey(), "1s")
|
||||
.put("index.number_of_replicas", 1))
|
||||
.get();
|
||||
|
||||
for (int j = 0; j < 10; j++) {
|
||||
final String id = Integer.toString(j);
|
||||
client().prepareIndex("test", "test", id).setSource("{\"foo\": " + id + "}", XContentType.JSON).get();
|
||||
}
|
||||
|
||||
assertBusy(() -> {
|
||||
SeqNoStats seqNoStats = client().admin().indices().prepareStats("test").get().getIndex("test").getShards()[0].getSeqNoStats();
|
||||
assertThat(seqNoStats.getGlobalCheckpoint(), equalTo(seqNoStats.getMaxSeqNo()));
|
||||
});
|
||||
}
|
||||
|
||||
public void testPostOperationGlobalCheckpointSync() throws Exception {
|
||||
// set the sync interval high so it does not execute during this test
|
||||
runGlobalCheckpointSyncTest(TimeValue.timeValueHours(24), client -> {}, client -> {});
|
||||
|
|
|
@ -55,43 +55,79 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
tracker = createEmptyTracker();
|
||||
}
|
||||
|
||||
public void testSimplePrimary() {
|
||||
public void testSimplePrimaryProcessed() {
|
||||
long seqNo1, seqNo2;
|
||||
assertThat(tracker.getCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
seqNo1 = tracker.generateSeqNo();
|
||||
assertThat(seqNo1, equalTo(0L));
|
||||
tracker.markSeqNoAsCompleted(seqNo1);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.contains(0L), equalTo(true));
|
||||
assertThat(tracker.contains(atLeast(1)), equalTo(false));
|
||||
tracker.markSeqNoAsProcessed(seqNo1);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.hasProcessed(0L), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(atLeast(1)), equalTo(false));
|
||||
seqNo1 = tracker.generateSeqNo();
|
||||
seqNo2 = tracker.generateSeqNo();
|
||||
assertThat(seqNo1, equalTo(1L));
|
||||
assertThat(seqNo2, equalTo(2L));
|
||||
tracker.markSeqNoAsCompleted(seqNo2);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.contains(seqNo1), equalTo(false));
|
||||
assertThat(tracker.contains(seqNo2), equalTo(true));
|
||||
tracker.markSeqNoAsCompleted(seqNo1);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.contains(between(0, 2)), equalTo(true));
|
||||
assertThat(tracker.contains(atLeast(3)), equalTo(false));
|
||||
tracker.markSeqNoAsProcessed(seqNo2);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.hasProcessed(seqNo1), equalTo(false));
|
||||
assertThat(tracker.hasProcessed(seqNo2), equalTo(true));
|
||||
tracker.markSeqNoAsProcessed(seqNo1);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.hasProcessed(between(0, 2)), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(atLeast(3)), equalTo(false));
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(2L));
|
||||
}
|
||||
|
||||
public void testSimpleReplica() {
|
||||
assertThat(tracker.getCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.contains(randomNonNegativeLong()), equalTo(false));
|
||||
tracker.markSeqNoAsCompleted(0L);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.contains(0), equalTo(true));
|
||||
tracker.markSeqNoAsCompleted(2L);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.contains(1L), equalTo(false));
|
||||
assertThat(tracker.contains(2L), equalTo(true));
|
||||
tracker.markSeqNoAsCompleted(1L);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.contains(between(0, 2)), equalTo(true));
|
||||
assertThat(tracker.contains(atLeast(3)), equalTo(false));
|
||||
public void testSimplePrimaryPersisted() {
|
||||
long seqNo1, seqNo2;
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
seqNo1 = tracker.generateSeqNo();
|
||||
assertThat(seqNo1, equalTo(0L));
|
||||
tracker.markSeqNoAsPersisted(seqNo1);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(0L));
|
||||
seqNo1 = tracker.generateSeqNo();
|
||||
seqNo2 = tracker.generateSeqNo();
|
||||
assertThat(seqNo1, equalTo(1L));
|
||||
assertThat(seqNo2, equalTo(2L));
|
||||
tracker.markSeqNoAsPersisted(seqNo2);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(0L));
|
||||
tracker.markSeqNoAsPersisted(seqNo1);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(2L));
|
||||
}
|
||||
|
||||
public void testSimpleReplicaProcessed() {
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.hasProcessed(randomNonNegativeLong()), equalTo(false));
|
||||
tracker.markSeqNoAsProcessed(0L);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.hasProcessed(0), equalTo(true));
|
||||
tracker.markSeqNoAsProcessed(2L);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(0L));
|
||||
assertThat(tracker.hasProcessed(1L), equalTo(false));
|
||||
assertThat(tracker.hasProcessed(2L), equalTo(true));
|
||||
tracker.markSeqNoAsProcessed(1L);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.hasProcessed(between(0, 2)), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(atLeast(3)), equalTo(false));
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(2L));
|
||||
}
|
||||
|
||||
public void testSimpleReplicaPersisted() {
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.hasProcessed(randomNonNegativeLong()), equalTo(false));
|
||||
tracker.markSeqNoAsPersisted(0L);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(0L));
|
||||
tracker.markSeqNoAsPersisted(2L);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(0L));
|
||||
tracker.markSeqNoAsPersisted(1L);
|
||||
assertThat(tracker.getPersistedCheckpoint(), equalTo(2L));
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(2L));
|
||||
}
|
||||
|
||||
public void testLazyInitialization() {
|
||||
|
@ -100,10 +136,10 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
* sequence numbers this could lead to excessive memory usage resulting in out of memory errors.
|
||||
*/
|
||||
long seqNo = randomNonNegativeLong();
|
||||
tracker.markSeqNoAsCompleted(seqNo);
|
||||
tracker.markSeqNoAsProcessed(seqNo);
|
||||
assertThat(tracker.processedSeqNo.size(), equalTo(1));
|
||||
assertThat(tracker.contains(seqNo), equalTo(true));
|
||||
assertThat(tracker.contains(randomValueOtherThan(seqNo, ESTestCase::randomNonNegativeLong)), equalTo(false));
|
||||
assertThat(tracker.hasProcessed(seqNo), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(randomValueOtherThan(seqNo, ESTestCase::randomNonNegativeLong)), equalTo(false));
|
||||
assertThat(tracker.processedSeqNo.size(), equalTo(1));
|
||||
}
|
||||
|
||||
|
@ -117,16 +153,16 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
}
|
||||
Collections.shuffle(seqNoList, random());
|
||||
for (Long seqNo : seqNoList) {
|
||||
tracker.markSeqNoAsCompleted(seqNo);
|
||||
tracker.markSeqNoAsProcessed(seqNo);
|
||||
}
|
||||
assertThat(tracker.checkpoint, equalTo(maxOps - 1L));
|
||||
assertThat(tracker.processedCheckpoint.get(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.processedSeqNo.size(), equalTo(aligned ? 0 : 1));
|
||||
if (aligned == false) {
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.checkpoint / BIT_SET_SIZE));
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.processedCheckpoint.get() / BIT_SET_SIZE));
|
||||
}
|
||||
assertThat(tracker.contains(randomFrom(seqNoList)), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(randomFrom(seqNoList)), equalTo(true));
|
||||
final long notCompletedSeqNo = randomValueOtherThanMany(seqNoList::contains, ESTestCase::randomNonNegativeLong);
|
||||
assertThat(tracker.contains(notCompletedSeqNo), equalTo(false));
|
||||
assertThat(tracker.hasProcessed(notCompletedSeqNo), equalTo(false));
|
||||
}
|
||||
|
||||
public void testConcurrentPrimary() throws InterruptedException {
|
||||
|
@ -151,7 +187,7 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
long seqNo = tracker.generateSeqNo();
|
||||
logger.info("[t{}] started [{}]", threadId, seqNo);
|
||||
if (seqNo != unFinishedSeq) {
|
||||
tracker.markSeqNoAsCompleted(seqNo);
|
||||
tracker.markSeqNoAsProcessed(seqNo);
|
||||
logger.info("[t{}] completed [{}]", threadId, seqNo);
|
||||
}
|
||||
}
|
||||
|
@ -163,12 +199,12 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
thread.join();
|
||||
}
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.getCheckpoint(), equalTo(unFinishedSeq - 1L));
|
||||
tracker.markSeqNoAsCompleted(unFinishedSeq);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(unFinishedSeq - 1L));
|
||||
tracker.markSeqNoAsProcessed(unFinishedSeq);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.processedSeqNo.size(), isOneOf(0, 1));
|
||||
if (tracker.processedSeqNo.size() == 1) {
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.checkpoint / BIT_SET_SIZE));
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.processedCheckpoint.get() / BIT_SET_SIZE));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -202,7 +238,7 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
Integer[] ops = seqNoPerThread[threadId];
|
||||
for (int seqNo : ops) {
|
||||
if (seqNo != unFinishedSeq) {
|
||||
tracker.markSeqNoAsCompleted(seqNo);
|
||||
tracker.markSeqNoAsProcessed(seqNo);
|
||||
logger.info("[t{}] completed [{}]", threadId, seqNo);
|
||||
}
|
||||
}
|
||||
|
@ -214,15 +250,15 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
thread.join();
|
||||
}
|
||||
assertThat(tracker.getMaxSeqNo(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.getCheckpoint(), equalTo(unFinishedSeq - 1L));
|
||||
assertThat(tracker.contains(unFinishedSeq), equalTo(false));
|
||||
tracker.markSeqNoAsCompleted(unFinishedSeq);
|
||||
assertThat(tracker.getCheckpoint(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.contains(unFinishedSeq), equalTo(true));
|
||||
assertThat(tracker.contains(randomLongBetween(maxOps, Long.MAX_VALUE)), equalTo(false));
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(unFinishedSeq - 1L));
|
||||
assertThat(tracker.hasProcessed(unFinishedSeq), equalTo(false));
|
||||
tracker.markSeqNoAsProcessed(unFinishedSeq);
|
||||
assertThat(tracker.getProcessedCheckpoint(), equalTo(maxOps - 1L));
|
||||
assertThat(tracker.hasProcessed(unFinishedSeq), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(randomLongBetween(maxOps, Long.MAX_VALUE)), equalTo(false));
|
||||
assertThat(tracker.processedSeqNo.size(), isOneOf(0, 1));
|
||||
if (tracker.processedSeqNo.size() == 1) {
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.checkpoint / BIT_SET_SIZE));
|
||||
assertThat(tracker.processedSeqNo.keys().iterator().next().value, equalTo(tracker.processedCheckpoint.get() / BIT_SET_SIZE));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -234,7 +270,7 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
try {
|
||||
// sychronize starting with the test thread
|
||||
barrier.await();
|
||||
tracker.waitForOpsToComplete(seqNo);
|
||||
tracker.waitForProcessedOpsToComplete(seqNo);
|
||||
complete.set(true);
|
||||
// synchronize with the test thread checking if we are no longer waiting
|
||||
barrier.await();
|
||||
|
@ -251,11 +287,11 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
final List<Integer> elements = IntStream.rangeClosed(0, seqNo).boxed().collect(Collectors.toList());
|
||||
Randomness.shuffle(elements);
|
||||
for (int i = 0; i < elements.size() - 1; i++) {
|
||||
tracker.markSeqNoAsCompleted(elements.get(i));
|
||||
tracker.markSeqNoAsProcessed(elements.get(i));
|
||||
assertFalse(complete.get());
|
||||
}
|
||||
|
||||
tracker.markSeqNoAsCompleted(elements.get(elements.size() - 1));
|
||||
tracker.markSeqNoAsProcessed(elements.get(elements.size() - 1));
|
||||
// synchronize with the waiting thread to mark that it is complete
|
||||
barrier.await();
|
||||
assertTrue(complete.get());
|
||||
|
@ -268,17 +304,17 @@ public class LocalCheckpointTrackerTests extends ESTestCase {
|
|||
final long localCheckpoint = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, maxSeqNo);
|
||||
final LocalCheckpointTracker tracker = new LocalCheckpointTracker(maxSeqNo, localCheckpoint);
|
||||
if (localCheckpoint >= 0) {
|
||||
assertThat(tracker.contains(randomLongBetween(0, localCheckpoint)), equalTo(true));
|
||||
assertThat(tracker.hasProcessed(randomLongBetween(0, localCheckpoint)), equalTo(true));
|
||||
}
|
||||
assertThat(tracker.contains(randomLongBetween(localCheckpoint + 1, Long.MAX_VALUE)), equalTo(false));
|
||||
assertThat(tracker.hasProcessed(randomLongBetween(localCheckpoint + 1, Long.MAX_VALUE)), equalTo(false));
|
||||
final int numOps = between(1, 100);
|
||||
final List<Long> seqNos = new ArrayList<>();
|
||||
for (int i = 0; i < numOps; i++) {
|
||||
long seqNo = randomLongBetween(0, 1000);
|
||||
seqNos.add(seqNo);
|
||||
tracker.markSeqNoAsCompleted(seqNo);
|
||||
tracker.markSeqNoAsProcessed(seqNo);
|
||||
}
|
||||
final long seqNo = randomNonNegativeLong();
|
||||
assertThat(tracker.contains(seqNo), equalTo(seqNo <= localCheckpoint || seqNos.contains(seqNo)));
|
||||
assertThat(tracker.hasProcessed(seqNo), equalTo(seqNo <= localCheckpoint || seqNos.contains(seqNo)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -419,7 +419,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
}
|
||||
indexShard.acquireReplicaOperationPermit(
|
||||
indexShard.getPendingPrimaryTerm(),
|
||||
indexShard.getGlobalCheckpoint(),
|
||||
indexShard.getLastKnownGlobalCheckpoint(),
|
||||
indexShard.getMaxSeqNoOfUpdatesOrDeletes(),
|
||||
new ActionListener<Releasable>() {
|
||||
@Override
|
||||
|
@ -717,7 +717,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
|
||||
if (Assertions.ENABLED && indexShard.routingEntry().isRelocationTarget() == false) {
|
||||
assertThat(expectThrows(AssertionError.class, () -> indexShard.acquireReplicaOperationPermit(pendingPrimaryTerm,
|
||||
indexShard.getGlobalCheckpoint(), indexShard.getMaxSeqNoOfUpdatesOrDeletes(), new ActionListener<Releasable>() {
|
||||
indexShard.getLastKnownGlobalCheckpoint(), indexShard.getMaxSeqNoOfUpdatesOrDeletes(), new ActionListener<Releasable>() {
|
||||
@Override
|
||||
public void onResponse(Releasable releasable) {
|
||||
fail();
|
||||
|
@ -843,7 +843,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
private Releasable acquireReplicaOperationPermitBlockingly(IndexShard indexShard, long opPrimaryTerm)
|
||||
throws ExecutionException, InterruptedException {
|
||||
PlainActionFuture<Releasable> fut = new PlainActionFuture<>();
|
||||
indexShard.acquireReplicaOperationPermit(opPrimaryTerm, indexShard.getGlobalCheckpoint(),
|
||||
indexShard.acquireReplicaOperationPermit(opPrimaryTerm, indexShard.getLastKnownGlobalCheckpoint(),
|
||||
randomNonNegativeLong(), fut, ThreadPool.Names.WRITE, "");
|
||||
return fut.get();
|
||||
}
|
||||
|
@ -922,18 +922,19 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
final long newPrimaryTerm = primaryTerm + 1 + randomInt(20);
|
||||
if (engineClosed == false) {
|
||||
assertThat(indexShard.getLocalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(indexShard.getGlobalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
assertThat(indexShard.getLastKnownGlobalCheckpoint(), equalTo(SequenceNumbers.NO_OPS_PERFORMED));
|
||||
}
|
||||
final long newGlobalCheckPoint;
|
||||
if (engineClosed || randomBoolean()) {
|
||||
newGlobalCheckPoint = SequenceNumbers.NO_OPS_PERFORMED;
|
||||
} else {
|
||||
long localCheckPoint = indexShard.getGlobalCheckpoint() + randomInt(100);
|
||||
long localCheckPoint = indexShard.getLastKnownGlobalCheckpoint() + randomInt(100);
|
||||
// advance local checkpoint
|
||||
for (int i = 0; i <= localCheckPoint; i++) {
|
||||
indexShard.markSeqNoAsNoop(i, "dummy doc");
|
||||
}
|
||||
newGlobalCheckPoint = randomIntBetween((int) indexShard.getGlobalCheckpoint(), (int) localCheckPoint);
|
||||
indexShard.sync(); // advance local checkpoint
|
||||
newGlobalCheckPoint = randomIntBetween((int) indexShard.getLastKnownGlobalCheckpoint(), (int) localCheckPoint);
|
||||
}
|
||||
final long expectedLocalCheckpoint;
|
||||
if (newGlobalCheckPoint == UNASSIGNED_SEQ_NO) {
|
||||
|
@ -954,7 +955,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
assertThat(indexShard.getPendingPrimaryTerm(), equalTo(newPrimaryTerm));
|
||||
assertThat(TestTranslog.getCurrentTerm(getTranslog(indexShard)), equalTo(newPrimaryTerm));
|
||||
assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint));
|
||||
assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint));
|
||||
assertThat(indexShard.getLastKnownGlobalCheckpoint(), equalTo(newGlobalCheckPoint));
|
||||
onResponse.set(true);
|
||||
releasable.close();
|
||||
finish();
|
||||
|
@ -1023,7 +1024,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
// and one after replaying translog (upto the global checkpoint); otherwise we roll translog once.
|
||||
either(equalTo(translogGen + 1)).or(equalTo(translogGen + 2)));
|
||||
assertThat(indexShard.getLocalCheckpoint(), equalTo(expectedLocalCheckpoint));
|
||||
assertThat(indexShard.getGlobalCheckpoint(), equalTo(newGlobalCheckPoint));
|
||||
assertThat(indexShard.getLastKnownGlobalCheckpoint(), equalTo(newGlobalCheckPoint));
|
||||
}
|
||||
}
|
||||
thread.join();
|
||||
|
@ -1051,7 +1052,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
};
|
||||
|
||||
final long oldPrimaryTerm = indexShard.getPendingPrimaryTerm() - 1;
|
||||
randomReplicaOperationPermitAcquisition(indexShard, oldPrimaryTerm, indexShard.getGlobalCheckpoint(),
|
||||
randomReplicaOperationPermitAcquisition(indexShard, oldPrimaryTerm, indexShard.getLastKnownGlobalCheckpoint(),
|
||||
randomNonNegativeLong(), onLockAcquired, "");
|
||||
latch.await();
|
||||
assertFalse(onResponse.get());
|
||||
|
@ -1072,7 +1073,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
|
||||
long newMaxSeqNoOfUpdates = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE);
|
||||
PlainActionFuture<Releasable> fut = new PlainActionFuture<>();
|
||||
randomReplicaOperationPermitAcquisition(replica, replica.getOperationPrimaryTerm(), replica.getGlobalCheckpoint(),
|
||||
randomReplicaOperationPermitAcquisition(replica, replica.getOperationPrimaryTerm(), replica.getLastKnownGlobalCheckpoint(),
|
||||
newMaxSeqNoOfUpdates, fut, "");
|
||||
try (Releasable ignored = fut.actionGet()) {
|
||||
assertThat(replica.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(Math.max(currentMaxSeqNoOfUpdates, newMaxSeqNoOfUpdates)));
|
||||
|
@ -1117,18 +1118,18 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
primaryShard.updateLocalCheckpointForShard(replicaAllocationId, replicaLocalCheckpoint);
|
||||
|
||||
// initialize the local knowledge on the primary of the global checkpoint on the replica shard
|
||||
final int replicaGlobalCheckpoint =
|
||||
randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Math.toIntExact(primaryShard.getGlobalCheckpoint()));
|
||||
final int replicaGlobalCheckpoint = randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED),
|
||||
Math.toIntExact(primaryShard.getLastKnownGlobalCheckpoint()));
|
||||
primaryShard.updateGlobalCheckpointForShard(replicaAllocationId, replicaGlobalCheckpoint);
|
||||
|
||||
// simulate a background maybe sync; it should only run if the knowledge on the replica of the global checkpoint lags the primary
|
||||
primaryShard.maybeSyncGlobalCheckpoint("test");
|
||||
assertThat(
|
||||
synced.get(),
|
||||
equalTo(maxSeqNo == primaryShard.getGlobalCheckpoint() && (replicaGlobalCheckpoint < checkpoint)));
|
||||
equalTo(maxSeqNo == primaryShard.getLastKnownGlobalCheckpoint() && (replicaGlobalCheckpoint < checkpoint)));
|
||||
|
||||
// simulate that the background sync advanced the global checkpoint on the replica
|
||||
primaryShard.updateGlobalCheckpointForShard(replicaAllocationId, primaryShard.getGlobalCheckpoint());
|
||||
primaryShard.updateGlobalCheckpointForShard(replicaAllocationId, primaryShard.getLastKnownGlobalCheckpoint());
|
||||
|
||||
// reset our boolean so that we can assert after another simulated maybe sync
|
||||
synced.set(false);
|
||||
|
@ -1289,7 +1290,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
}
|
||||
indexShard.acquireReplicaOperationPermit(
|
||||
primaryTerm + increment,
|
||||
indexShard.getGlobalCheckpoint(),
|
||||
indexShard.getLastKnownGlobalCheckpoint(),
|
||||
randomNonNegativeLong(),
|
||||
new ActionListener<Releasable>() {
|
||||
@Override
|
||||
|
@ -1990,6 +1991,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
new SourceToParse(shard.shardId().getIndexName(), "_doc", "id-2", new BytesArray("{}"), XContentType.JSON));
|
||||
shard.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false,
|
||||
new SourceToParse(shard.shardId().getIndexName(), "_doc", "id-5", new BytesArray("{}"), XContentType.JSON));
|
||||
shard.sync(); // advance local checkpoint
|
||||
|
||||
final int translogOps;
|
||||
if (randomBoolean()) {
|
||||
|
@ -2921,6 +2923,8 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
// Need to update and sync the global checkpoint as the soft-deletes retention MergePolicy depends on it.
|
||||
if (indexShard.indexSettings.isSoftDeleteEnabled()) {
|
||||
if (indexShard.routingEntry().primary()) {
|
||||
indexShard.updateLocalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
||||
indexShard.getLocalCheckpoint());
|
||||
indexShard.updateGlobalCheckpointForShard(indexShard.routingEntry().allocationId().getId(),
|
||||
indexShard.getLocalCheckpoint());
|
||||
} else {
|
||||
|
@ -3306,6 +3310,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
indexShard.flush(new FlushRequest());
|
||||
}
|
||||
}
|
||||
indexShard.sync(); // advance local checkpoint
|
||||
assert localCheckpoint == indexShard.getLocalCheckpoint();
|
||||
assert !gap || (localCheckpoint != max);
|
||||
return new Result(localCheckpoint, max);
|
||||
|
@ -3753,7 +3758,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
IndexShard shard = newStartedShard(false);
|
||||
indexOnReplicaWithGaps(shard, between(0, 1000), Math.toIntExact(shard.getLocalCheckpoint()));
|
||||
long maxSeqNoBeforeRollback = shard.seqNoStats().getMaxSeqNo();
|
||||
final long globalCheckpoint = randomLongBetween(shard.getGlobalCheckpoint(), shard.getLocalCheckpoint());
|
||||
final long globalCheckpoint = randomLongBetween(shard.getLastKnownGlobalCheckpoint(), shard.getLocalCheckpoint());
|
||||
shard.updateGlobalCheckpointOnReplica(globalCheckpoint, "test");
|
||||
Set<String> docBelowGlobalCheckpoint = getShardDocUIDs(shard).stream()
|
||||
.filter(id -> Long.parseLong(id) <= globalCheckpoint).collect(Collectors.toSet());
|
||||
|
@ -3837,7 +3842,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
closeShardThread.start();
|
||||
|
||||
final CountDownLatch engineResetLatch = new CountDownLatch(1);
|
||||
shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getGlobalCheckpoint(), 0L,
|
||||
shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getLastKnownGlobalCheckpoint(), 0L,
|
||||
ActionListener.wrap(r -> {
|
||||
try (Releasable dummy = r) {
|
||||
shard.resetEngineToGlobalCheckpoint();
|
||||
|
@ -3877,7 +3882,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
});
|
||||
|
||||
indexOnReplicaWithGaps(shard, between(0, 1000), Math.toIntExact(shard.getLocalCheckpoint()));
|
||||
final long globalCheckpoint = randomLongBetween(shard.getGlobalCheckpoint(), shard.getLocalCheckpoint());
|
||||
final long globalCheckpoint = randomLongBetween(shard.getLastKnownGlobalCheckpoint(), shard.getLocalCheckpoint());
|
||||
shard.updateGlobalCheckpointOnReplica(globalCheckpoint, "test");
|
||||
|
||||
Thread snapshotThread = new Thread(() -> {
|
||||
|
@ -3900,7 +3905,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
snapshotThread.start();
|
||||
|
||||
final CountDownLatch engineResetLatch = new CountDownLatch(1);
|
||||
shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getGlobalCheckpoint(), 0L,
|
||||
shard.acquireAllReplicaOperationsPermits(shard.getOperationPrimaryTerm(), shard.getLastKnownGlobalCheckpoint(), 0L,
|
||||
ActionListener.wrap(r -> {
|
||||
try (Releasable dummy = r) {
|
||||
shard.resetEngineToGlobalCheckpoint();
|
||||
|
@ -3924,7 +3929,7 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
|
||||
for (int i = 0; i < nbTermUpdates; i++) {
|
||||
long opPrimaryTerm = replica.getOperationPrimaryTerm() + 1;
|
||||
final long globalCheckpoint = replica.getGlobalCheckpoint();
|
||||
final long globalCheckpoint = replica.getLastKnownGlobalCheckpoint();
|
||||
final long maxSeqNoOfUpdatesOrDeletes = replica.getMaxSeqNoOfUpdatesOrDeletes();
|
||||
|
||||
final int operations = scaledRandomIntBetween(5, 32);
|
||||
|
|
|
@ -101,7 +101,7 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase {
|
|||
shard.updateShardState(shard.routingEntry(), shard.getPendingPrimaryTerm(), null, 1000L, Collections.singleton(allocationId),
|
||||
new IndexShardRoutingTable.Builder(shard.shardId()).addShard(shard.routingEntry()).build());
|
||||
shard.updateLocalCheckpointForShard(allocationId, globalCheckPoint);
|
||||
assertEquals(globalCheckPoint, shard.getGlobalCheckpoint());
|
||||
assertEquals(globalCheckPoint, shard.getLastKnownGlobalCheckpoint());
|
||||
|
||||
logger.info("Total ops: {}, global checkpoint: {}", numDocs, globalCheckPoint);
|
||||
|
||||
|
@ -197,7 +197,7 @@ public class PrimaryReplicaSyncerTests extends IndexShardTestCase {
|
|||
|
||||
public void testDoNotSendOperationsWithoutSequenceNumber() throws Exception {
|
||||
IndexShard shard = spy(newStartedShard(true));
|
||||
when(shard.getGlobalCheckpoint()).thenReturn(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
when(shard.getLastKnownGlobalCheckpoint()).thenReturn(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
int numOps = between(0, 20);
|
||||
List<Translog.Operation> operations = new ArrayList<>();
|
||||
for (int i = 0; i < numOps; i++) {
|
||||
|
|
|
@ -160,7 +160,9 @@ public class CorruptedFileIT extends ESIntegTestCase {
|
|||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).execute().actionGet());
|
||||
// double flush to create safe commit in case of async durability
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).get());
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).get());
|
||||
// we have to flush at least once here since we don't corrupt the translog
|
||||
SearchResponse countResponse = client().prepareSearch().setSize(0).get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
@ -264,7 +266,9 @@ public class CorruptedFileIT extends ESIntegTestCase {
|
|||
}
|
||||
indexRandom(true, builders);
|
||||
ensureGreen();
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).execute().actionGet());
|
||||
// double flush to create safe commit in case of async durability
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).get());
|
||||
assertAllSuccessful(client().admin().indices().prepareFlush().setForce(true).get());
|
||||
// we have to flush at least once here since we don't corrupt the translog
|
||||
SearchResponse countResponse = client().prepareSearch().setSize(0).get();
|
||||
assertHitCount(countResponse, numDocs);
|
||||
|
|
|
@ -171,7 +171,7 @@ public class TranslogDeletionPolicyTests extends ESTestCase {
|
|||
}
|
||||
writer = TranslogWriter.create(new ShardId("index", "uuid", 0), translogUUID, gen,
|
||||
tempDir.resolve(Translog.getFilename(gen)), FileChannel::open, TranslogConfig.DEFAULT_BUFFER_SIZE, 1L, 1L, () -> 1L,
|
||||
() -> 1L, randomNonNegativeLong(), new TragicExceptionHolder());
|
||||
() -> 1L, randomNonNegativeLong(), new TragicExceptionHolder(), seqNo -> {});
|
||||
writer = Mockito.spy(writer);
|
||||
Mockito.doReturn(now - (numberOfReaders - gen + 1) * 1000).when(writer).getLastModifiedTime();
|
||||
|
||||
|
|
|
@ -113,6 +113,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.function.LongConsumer;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
@ -147,6 +148,7 @@ public class TranslogTests extends ESTestCase {
|
|||
protected Path translogDir;
|
||||
// A default primary term is used by translog instances created in this test.
|
||||
private final AtomicLong primaryTerm = new AtomicLong();
|
||||
private final AtomicReference<LongConsumer> persistedSeqNoConsumer = new AtomicReference<>();
|
||||
|
||||
@Override
|
||||
protected void afterIfSuccessful() throws Exception {
|
||||
|
@ -165,16 +167,25 @@ public class TranslogTests extends ESTestCase {
|
|||
|
||||
}
|
||||
|
||||
private LongConsumer getPersistedSeqNoConsumer() {
|
||||
return seqNo -> {
|
||||
final LongConsumer consumer = persistedSeqNoConsumer.get();
|
||||
if (consumer != null) {
|
||||
consumer.accept(seqNo);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
protected Translog createTranslog(TranslogConfig config) throws IOException {
|
||||
String translogUUID =
|
||||
Translog.createEmptyTranslog(config.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get());
|
||||
return new Translog(config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, getPersistedSeqNoConsumer());
|
||||
}
|
||||
|
||||
protected Translog openTranslog(TranslogConfig config, String translogUUID) throws IOException {
|
||||
return new Translog(config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, getPersistedSeqNoConsumer());
|
||||
}
|
||||
|
||||
|
||||
|
@ -226,7 +237,8 @@ public class TranslogTests extends ESTestCase {
|
|||
final TranslogConfig translogConfig = getTranslogConfig(path);
|
||||
final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings());
|
||||
final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get());
|
||||
return new Translog(translogConfig, translogUUID, deletionPolicy, () -> globalCheckpoint.get(), primaryTerm::get);
|
||||
return new Translog(translogConfig, translogUUID, deletionPolicy, () -> globalCheckpoint.get(), primaryTerm::get,
|
||||
getPersistedSeqNoConsumer());
|
||||
}
|
||||
|
||||
private TranslogConfig getTranslogConfig(final Path path) {
|
||||
|
@ -982,7 +994,7 @@ public class TranslogTests extends ESTestCase {
|
|||
throw new AssertionError("unsupported operation type [" + type + "]");
|
||||
}
|
||||
Translog.Location location = translog.add(op);
|
||||
tracker.markSeqNoAsCompleted(id);
|
||||
tracker.markSeqNoAsProcessed(id);
|
||||
Translog.Location existing = writtenOps.put(op, location);
|
||||
if (existing != null) {
|
||||
fail("duplicate op [" + op + "], old entry at " + location);
|
||||
|
@ -994,7 +1006,7 @@ public class TranslogTests extends ESTestCase {
|
|||
synchronized (flushMutex) {
|
||||
// we need not do this concurrently as we need to make sure that the generation
|
||||
// we're committing - is still present when we're committing
|
||||
long localCheckpoint = tracker.getCheckpoint();
|
||||
long localCheckpoint = tracker.getProcessedCheckpoint();
|
||||
translog.rollGeneration();
|
||||
// expose the new checkpoint (simulating a commit), before we trim the translog
|
||||
lastCommittedLocalCheckpoint.set(localCheckpoint);
|
||||
|
@ -1279,6 +1291,8 @@ public class TranslogTests extends ESTestCase {
|
|||
|
||||
public void testTranslogWriter() throws IOException {
|
||||
final TranslogWriter writer = translog.createWriter(translog.currentFileGeneration() + 1);
|
||||
final Set<Long> persistedSeqNos = new HashSet<>();
|
||||
persistedSeqNoConsumer.set(persistedSeqNos::add);
|
||||
final int numOps = randomIntBetween(8, 128);
|
||||
byte[] bytes = new byte[4];
|
||||
ByteArrayDataOutput out = new ByteArrayDataOutput(bytes);
|
||||
|
@ -1297,7 +1311,10 @@ public class TranslogTests extends ESTestCase {
|
|||
}
|
||||
writer.add(new BytesArray(bytes), seqNo);
|
||||
}
|
||||
assertThat(persistedSeqNos, empty());
|
||||
writer.sync();
|
||||
persistedSeqNos.remove(SequenceNumbers.UNASSIGNED_SEQ_NO);
|
||||
assertEquals(seenSeqNos, persistedSeqNos);
|
||||
|
||||
final BaseTranslogReader reader = randomBoolean() ? writer :
|
||||
translog.openReader(writer.path(), Checkpoint.read(translog.location().resolve(Translog.CHECKPOINT_FILE_NAME)));
|
||||
|
@ -1401,7 +1418,7 @@ public class TranslogTests extends ESTestCase {
|
|||
}
|
||||
} else {
|
||||
translog = new Translog(config, translogGeneration.translogUUID, translog.getDeletionPolicy(),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {});
|
||||
assertEquals("lastCommitted must be 1 less than current",
|
||||
translogGeneration.translogFileGeneration + 1, translog.currentFileGeneration());
|
||||
assertFalse(translog.syncNeeded());
|
||||
|
@ -1443,7 +1460,7 @@ public class TranslogTests extends ESTestCase {
|
|||
final String translogUUID = translog.getTranslogUUID();
|
||||
final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy();
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertNotNull(translogGeneration);
|
||||
assertEquals("lastCommitted must be 2 less than current - we never finished the commit",
|
||||
translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration());
|
||||
|
@ -1459,7 +1476,7 @@ public class TranslogTests extends ESTestCase {
|
|||
}
|
||||
if (randomBoolean()) { // recover twice
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertNotNull(translogGeneration);
|
||||
assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice",
|
||||
translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration());
|
||||
|
@ -1508,7 +1525,7 @@ public class TranslogTests extends ESTestCase {
|
|||
final String translogUUID = translog.getTranslogUUID();
|
||||
final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy();
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertNotNull(translogGeneration);
|
||||
assertEquals("lastCommitted must be 2 less than current - we never finished the commit",
|
||||
translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration());
|
||||
|
@ -1525,7 +1542,7 @@ public class TranslogTests extends ESTestCase {
|
|||
|
||||
if (randomBoolean()) { // recover twice
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertNotNull(translogGeneration);
|
||||
assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice",
|
||||
translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration());
|
||||
|
@ -1573,7 +1590,7 @@ public class TranslogTests extends ESTestCase {
|
|||
final String translogUUID = translog.getTranslogUUID();
|
||||
final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy();
|
||||
try (Translog ignored = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
fail("corrupted");
|
||||
} catch (IllegalStateException ex) {
|
||||
assertEquals("Checkpoint file translog-3.ckp already exists but has corrupted content expected: Checkpoint{offset=3025, " +
|
||||
|
@ -1584,7 +1601,7 @@ public class TranslogTests extends ESTestCase {
|
|||
Checkpoint.write(FileChannel::open, config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)),
|
||||
read, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING);
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertNotNull(translogGeneration);
|
||||
assertEquals("lastCommitted must be 2 less than current - we never finished the commit",
|
||||
translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration());
|
||||
|
@ -1853,12 +1870,14 @@ public class TranslogTests extends ESTestCase {
|
|||
final String foreignTranslog = randomRealisticUnicodeOfCodepointLengthBetween(1,
|
||||
translogGeneration.translogUUID.length());
|
||||
try {
|
||||
new Translog(config, foreignTranslog, createTranslogDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
new Translog(config, foreignTranslog, createTranslogDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {});
|
||||
fail("translog doesn't belong to this UUID");
|
||||
} catch (TranslogCorruptedException ex) {
|
||||
|
||||
}
|
||||
this.translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
this.translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {});
|
||||
try (Translog.Snapshot snapshot = this.translog.newSnapshotFromGen(translogGeneration, Long.MAX_VALUE)) {
|
||||
for (int i = firstUncommitted; i < translogOperations; i++) {
|
||||
Translog.Operation next = snapshot.next();
|
||||
|
@ -2052,7 +2071,7 @@ public class TranslogTests extends ESTestCase {
|
|||
final String translogUUID = translog.getTranslogUUID();
|
||||
final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy();
|
||||
try (Translog tlog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertEquals("lastCommitted must be 1 less than current",
|
||||
translogGeneration.translogFileGeneration + 1, tlog.currentFileGeneration());
|
||||
assertFalse(tlog.syncNeeded());
|
||||
|
@ -2191,7 +2210,7 @@ public class TranslogTests extends ESTestCase {
|
|||
writtenOperations.removeIf(next -> checkpoint.offset < (next.location.translogLocation + next.location.size));
|
||||
try (Translog tlog =
|
||||
new Translog(config, translogUUID, createTranslogDeletionPolicy(),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {});
|
||||
Translog.Snapshot snapshot = tlog.newSnapshot()) {
|
||||
if (writtenOperations.size() != snapshot.totalOperations()) {
|
||||
for (int i = 0; i < threadCount; i++) {
|
||||
|
@ -2241,7 +2260,7 @@ public class TranslogTests extends ESTestCase {
|
|||
deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE));
|
||||
deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration);
|
||||
translog = new Translog(config, translog.getTranslogUUID(), deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {});
|
||||
assertThat(translog.getMinFileGeneration(), equalTo(1L));
|
||||
// no trimming done yet, just recovered
|
||||
for (long gen = 1; gen < translog.currentFileGeneration(); gen++) {
|
||||
|
@ -2300,7 +2319,7 @@ public class TranslogTests extends ESTestCase {
|
|||
deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE));
|
||||
deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration);
|
||||
try (Translog translog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
// we don't know when things broke exactly
|
||||
assertThat(translog.getMinFileGeneration(), greaterThanOrEqualTo(1L));
|
||||
assertThat(translog.getMinFileGeneration(), lessThanOrEqualTo(comittedGeneration));
|
||||
|
@ -2382,7 +2401,8 @@ public class TranslogTests extends ESTestCase {
|
|||
translogUUID = Translog.createEmptyTranslog(
|
||||
config.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId, channelFactory, primaryTerm.get());
|
||||
}
|
||||
return new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get) {
|
||||
return new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {}) {
|
||||
@Override
|
||||
ChannelFactory getChannelFactory() {
|
||||
return channelFactory;
|
||||
|
@ -2496,9 +2516,10 @@ public class TranslogTests extends ESTestCase {
|
|||
translog.close();
|
||||
try {
|
||||
new Translog(config, translog.getTranslogUUID(), createTranslogDeletionPolicy(),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {}) {
|
||||
@Override
|
||||
protected TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint)
|
||||
protected TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint,
|
||||
LongConsumer persistedSequenceNumberConsumer)
|
||||
throws IOException {
|
||||
throw new MockDirectoryWrapper.FakeIOException();
|
||||
}
|
||||
|
@ -2559,7 +2580,7 @@ public class TranslogTests extends ESTestCase {
|
|||
Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 1) + ".tlog"));
|
||||
|
||||
TranslogException ex = expectThrows(TranslogException.class, () -> new Translog(config, translog.getTranslogUUID(),
|
||||
translog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get));
|
||||
translog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {}));
|
||||
assertEquals(ex.getMessage(), "failed to create new translog file");
|
||||
assertEquals(ex.getCause().getClass(), FileAlreadyExistsException.class);
|
||||
}
|
||||
|
@ -2579,7 +2600,7 @@ public class TranslogTests extends ESTestCase {
|
|||
// we add N+1 and N+2 to ensure we only delete the N+1 file and never jump ahead and wipe without the right condition
|
||||
Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 2) + ".tlog"));
|
||||
try (Translog tlog = new Translog(config, translogUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
assertFalse(tlog.syncNeeded());
|
||||
try (Translog.Snapshot snapshot = tlog.newSnapshot()) {
|
||||
for (int i = 0; i < 1; i++) {
|
||||
|
@ -2593,7 +2614,8 @@ public class TranslogTests extends ESTestCase {
|
|||
}
|
||||
|
||||
TranslogException ex = expectThrows(TranslogException.class,
|
||||
() -> new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get));
|
||||
() -> new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {}));
|
||||
assertEquals(ex.getMessage(), "failed to create new translog file");
|
||||
assertEquals(ex.getCause().getClass(), FileAlreadyExistsException.class);
|
||||
}
|
||||
|
@ -2706,7 +2728,7 @@ public class TranslogTests extends ESTestCase {
|
|||
SequenceNumbers.NO_OPS_PERFORMED, shardId, primaryTerm.get());
|
||||
}
|
||||
try (Translog translog = new Translog(config, generationUUID, deletionPolicy,
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {});
|
||||
Translog.Snapshot snapshot = translog.newSnapshotFromGen(
|
||||
new Translog.TranslogGeneration(generationUUID, minGenForRecovery), Long.MAX_VALUE)) {
|
||||
assertEquals(syncedDocs.size(), snapshot.totalOperations());
|
||||
|
@ -2773,14 +2795,16 @@ public class TranslogTests extends ESTestCase {
|
|||
final String translogUUID = translog.getTranslogUUID();
|
||||
final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(config.getIndexSettings());
|
||||
translog.close();
|
||||
translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {});
|
||||
translog.add(new Translog.Index("test", "2", 1, primaryTerm.get(), new byte[]{2}));
|
||||
translog.rollGeneration();
|
||||
Closeable lock = translog.acquireRetentionLock();
|
||||
translog.add(new Translog.Index("test", "3", 2, primaryTerm.get(), new byte[]{3}));
|
||||
translog.close();
|
||||
IOUtils.close(lock);
|
||||
translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get);
|
||||
translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get,
|
||||
seqNo -> {});
|
||||
}
|
||||
|
||||
public static Translog.Location randomTranslogLocation() {
|
||||
|
@ -3101,7 +3125,7 @@ public class TranslogTests extends ESTestCase {
|
|||
class MisbehavingTranslog extends Translog {
|
||||
MisbehavingTranslog(TranslogConfig config, String translogUUID, TranslogDeletionPolicy deletionPolicy,
|
||||
LongSupplier globalCheckpointSupplier, LongSupplier primaryTermSupplier) throws IOException {
|
||||
super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier);
|
||||
super(config, translogUUID, deletionPolicy, globalCheckpointSupplier, primaryTermSupplier, seqNo -> {});
|
||||
}
|
||||
|
||||
void callCloseDirectly() throws IOException {
|
||||
|
@ -3223,7 +3247,7 @@ public class TranslogTests extends ESTestCase {
|
|||
assertFalse(brokenTranslog.isOpen());
|
||||
|
||||
try (Translog recoveredTranslog = new Translog(getTranslogConfig(path), brokenTranslog.getTranslogUUID(),
|
||||
brokenTranslog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get)) {
|
||||
brokenTranslog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED, primaryTerm::get, seqNo -> {})) {
|
||||
recoveredTranslog.rollGeneration();
|
||||
assertFilePresences(recoveredTranslog);
|
||||
}
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentType;
|
|||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.engine.InternalEngine;
|
||||
import org.elasticsearch.index.engine.InternalEngineTests;
|
||||
import org.elasticsearch.index.mapper.ParsedDocument;
|
||||
import org.elasticsearch.index.mapper.Uid;
|
||||
|
@ -274,9 +275,10 @@ public class FlushIT extends ESIntegTestCase {
|
|||
private void indexDoc(Engine engine, String id) throws IOException {
|
||||
final ParsedDocument doc = InternalEngineTests.createParsedDoc(id, null);
|
||||
final Engine.IndexResult indexResult = engine.index(new Engine.Index(new Term("_id", Uid.encodeId(doc.id())), doc,
|
||||
engine.getLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, randomLong(), -1L, false,
|
||||
SequenceNumbers.UNASSIGNED_SEQ_NO, 0));
|
||||
((InternalEngine) engine).getProcessedLocalCheckpoint() + 1, 1L, 1L, null, Engine.Operation.Origin.REPLICA, System.nanoTime(),
|
||||
-1L, false, SequenceNumbers.UNASSIGNED_SEQ_NO, 0));
|
||||
assertThat(indexResult.getFailure(), nullValue());
|
||||
engine.syncTranslog();
|
||||
}
|
||||
|
||||
public void testSyncedFlushSkipOutOfSyncReplicas() throws Exception {
|
||||
|
|
|
@ -143,6 +143,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
|||
// index #2
|
||||
orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false,
|
||||
new SourceToParse(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON));
|
||||
orgReplica.sync(); // advance local checkpoint
|
||||
orgReplica.updateGlobalCheckpointOnReplica(3L, "test");
|
||||
// index #5 -> force NoOp #4.
|
||||
orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false,
|
||||
|
@ -207,6 +208,7 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
|||
// index #2
|
||||
orgReplica.applyIndexOperationOnReplica(2, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false,
|
||||
new SourceToParse(indexName, "type", "id-2", new BytesArray("{}"), XContentType.JSON));
|
||||
orgReplica.sync(); // advance local checkpoint
|
||||
orgReplica.updateGlobalCheckpointOnReplica(3L, "test");
|
||||
// index #5 -> force NoOp #4.
|
||||
orgReplica.applyIndexOperationOnReplica(5, 1, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false,
|
||||
|
@ -330,11 +332,11 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
|
|||
@Override
|
||||
public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps, ActionListener<Void> listener) {
|
||||
super.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps, listener);
|
||||
assertThat(replicaShard.getGlobalCheckpoint(), equalTo(primaryShard.getGlobalCheckpoint()));
|
||||
assertThat(replicaShard.getLastKnownGlobalCheckpoint(), equalTo(primaryShard.getLastKnownGlobalCheckpoint()));
|
||||
}
|
||||
@Override
|
||||
public void cleanFiles(int totalTranslogOps, long globalCheckpoint, Store.MetadataSnapshot sourceMetaData) throws IOException {
|
||||
assertThat(globalCheckpoint, equalTo(primaryShard.getGlobalCheckpoint()));
|
||||
assertThat(globalCheckpoint, equalTo(primaryShard.getLastKnownGlobalCheckpoint()));
|
||||
super.cleanFiles(totalTranslogOps, globalCheckpoint, sourceMetaData);
|
||||
}
|
||||
}, true, true);
|
||||
|
|
|
@ -74,10 +74,9 @@ public class CloseIndexIT extends ESIntegTestCase {
|
|||
|
||||
@Override
|
||||
public Settings indexSettings() {
|
||||
Settings.builder().put(super.indexSettings())
|
||||
return Settings.builder().put(super.indexSettings())
|
||||
.put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(),
|
||||
new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB));
|
||||
return super.indexSettings();
|
||||
new ByteSizeValue(randomIntBetween(1, 4096), ByteSizeUnit.KB)).build();
|
||||
}
|
||||
|
||||
public void testCloseMissingIndex() {
|
||||
|
|
|
@ -1204,7 +1204,7 @@ public class IndexStatsIT extends ESIntegTestCase {
|
|||
for (IndexService indexService : indexServices) {
|
||||
for (IndexShard indexShard : indexService) {
|
||||
indexShard.sync();
|
||||
assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getGlobalCheckpoint()));
|
||||
assertThat(indexShard.getLastSyncedGlobalCheckpoint(), equalTo(indexShard.getLastKnownGlobalCheckpoint()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -108,6 +108,7 @@ public class TruncatedRecoveryIT extends ESIntegTestCase {
|
|||
ensureGreen();
|
||||
// ensure we have flushed segments and make them a big one via optimize
|
||||
client().admin().indices().prepareFlush().setForce(true).get();
|
||||
client().admin().indices().prepareFlush().setForce(true).get(); // double flush to create safe commit in case of async durability
|
||||
client().admin().indices().prepareForceMerge().setMaxNumSegments(1).setFlush(true).get();
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
@ -119,7 +120,7 @@ public class TruncatedRecoveryIT extends ESIntegTestCase {
|
|||
(connection, requestId, action, request, options) -> {
|
||||
if (action.equals(PeerRecoveryTargetService.Actions.FILE_CHUNK)) {
|
||||
RecoveryFileChunkRequest req = (RecoveryFileChunkRequest) request;
|
||||
logger.debug("file chunk [{}] lastChunk: {}", req, req.lastChunk());
|
||||
logger.info("file chunk [{}] lastChunk: {}", req, req.lastChunk());
|
||||
if ((req.name().endsWith("cfs") || req.name().endsWith("fdt")) && req.lastChunk() && truncate.get()) {
|
||||
latch.countDown();
|
||||
throw new RuntimeException("Caused some truncated files for fun and profit");
|
||||
|
|
|
@ -3606,6 +3606,7 @@ public class SharedClusterSnapshotRestoreIT extends AbstractSnapshotIntegTestCas
|
|||
for (int i = 10; i < 15; i++) {
|
||||
index(indexName, "_doc", Integer.toString(i), "foo", "bar" + i);
|
||||
}
|
||||
client().admin().indices().prepareFlush(indexName).setForce(true).setWaitIfOngoing(true).get();
|
||||
|
||||
stats = client().admin().indices().prepareStats(indexName).clear().get();
|
||||
shardStats = stats.getShards()[0];
|
||||
|
|
|
@ -411,7 +411,7 @@ public abstract class EngineTestCase extends ESTestCase {
|
|||
String translogUUID = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId,
|
||||
primaryTermSupplier.getAsLong());
|
||||
return new Translog(translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS),
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTermSupplier);
|
||||
() -> SequenceNumbers.NO_OPS_PERFORMED, primaryTermSupplier, seqNo -> {});
|
||||
}
|
||||
|
||||
protected TranslogHandler createTranslogHandler(IndexSettings indexSettings) {
|
||||
|
@ -1147,7 +1147,7 @@ public abstract class EngineTestCase extends ESTestCase {
|
|||
* @throws InterruptedException if the thread was interrupted while blocking on the condition
|
||||
*/
|
||||
public static void waitForOpsToComplete(InternalEngine engine, long seqNo) throws InterruptedException {
|
||||
engine.getLocalCheckpointTracker().waitForOpsToComplete(seqNo);
|
||||
engine.getLocalCheckpointTracker().waitForProcessedOpsToComplete(seqNo);
|
||||
}
|
||||
|
||||
public static boolean hasSnapshottedCommits(Engine engine) {
|
||||
|
|
|
@ -113,6 +113,7 @@ public class TranslogHandler implements Engine.TranslogRecoveryRunner {
|
|||
opsRecovered++;
|
||||
appliedOperations.incrementAndGet();
|
||||
}
|
||||
engine.syncTranslog();
|
||||
return opsRecovered;
|
||||
}
|
||||
|
||||
|
|
|
@ -552,7 +552,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
|
||||
public void executeRetentionLeasesSyncRequestOnReplica(RetentionLeaseSyncAction.Request request, IndexShard replica) {
|
||||
final PlainActionFuture<Releasable> acquirePermitFuture = new PlainActionFuture<>();
|
||||
replica.acquireReplicaOperationPermit(getPrimary().getOperationPrimaryTerm(), getPrimary().getGlobalCheckpoint(),
|
||||
replica.acquireReplicaOperationPermit(getPrimary().getOperationPrimaryTerm(), getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
getPrimary().getMaxSeqNoOfUpdatesOrDeletes(), acquirePermitFuture, ThreadPool.Names.SAME, request);
|
||||
try (Releasable ignored = acquirePermitFuture.actionGet()) {
|
||||
replica.updateRetentionLeasesOnReplica(request.getRetentionLeases());
|
||||
|
@ -659,7 +659,12 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
|
||||
@Override
|
||||
public long globalCheckpoint() {
|
||||
return getPrimaryShard().getGlobalCheckpoint();
|
||||
return getPrimaryShard().getLastSyncedGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
public long computedGlobalCheckpoint() {
|
||||
return getPrimaryShard().getLastKnownGlobalCheckpoint();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -693,7 +698,8 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
try {
|
||||
performOnReplica(request, replica);
|
||||
releasable.close();
|
||||
delegatedListener.onResponse(new ReplicaResponse(replica.getLocalCheckpoint(), replica.getGlobalCheckpoint()));
|
||||
delegatedListener.onResponse(new ReplicaResponse(replica.getLocalCheckpoint(),
|
||||
replica.getLastKnownGlobalCheckpoint()));
|
||||
} catch (final Exception e) {
|
||||
Releasables.closeWhileHandlingException(releasable);
|
||||
delegatedListener.onFailure(e);
|
||||
|
@ -756,7 +762,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
@Override
|
||||
protected void performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
|
||||
executeShardBulkOnReplica(request, replica, getPrimaryShard().getPendingPrimaryTerm(),
|
||||
getPrimaryShard().getGlobalCheckpoint(), getPrimaryShard().getMaxSeqNoOfUpdatesOrDeletes());
|
||||
getPrimaryShard().getLastKnownGlobalCheckpoint(), getPrimaryShard().getMaxSeqNoOfUpdatesOrDeletes());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -827,7 +833,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
|
||||
void indexOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica, long term) throws Exception {
|
||||
executeShardBulkOnReplica(request, replica, term,
|
||||
group.primary.getGlobalCheckpoint(), group.primary.getMaxSeqNoOfUpdatesOrDeletes());
|
||||
group.primary.getLastKnownGlobalCheckpoint(), group.primary.getMaxSeqNoOfUpdatesOrDeletes());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -835,7 +841,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
*/
|
||||
void deleteOnReplica(BulkShardRequest request, ReplicationGroup group, IndexShard replica) throws Exception {
|
||||
executeShardBulkOnReplica(request, replica, group.primary.getPendingPrimaryTerm(),
|
||||
group.primary.getGlobalCheckpoint(), group.primary.getMaxSeqNoOfUpdatesOrDeletes());
|
||||
group.primary.getLastKnownGlobalCheckpoint(), group.primary.getMaxSeqNoOfUpdatesOrDeletes());
|
||||
}
|
||||
|
||||
class GlobalCheckpointSync extends ReplicationAction<
|
||||
|
@ -884,7 +890,7 @@ public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase
|
|||
@Override
|
||||
protected void performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception {
|
||||
executeResyncOnReplica(replica, request, getPrimaryShard().getPendingPrimaryTerm(),
|
||||
getPrimaryShard().getGlobalCheckpoint(), getPrimaryShard().getMaxSeqNoOfUpdatesOrDeletes());
|
||||
getPrimaryShard().getLastKnownGlobalCheckpoint(), getPrimaryShard().getMaxSeqNoOfUpdatesOrDeletes());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -753,12 +753,14 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
result = shard.applyIndexOperationOnPrimary(Versions.MATCH_ANY, VersionType.INTERNAL, sourceToParse,
|
||||
SequenceNumbers.UNASSIGNED_SEQ_NO, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false);
|
||||
}
|
||||
shard.sync(); // advance local checkpoint
|
||||
shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(),
|
||||
shard.getLocalCheckpoint());
|
||||
} else {
|
||||
final long seqNo = shard.seqNoStats().getMaxSeqNo() + 1;
|
||||
shard.advanceMaxSeqNoOfUpdatesOrDeletes(seqNo); // manually replicate max_seq_no_of_updates
|
||||
result = shard.applyIndexOperationOnReplica(seqNo, 0, IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, sourceToParse);
|
||||
shard.sync(); // advance local checkpoint
|
||||
if (result.getResultType() == Engine.Result.Type.MAPPING_UPDATE_REQUIRED) {
|
||||
throw new TransportReplicationAction.RetryOnReplicaException(shard.shardId,
|
||||
"Mappings are not available on the replica yet, triggered update: " + result.getRequiredMappingUpdate());
|
||||
|
@ -777,11 +779,14 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
if (shard.routingEntry().primary()) {
|
||||
result = shard.applyDeleteOperationOnPrimary(
|
||||
Versions.MATCH_ANY, type, id, VersionType.INTERNAL, SequenceNumbers.UNASSIGNED_SEQ_NO, 0);
|
||||
shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(), shard.getEngine().getLocalCheckpoint());
|
||||
shard.sync(); // advance local checkpoint
|
||||
shard.updateLocalCheckpointForShard(shard.routingEntry().allocationId().getId(),
|
||||
shard.getLocalCheckpoint());
|
||||
} else {
|
||||
final long seqNo = shard.seqNoStats().getMaxSeqNo() + 1;
|
||||
shard.advanceMaxSeqNoOfUpdatesOrDeletes(seqNo); // manually replicate max_seq_no_of_updates
|
||||
result = shard.applyDeleteOperationOnReplica(seqNo, 0L, type, id);
|
||||
shard.sync(); // advance local checkpoint
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
|
|
@ -27,8 +27,10 @@ import org.apache.logging.log4j.util.Supplier;
|
|||
import org.elasticsearch.action.bulk.BulkItemResponse;
|
||||
import org.elasticsearch.action.bulk.BulkRequestBuilder;
|
||||
import org.elasticsearch.action.bulk.BulkResponse;
|
||||
import org.elasticsearch.action.bulk.BulkShardRequest;
|
||||
import org.elasticsearch.action.index.IndexResponse;
|
||||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.common.xcontent.XContentFactory;
|
||||
|
@ -141,7 +143,7 @@ public class BackgroundIndexer implements AutoCloseable {
|
|||
}
|
||||
|
||||
}
|
||||
BulkRequestBuilder bulkRequest = client.prepareBulk();
|
||||
BulkRequestBuilder bulkRequest = client.prepareBulk().setTimeout(timeout);
|
||||
for (int i = 0; i < batchSize; i++) {
|
||||
id = idGenerator.incrementAndGet();
|
||||
if (useAutoGeneratedIDs) {
|
||||
|
@ -151,16 +153,21 @@ public class BackgroundIndexer implements AutoCloseable {
|
|||
.setSource(generateSource(id, threadRandom)));
|
||||
}
|
||||
}
|
||||
BulkResponse bulkResponse = bulkRequest.get();
|
||||
for (BulkItemResponse bulkItemResponse : bulkResponse) {
|
||||
if (bulkItemResponse.isFailed() == false) {
|
||||
boolean add = ids.add(bulkItemResponse.getId());
|
||||
assert add : "ID: " + bulkItemResponse.getId() + " already used";
|
||||
} else {
|
||||
failures.add(bulkItemResponse.getFailure().getCause());
|
||||
try {
|
||||
BulkResponse bulkResponse = bulkRequest.get();
|
||||
for (BulkItemResponse bulkItemResponse : bulkResponse) {
|
||||
if (bulkItemResponse.isFailed() == false) {
|
||||
boolean add = ids.add(bulkItemResponse.getId());
|
||||
assert add : "ID: " + bulkItemResponse.getId() + " already used";
|
||||
} else {
|
||||
failures.add(bulkItemResponse.getFailure().getCause());
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
if (ignoreIndexingFailures == false) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
} else {
|
||||
|
||||
if (hasBudget.get() && !availableBudget.tryAcquire(250, TimeUnit.MILLISECONDS)) {
|
||||
|
@ -169,15 +176,27 @@ public class BackgroundIndexer implements AutoCloseable {
|
|||
}
|
||||
id = idGenerator.incrementAndGet();
|
||||
if (useAutoGeneratedIDs) {
|
||||
IndexResponse indexResponse = client.prepareIndex(index, type)
|
||||
.setSource(generateSource(id, threadRandom)).get();
|
||||
boolean add = ids.add(indexResponse.getId());
|
||||
assert add : "ID: " + indexResponse.getId() + " already used";
|
||||
try {
|
||||
IndexResponse indexResponse = client.prepareIndex(index, type)
|
||||
.setTimeout(timeout).setSource(generateSource(id, threadRandom)).get();
|
||||
boolean add = ids.add(indexResponse.getId());
|
||||
assert add : "ID: " + indexResponse.getId() + " already used";
|
||||
} catch (Exception e) {
|
||||
if (ignoreIndexingFailures == false) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
} else {
|
||||
IndexResponse indexResponse = client.prepareIndex(index, type, Long.toString(id))
|
||||
.setSource(generateSource(id, threadRandom)).get();
|
||||
boolean add = ids.add(indexResponse.getId());
|
||||
assert add : "ID: " + indexResponse.getId() + " already used";
|
||||
try {
|
||||
IndexResponse indexResponse = client.prepareIndex(index, type, Long.toString(id))
|
||||
.setTimeout(timeout).setSource(generateSource(id, threadRandom)).get();
|
||||
boolean add = ids.add(indexResponse.getId());
|
||||
assert add : "ID: " + indexResponse.getId() + " already used";
|
||||
} catch (Exception e) {
|
||||
if (ignoreIndexingFailures == false) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -217,6 +236,18 @@ public class BackgroundIndexer implements AutoCloseable {
|
|||
|
||||
}
|
||||
|
||||
private volatile TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT;
|
||||
|
||||
public void setRequestTimeout(TimeValue timeout) {
|
||||
this.timeout = timeout;
|
||||
}
|
||||
|
||||
private volatile boolean ignoreIndexingFailures;
|
||||
|
||||
public void setIgnoreIndexingFailures(boolean ignoreIndexingFailures) {
|
||||
this.ignoreIndexingFailures = ignoreIndexingFailures;
|
||||
}
|
||||
|
||||
private void setBudget(int numOfDocs) {
|
||||
logger.debug("updating budget to [{}]", numOfDocs);
|
||||
if (numOfDocs >= 0) {
|
||||
|
|
|
@ -1897,6 +1897,8 @@ public final class InternalTestCluster extends TestCluster {
|
|||
nodesByRoles.computeIfAbsent(discoveryNode.getRoles(), k -> new ArrayList<>()).add(nodeAndClient);
|
||||
}
|
||||
|
||||
callback.onAllNodesStopped();
|
||||
|
||||
assert nodesByRoles.values().stream().mapToInt(List::size).sum() == nodeCount;
|
||||
|
||||
// randomize start up order, but making sure that:
|
||||
|
@ -2374,6 +2376,9 @@ public final class InternalTestCluster extends TestCluster {
|
|||
return Settings.EMPTY;
|
||||
}
|
||||
|
||||
public void onAllNodesStopped() throws Exception {
|
||||
}
|
||||
|
||||
/**
|
||||
* Executed for each node before the {@code n + 1} node is restarted. The given client is
|
||||
* an active client to the node that will be restarted next.
|
||||
|
|
|
@ -144,10 +144,11 @@ public class TransportBulkShardOperationsAction
|
|||
assert failure.getSeqNo() == targetOp.seqNo() : targetOp.seqNo() + " != " + failure.getSeqNo();
|
||||
if (failure.getExistingPrimaryTerm().isPresent()) {
|
||||
appliedOperations.add(rewriteOperationWithPrimaryTerm(sourceOp, failure.getExistingPrimaryTerm().getAsLong()));
|
||||
} else if (targetOp.seqNo() > primary.getGlobalCheckpoint()) {
|
||||
assert false : "can't find primary_term for existing op=" + targetOp + " gcp=" + primary.getGlobalCheckpoint();
|
||||
} else if (targetOp.seqNo() > primary.getLastKnownGlobalCheckpoint()) {
|
||||
assert false :
|
||||
"can't find primary_term for existing op=" + targetOp + " gcp=" + primary.getLastKnownGlobalCheckpoint();
|
||||
throw new IllegalStateException("can't find primary_term for existing op=" + targetOp +
|
||||
" global_checkpoint=" + primary.getGlobalCheckpoint(), failure);
|
||||
" global_checkpoint=" + primary.getLastKnownGlobalCheckpoint(), failure);
|
||||
}
|
||||
} else {
|
||||
assert false : "Only already-processed error should happen; op=[" + targetOp + "] error=[" + result.getFailure() + "]";
|
||||
|
|
|
@ -90,7 +90,7 @@ public final class FollowingEngine extends InternalEngine {
|
|||
} else {
|
||||
return IndexingStrategy.processButSkipLucene(false, index.version());
|
||||
}
|
||||
} else if (maxSeqNoOfUpdatesOrDeletes <= getLocalCheckpoint()) {
|
||||
} else if (maxSeqNoOfUpdatesOrDeletes <= getProcessedLocalCheckpoint()) {
|
||||
assert maxSeqNoOfUpdatesOrDeletes < index.seqNo() : "seq_no[" + index.seqNo() + "] <= msu[" + maxSeqNoOfUpdatesOrDeletes + "]";
|
||||
numOfOptimizedIndexing.inc();
|
||||
return InternalEngine.IndexingStrategy.optimizedAppendOnly(index.version());
|
||||
|
|
|
@ -491,10 +491,10 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|||
while (offset < fileLength && error.get() == null) {
|
||||
final long requestSeqId = requestSeqIdTracker.generateSeqNo();
|
||||
try {
|
||||
requestSeqIdTracker.waitForOpsToComplete(requestSeqId - ccrSettings.getMaxConcurrentFileChunks());
|
||||
requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqId - ccrSettings.getMaxConcurrentFileChunks());
|
||||
|
||||
if (error.get() != null) {
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -514,7 +514,7 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e));
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -526,24 +526,24 @@ public class CcrRepository extends AbstractLifecycleComponent implements Reposit
|
|||
throttleListener.accept(nanosPaused);
|
||||
final boolean lastChunk = r.getOffset() + actualChunkSize >= fileLength;
|
||||
multiFileWriter.writeFileChunk(fileInfo.metadata(), r.getOffset(), r.getChunk(), lastChunk);
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
}
|
||||
}),
|
||||
e -> {
|
||||
error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e));
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
}
|
||||
), timeout, ThreadPool.Names.GENERIC, GetCcrRestoreFileChunkAction.NAME);
|
||||
remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request, listener);
|
||||
} catch (Exception e) {
|
||||
error.compareAndSet(null, Tuple.tuple(fileInfo.metadata(), e));
|
||||
requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId);
|
||||
requestSeqIdTracker.markSeqNoAsProcessed(requestSeqId);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
requestSeqIdTracker.waitForOpsToComplete(requestSeqIdTracker.getMaxSeqNo());
|
||||
requestSeqIdTracker.waitForProcessedOpsToComplete(requestSeqIdTracker.getMaxSeqNo());
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new ElasticsearchException(e);
|
||||
|
|
|
@ -284,7 +284,7 @@ public class FollowerFailOverIT extends CcrIntegTestCase {
|
|||
IndexResponse indexResp = leaderCluster.client().prepareIndex("leader-index", "doc", "1")
|
||||
.setSource("{\"balance\": 100}", XContentType.JSON).setTimeout(TimeValue.ZERO).get();
|
||||
assertThat(indexResp.getResult(), equalTo(DocWriteResponse.Result.CREATED));
|
||||
assertThat(indexShard.getGlobalCheckpoint(), equalTo(0L));
|
||||
assertThat(indexShard.getLastKnownGlobalCheckpoint(), equalTo(0L));
|
||||
// Make sure at least one read-request which requires mapping sync is completed.
|
||||
assertBusy(() -> {
|
||||
CcrClient ccrClient = new CcrClient(followerClient());
|
||||
|
|
|
@ -67,7 +67,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
int size = max - min + 1;
|
||||
final Translog.Operation[] operations = ShardChangesAction.getOperations(
|
||||
indexShard,
|
||||
indexShard.getGlobalCheckpoint(),
|
||||
indexShard.getLastKnownGlobalCheckpoint(),
|
||||
min,
|
||||
size,
|
||||
indexShard.getHistoryUUID(),
|
||||
|
@ -83,7 +83,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
IllegalStateException.class,
|
||||
() -> ShardChangesAction.getOperations(
|
||||
indexShard,
|
||||
indexShard.getGlobalCheckpoint(),
|
||||
indexShard.getLastKnownGlobalCheckpoint(),
|
||||
numWrites,
|
||||
numWrites + 1,
|
||||
indexShard.getHistoryUUID(),
|
||||
|
@ -92,18 +92,19 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
Locale.ROOT,
|
||||
"not exposing operations from [%d] greater than the global checkpoint [%d]",
|
||||
numWrites,
|
||||
indexShard.getGlobalCheckpoint());
|
||||
indexShard.getLastKnownGlobalCheckpoint());
|
||||
assertThat(e, hasToString(containsString(message)));
|
||||
}
|
||||
|
||||
// get operations for a range some operations do not exist:
|
||||
Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(),
|
||||
Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getLastKnownGlobalCheckpoint(),
|
||||
numWrites - 10, numWrites + 10, indexShard.getHistoryUUID(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES));
|
||||
assertThat(operations.length, equalTo(10));
|
||||
|
||||
// Unexpected history UUID:
|
||||
Exception e = expectThrows(IllegalStateException.class, () -> ShardChangesAction.getOperations(indexShard,
|
||||
indexShard.getGlobalCheckpoint(), 0, 10, "different-history-uuid", new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)));
|
||||
indexShard.getLastKnownGlobalCheckpoint(), 0, 10, "different-history-uuid",
|
||||
new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)));
|
||||
assertThat(e.getMessage(), equalTo("unexpected history uuid, expected [different-history-uuid], actual [" +
|
||||
indexShard.getHistoryUUID() + "]"));
|
||||
|
||||
|
@ -112,7 +113,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
final long fromSeqNo = randomLongBetween(Long.MIN_VALUE, -1);
|
||||
final int batchSize = randomIntBetween(0, Integer.MAX_VALUE);
|
||||
final IllegalArgumentException invalidRangeError = expectThrows(IllegalArgumentException.class,
|
||||
() -> ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(),
|
||||
() -> ShardChangesAction.getOperations(indexShard, indexShard.getLastKnownGlobalCheckpoint(),
|
||||
fromSeqNo, batchSize, indexShard.getHistoryUUID(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)));
|
||||
assertThat(invalidRangeError.getMessage(),
|
||||
equalTo("Invalid range; from_seqno [" + fromSeqNo + "], to_seqno [" + (fromSeqNo + batchSize - 1) + "]"));
|
||||
|
@ -125,7 +126,8 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
ShardRouting shardRouting = TestShardRouting.newShardRouting("index", 0, "_node_id", true, ShardRoutingState.INITIALIZING);
|
||||
Mockito.when(indexShard.routingEntry()).thenReturn(shardRouting);
|
||||
expectThrows(IndexShardNotStartedException.class, () -> ShardChangesAction.getOperations(indexShard,
|
||||
indexShard.getGlobalCheckpoint(), 0, 1, indexShard.getHistoryUUID(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)));
|
||||
indexShard.getLastKnownGlobalCheckpoint(), 0, 1, indexShard.getHistoryUUID(),
|
||||
new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES)));
|
||||
}
|
||||
|
||||
public void testGetOperationsExceedByteLimit() throws Exception {
|
||||
|
@ -142,7 +144,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
}
|
||||
|
||||
final IndexShard indexShard = indexService.getShard(0);
|
||||
final Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getGlobalCheckpoint(),
|
||||
final Translog.Operation[] operations = ShardChangesAction.getOperations(indexShard, indexShard.getLastKnownGlobalCheckpoint(),
|
||||
0, 12, indexShard.getHistoryUUID(), new ByteSizeValue(256, ByteSizeUnit.BYTES));
|
||||
assertThat(operations.length, equalTo(12));
|
||||
assertThat(operations[0].seqNo(), equalTo(0L));
|
||||
|
@ -172,7 +174,7 @@ public class ShardChangesActionTests extends ESSingleNodeTestCase {
|
|||
final IndexShard indexShard = indexService.getShard(0);
|
||||
final Translog.Operation[] operations =
|
||||
ShardChangesAction.getOperations(
|
||||
indexShard, indexShard.getGlobalCheckpoint(), 0, 1, indexShard.getHistoryUUID(), ByteSizeValue.ZERO);
|
||||
indexShard, indexShard.getLastKnownGlobalCheckpoint(), 0, 1, indexShard.getHistoryUUID(), ByteSizeValue.ZERO);
|
||||
assertThat(operations.length, equalTo(1));
|
||||
assertThat(operations[0].seqNo(), equalTo(0L));
|
||||
}
|
||||
|
|
|
@ -139,13 +139,13 @@ public class ShardFollowNodeTaskRandomTests extends ESTestCase {
|
|||
Consumer<BulkShardOperationsResponse> handler,
|
||||
Consumer<Exception> errorHandler) {
|
||||
for(Translog.Operation op : operations) {
|
||||
tracker.markSeqNoAsCompleted(op.seqNo());
|
||||
tracker.markSeqNoAsProcessed(op.seqNo());
|
||||
}
|
||||
receivedOperations.addAll(operations);
|
||||
|
||||
// Emulate network thread and avoid SO:
|
||||
final BulkShardOperationsResponse response = new BulkShardOperationsResponse();
|
||||
response.setGlobalCheckpoint(tracker.getCheckpoint());
|
||||
response.setGlobalCheckpoint(tracker.getProcessedCheckpoint());
|
||||
response.setMaxSeqNo(tracker.getMaxSeqNo());
|
||||
threadPool.generic().execute(() -> handler.accept(response));
|
||||
}
|
||||
|
@ -180,7 +180,7 @@ public class ShardFollowNodeTaskRandomTests extends ESTestCase {
|
|||
}
|
||||
} else {
|
||||
assert from >= testRun.finalExpectedGlobalCheckpoint;
|
||||
final long globalCheckpoint = tracker.getCheckpoint();
|
||||
final long globalCheckpoint = tracker.getProcessedCheckpoint();
|
||||
final long maxSeqNo = tracker.getMaxSeqNo();
|
||||
handler.accept(new ShardChangesAction.Response(
|
||||
0L,
|
||||
|
|
|
@ -105,7 +105,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
leaderGroup.assertAllEqual(docCount);
|
||||
Set<String> indexedDocIds = getShardDocUIDs(leaderGroup.getPrimary());
|
||||
assertBusy(() -> {
|
||||
assertThat(followerGroup.getPrimary().getGlobalCheckpoint(), equalTo(leaderGroup.getPrimary().getGlobalCheckpoint()));
|
||||
assertThat(followerGroup.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
equalTo(leaderGroup.getPrimary().getLastKnownGlobalCheckpoint()));
|
||||
followerGroup.assertAllEqual(indexedDocIds.size());
|
||||
});
|
||||
for (IndexShard shard : followerGroup) {
|
||||
|
@ -119,7 +120,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
}
|
||||
leaderGroup.syncGlobalCheckpoint();
|
||||
assertBusy(() -> {
|
||||
assertThat(followerGroup.getPrimary().getGlobalCheckpoint(), equalTo(leaderGroup.getPrimary().getGlobalCheckpoint()));
|
||||
assertThat(followerGroup.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
equalTo(leaderGroup.getPrimary().getLastKnownGlobalCheckpoint()));
|
||||
followerGroup.assertAllEqual(indexedDocIds.size() - deleteDocIds.size());
|
||||
});
|
||||
shardFollowTask.markAsCompleted();
|
||||
|
@ -192,7 +194,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
leaderGroup.assertAllEqual(docCount);
|
||||
Set<String> indexedDocIds = getShardDocUIDs(leaderGroup.getPrimary());
|
||||
assertBusy(() -> {
|
||||
assertThat(followerGroup.getPrimary().getGlobalCheckpoint(), equalTo(leaderGroup.getPrimary().getGlobalCheckpoint()));
|
||||
assertThat(followerGroup.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
equalTo(leaderGroup.getPrimary().getLastKnownGlobalCheckpoint()));
|
||||
followerGroup.assertAllEqual(indexedDocIds.size());
|
||||
});
|
||||
|
||||
|
@ -235,7 +238,8 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
leaderGroup.assertAllEqual(docCount);
|
||||
Set<String> indexedDocIds = getShardDocUIDs(leaderGroup.getPrimary());
|
||||
assertBusy(() -> {
|
||||
assertThat(followerGroup.getPrimary().getGlobalCheckpoint(), equalTo(leaderGroup.getPrimary().getGlobalCheckpoint()));
|
||||
assertThat(followerGroup.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
equalTo(leaderGroup.getPrimary().getLastKnownGlobalCheckpoint()));
|
||||
followerGroup.assertAllEqual(indexedDocIds.size());
|
||||
});
|
||||
|
||||
|
@ -282,11 +286,12 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
// Simulates some bulk requests are completed on the primary and replicated to some (but all) replicas of the follower
|
||||
// but the primary of the follower crashed before these requests completed.
|
||||
for (int numBulks = between(1, 5), i = 0; i < numBulks; i++) {
|
||||
long fromSeqNo = randomLongBetween(0, leadingPrimary.getGlobalCheckpoint());
|
||||
long toSeqNo = randomLongBetween(fromSeqNo, leadingPrimary.getGlobalCheckpoint());
|
||||
long fromSeqNo = randomLongBetween(0, leadingPrimary.getLastKnownGlobalCheckpoint());
|
||||
long toSeqNo = randomLongBetween(fromSeqNo, leadingPrimary.getLastKnownGlobalCheckpoint());
|
||||
int numOps = Math.toIntExact(toSeqNo + 1 - fromSeqNo);
|
||||
Translog.Operation[] ops = ShardChangesAction.getOperations(leadingPrimary, leadingPrimary.getGlobalCheckpoint(),
|
||||
fromSeqNo, numOps, leadingPrimary.getHistoryUUID(), new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES));
|
||||
Translog.Operation[] ops = ShardChangesAction.getOperations(leadingPrimary,
|
||||
leadingPrimary.getLastKnownGlobalCheckpoint(), fromSeqNo, numOps, leadingPrimary.getHistoryUUID(),
|
||||
new ByteSizeValue(Long.MAX_VALUE, ByteSizeUnit.BYTES));
|
||||
|
||||
IndexShard followingPrimary = followerGroup.getPrimary();
|
||||
TransportWriteAction.WritePrimaryResult<BulkShardOperationsRequest, BulkShardOperationsResponse> primaryResult =
|
||||
|
@ -296,7 +301,7 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
for (IndexShard replica : randomSubsetOf(followerGroup.getReplicas())) {
|
||||
final PlainActionFuture<Releasable> permitFuture = new PlainActionFuture<>();
|
||||
replica.acquireReplicaOperationPermit(followingPrimary.getOperationPrimaryTerm(),
|
||||
followingPrimary.getGlobalCheckpoint(), followingPrimary.getMaxSeqNoOfUpdatesOrDeletes(),
|
||||
followingPrimary.getLastKnownGlobalCheckpoint(), followingPrimary.getMaxSeqNoOfUpdatesOrDeletes(),
|
||||
permitFuture, ThreadPool.Names.SAME, primaryResult);
|
||||
try (Releasable ignored = permitFuture.get()) {
|
||||
TransportBulkShardOperationsAction.shardOperationOnReplica(primaryResult.replicaRequest(), replica, logger);
|
||||
|
@ -308,13 +313,14 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
ShardFollowNodeTask shardFollowTask = createShardFollowTask(leaderGroup, followerGroup);
|
||||
SeqNoStats followerSeqNoStats = followerGroup.getPrimary().seqNoStats();
|
||||
shardFollowTask.start(followerGroup.getPrimary().getHistoryUUID(),
|
||||
leadingPrimary.getGlobalCheckpoint(),
|
||||
leadingPrimary.getLastKnownGlobalCheckpoint(),
|
||||
leadingPrimary.getMaxSeqNoOfUpdatesOrDeletes(),
|
||||
followerSeqNoStats.getGlobalCheckpoint(),
|
||||
followerSeqNoStats.getMaxSeqNo());
|
||||
try {
|
||||
assertBusy(() -> {
|
||||
assertThat(followerGroup.getPrimary().getGlobalCheckpoint(), equalTo(leadingPrimary.getGlobalCheckpoint()));
|
||||
assertThat(followerGroup.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
equalTo(leadingPrimary.getLastKnownGlobalCheckpoint()));
|
||||
assertConsistentHistoryBetweenLeaderAndFollower(leaderGroup, followerGroup, true);
|
||||
});
|
||||
} finally {
|
||||
|
@ -380,9 +386,9 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
ShardFollowNodeTask followTask = createShardFollowTask(leader, follower);
|
||||
followTask.start(
|
||||
follower.getPrimary().getHistoryUUID(),
|
||||
leader.getPrimary().getGlobalCheckpoint(),
|
||||
leader.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
leader.getPrimary().seqNoStats().getMaxSeqNo(),
|
||||
follower.getPrimary().getGlobalCheckpoint(),
|
||||
follower.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
follower.getPrimary().seqNoStats().getMaxSeqNo()
|
||||
);
|
||||
leader.appendDocs(between(0, 100));
|
||||
|
@ -403,9 +409,9 @@ public class ShardFollowTaskReplicationTests extends ESIndexLevelReplicationTest
|
|||
final ShardFollowNodeTask task = createShardFollowTask(leader, follower);
|
||||
task.start(
|
||||
follower.getPrimary().getHistoryUUID(),
|
||||
leader.getPrimary().getGlobalCheckpoint(),
|
||||
leader.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
leader.getPrimary().seqNoStats().getMaxSeqNo(),
|
||||
follower.getPrimary().getGlobalCheckpoint(),
|
||||
follower.getPrimary().getLastKnownGlobalCheckpoint(),
|
||||
follower.getPrimary().seqNoStats().getMaxSeqNo());
|
||||
final Scheduler.Cancellable renewable = task.getRenewable();
|
||||
assertNotNull(renewable);
|
||||
|
|
|
@ -142,7 +142,7 @@ public class BulkShardOperationsTests extends IndexShardTestCase {
|
|||
newPrimary.getHistoryUUID(), Stream.concat(secondBulk.stream(), existingOps.stream()).collect(Collectors.toList()),
|
||||
seqno, newPrimary, logger);
|
||||
final long newPrimaryTerm = newPrimary.getOperationPrimaryTerm();
|
||||
final long globalCheckpoint = newPrimary.getGlobalCheckpoint();
|
||||
final long globalCheckpoint = newPrimary.getLastKnownGlobalCheckpoint();
|
||||
final List<Translog.Operation> appliedOperations = Stream.concat(
|
||||
secondBulk.stream().map(op -> rewriteOperationWithPrimaryTerm(op, newPrimaryTerm)),
|
||||
existingOps.stream().filter(op -> op.seqNo() > globalCheckpoint).map(op -> rewriteOperationWithPrimaryTerm(op, oldPrimaryTerm))
|
||||
|
|
|
@ -337,7 +337,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
for (int i = 0; i < numDocs; i++) {
|
||||
leader.index(indexForPrimary(Integer.toString(i)));
|
||||
}
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(-1L));
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numDocs));
|
||||
assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true)));
|
||||
|
@ -350,7 +350,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
leader.delete(deleteForPrimary(Integer.toString(i)));
|
||||
}
|
||||
}
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(leader.getMaxSeqNoOfUpdatesOrDeletes()));
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numDocs));
|
||||
assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true)));
|
||||
|
@ -362,7 +362,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
docIds.add(docId);
|
||||
leader.index(indexForPrimary(docId));
|
||||
}
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(leader.getMaxSeqNoOfUpdatesOrDeletes()));
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numDocs + moreDocs));
|
||||
assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true)));
|
||||
|
@ -378,7 +378,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
runFollowTest((leader, follower) -> {
|
||||
EngineTestCase.concurrentlyApplyOps(ops, leader);
|
||||
assertThat(follower.getMaxSeqNoOfUpdatesOrDeletes(), equalTo(-1L));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo((long) numOps));
|
||||
});
|
||||
}
|
||||
|
@ -396,13 +396,13 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
Randomness.shuffle(ops);
|
||||
runFollowTest((leader, follower) -> {
|
||||
EngineTestCase.concurrentlyApplyOps(ops, leader);
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
final List<Engine.Operation> appendOps = new ArrayList<>();
|
||||
for (int numAppends = scaledRandomIntBetween(0, 100), i = 0; i < numAppends; i++) {
|
||||
appendOps.add(indexForPrimary("append-" + i));
|
||||
}
|
||||
EngineTestCase.concurrentlyApplyOps(appendOps, leader);
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), greaterThanOrEqualTo((long) appendOps.size()));
|
||||
});
|
||||
}
|
||||
|
@ -410,19 +410,19 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
public void testOptimizeSingleDocSequentially() throws Exception {
|
||||
runFollowTest((leader, follower) -> {
|
||||
leader.index(indexForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(1L));
|
||||
|
||||
leader.delete(deleteForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(1L));
|
||||
|
||||
leader.index(indexForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(2L));
|
||||
|
||||
leader.index(indexForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(2L));
|
||||
});
|
||||
}
|
||||
|
@ -432,20 +432,20 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
Randomness.shuffle(ops);
|
||||
runFollowTest((leader, follower) -> {
|
||||
EngineTestCase.concurrentlyApplyOps(ops, leader);
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(getDocIds(follower, true), equalTo(getDocIds(leader, true)));
|
||||
long numOptimized = follower.getNumberOfOptimizedIndexing();
|
||||
|
||||
leader.delete(deleteForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numOptimized));
|
||||
|
||||
leader.index(indexForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numOptimized + 1L));
|
||||
|
||||
leader.index(indexForPrimary("id"));
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
assertThat(follower.getNumberOfOptimizedIndexing(), equalTo(numOptimized + 1L));
|
||||
});
|
||||
}
|
||||
|
@ -454,7 +454,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
final CheckedBiConsumer<InternalEngine, FollowingEngine, Exception> wrappedTask = (leader, follower) -> {
|
||||
Thread[] threads = new Thread[between(1, 8)];
|
||||
AtomicBoolean taskIsCompleted = new AtomicBoolean();
|
||||
AtomicLong lastFetchedSeqNo = new AtomicLong(follower.getLocalCheckpoint());
|
||||
AtomicLong lastFetchedSeqNo = new AtomicLong(follower.getProcessedLocalCheckpoint());
|
||||
CountDownLatch latch = new CountDownLatch(threads.length + 1);
|
||||
for (int i = 0; i < threads.length; i++) {
|
||||
threads[i] = new Thread(() -> {
|
||||
|
@ -472,7 +472,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
latch.countDown();
|
||||
latch.await();
|
||||
task.accept(leader, follower);
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getLocalCheckpoint());
|
||||
EngineTestCase.waitForOpsToComplete(follower, leader.getProcessedLocalCheckpoint());
|
||||
} finally {
|
||||
taskIsCompleted.set(true);
|
||||
for (Thread thread : threads) {
|
||||
|
@ -516,7 +516,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
final MapperService mapperService = EngineTestCase.createMapperService("test");
|
||||
final TranslogHandler translogHandler = new TranslogHandler(xContentRegistry(), follower.config().getIndexSettings());
|
||||
while (stopped.get() == false) {
|
||||
final long checkpoint = leader.getLocalCheckpoint();
|
||||
final long checkpoint = leader.getProcessedLocalCheckpoint();
|
||||
final long lastSeqNo = lastFetchedSeqNo.get();
|
||||
if (lastSeqNo < checkpoint) {
|
||||
final long nextSeqNo = randomLongBetween(lastSeqNo + 1, checkpoint);
|
||||
|
@ -607,7 +607,7 @@ public class FollowingEngineTests extends ESTestCase {
|
|||
}
|
||||
}
|
||||
// Primary should reject duplicates
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), followingEngine.getLocalCheckpoint()));
|
||||
globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), followingEngine.getProcessedLocalCheckpoint()));
|
||||
final long newTerm = randomLongBetween(oldTerm + 1, Long.MAX_VALUE);
|
||||
for (Engine.Operation op : operations) {
|
||||
Engine.Result result = applyOperation(followingEngine, op, newTerm, Engine.Operation.Origin.PRIMARY);
|
||||
|
|
|
@ -188,7 +188,7 @@ public class FrozenEngineTests extends EngineTestCase {
|
|||
if (rarely()) {
|
||||
engine.flush();
|
||||
}
|
||||
globalCheckpoint.set(engine.getLocalCheckpoint());
|
||||
globalCheckpoint.set(engine.getProcessedLocalCheckpoint());
|
||||
}
|
||||
engine.syncTranslog();
|
||||
return numDocsAdded;
|
||||
|
|
|
@ -371,7 +371,7 @@ public class FrozenIndexTests extends ESSingleNodeTestCase {
|
|||
final Index index = client().admin().cluster().prepareState().get().getState().metaData().index(indexName).getIndex();
|
||||
final IndexService indexService = indicesService.indexService(index);
|
||||
assertThat(indexService.hasShard(0), is(true));
|
||||
assertThat(indexService.getShard(0).getGlobalCheckpoint(), greaterThanOrEqualTo(nbNoOps - 1L));
|
||||
assertThat(indexService.getShard(0).getLastKnownGlobalCheckpoint(), greaterThanOrEqualTo(nbNoOps - 1L));
|
||||
});
|
||||
|
||||
assertAcked(new XPackClient(client()).freeze(new TransportFreezeIndexAction.FreezeRequest(indexName)));
|
||||
|
|
Loading…
Reference in New Issue