|
@@ -41,6 +41,8 @@ import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
|
|
|
import org.elasticsearch.common.unit.ByteSizeValue;
|
|
|
import org.elasticsearch.common.util.CancellableThreads;
|
|
|
import org.elasticsearch.index.engine.RecoveryEngineException;
|
|
|
+import org.elasticsearch.index.seqno.LocalCheckpointTracker;
|
|
|
+import org.elasticsearch.index.seqno.SequenceNumbersService;
|
|
|
import org.elasticsearch.index.shard.IndexShard;
|
|
|
import org.elasticsearch.index.shard.IndexShardClosedException;
|
|
|
import org.elasticsearch.index.shard.IndexShardRelocatedException;
|
|
@@ -124,47 +126,58 @@ public class RecoverySourceHandler {
|
|
|
* performs the recovery from the local engine to the target
|
|
|
*/
|
|
|
public RecoveryResponse recoverToTarget() throws IOException {
|
|
|
- try (Translog.View translogView = shard.acquireTranslogView()) {
|
|
|
- logger.trace("captured translog id [{}] for recovery", translogView.minTranslogGeneration());
|
|
|
- final IndexCommit phase1Snapshot;
|
|
|
- try {
|
|
|
- phase1Snapshot = shard.acquireIndexCommit(false);
|
|
|
- } catch (Exception e) {
|
|
|
- IOUtils.closeWhileHandlingException(translogView);
|
|
|
- throw new RecoveryEngineException(shard.shardId(), 1, "Snapshot failed", e);
|
|
|
- }
|
|
|
+ try (final Translog.View translogView = shard.acquireTranslogView()) {
|
|
|
+ logger.trace("{} captured translog id [{}] for recovery", shard.shardId(), translogView.minTranslogGeneration());
|
|
|
|
|
|
- try {
|
|
|
- phase1(phase1Snapshot, translogView);
|
|
|
- } catch (Exception e) {
|
|
|
- throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
|
|
|
- } finally {
|
|
|
+ boolean isSequenceNumberBasedRecoveryPossible = request.startingSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO &&
|
|
|
+ isTranslogReadyForSequenceNumberBasedRecovery(translogView);
|
|
|
+
|
|
|
+ if (!isSequenceNumberBasedRecoveryPossible) {
|
|
|
+ final IndexCommit phase1Snapshot;
|
|
|
try {
|
|
|
- shard.releaseIndexCommit(phase1Snapshot);
|
|
|
- } catch (IOException ex) {
|
|
|
- logger.warn("releasing snapshot caused exception", ex);
|
|
|
+ phase1Snapshot = shard.acquireIndexCommit(false);
|
|
|
+ } catch (final Exception e) {
|
|
|
+ IOUtils.closeWhileHandlingException(translogView);
|
|
|
+ throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e);
|
|
|
+ }
|
|
|
+ try {
|
|
|
+ phase1(phase1Snapshot, translogView);
|
|
|
+ } catch (final Exception e) {
|
|
|
+ throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e);
|
|
|
+ } finally {
|
|
|
+ try {
|
|
|
+ shard.releaseIndexCommit(phase1Snapshot);
|
|
|
+ } catch (final IOException ex) {
|
|
|
+ logger.warn("releasing snapshot caused exception", ex);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // engine was just started at the end of phase 1
|
|
|
+ try {
|
|
|
+ prepareTargetForTranslog(translogView.totalOperations(), shard.segmentStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
+ } catch (final Exception e) {
|
|
|
+ throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e);
|
|
|
+ }
|
|
|
+
|
|
|
+ // engine was just started at the end of phase1
|
|
|
if (shard.state() == IndexShardState.RELOCATED) {
|
|
|
assert request.isPrimaryRelocation() == false :
|
|
|
"recovery target should not retry primary relocation if previous attempt made it past finalization step";
|
|
|
- /**
|
|
|
+ /*
|
|
|
* The primary shard has been relocated while we copied files. This means that we can't guarantee any more that all
|
|
|
* operations that were replicated during the file copy (when the target engine was not yet opened) will be present in the
|
|
|
- * local translog and thus will be resent on phase 2. The reason is that an operation replicated by the target primary is
|
|
|
+ * local translog and thus will be resent on phase2. The reason is that an operation replicated by the target primary is
|
|
|
* sent to the recovery target and the local shard (old primary) concurrently, meaning it may have arrived at the recovery
|
|
|
* target before we opened the engine and is still in-flight on the local shard.
|
|
|
*
|
|
|
* Checking the relocated status here, after we opened the engine on the target, is safe because primary relocation waits
|
|
|
* for all ongoing operations to complete and be fully replicated. Therefore all future operation by the new primary are
|
|
|
- * guaranteed to reach the target shard when it's engine is open.
|
|
|
+ * guaranteed to reach the target shard when its engine is open.
|
|
|
*/
|
|
|
throw new IndexShardRelocatedException(request.shardId());
|
|
|
}
|
|
|
|
|
|
- logger.trace("{} snapshot translog for recovery. current size is [{}]", shard.shardId(), translogView.totalOperations());
|
|
|
+ logger.trace("{} snapshot translog for recovery; current size is [{}]", shard.shardId(), translogView.totalOperations());
|
|
|
try {
|
|
|
phase2(translogView.snapshot());
|
|
|
} catch (Exception e) {
|
|
@@ -176,6 +189,49 @@ public class RecoverySourceHandler {
|
|
|
return response;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Determines if the source translog is ready for a sequence-number-based peer recovery. The main condition here is that the source
|
|
|
+ * translog contains all operations between the local checkpoint on the target and the current maximum sequence number on the source.
|
|
|
+ *
|
|
|
+ * @param translogView a view of the translog on the source
|
|
|
+ * @return {@code true} if the source is ready for a sequence-number-based recovery
|
|
|
+ * @throws IOException if an I/O exception occurred reading the translog snapshot
|
|
|
+ */
|
|
|
+ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) throws IOException {
|
|
|
+ final long startingSeqNo = request.startingSeqNo();
|
|
|
+ assert startingSeqNo >= 0;
|
|
|
+ final long endingSeqNo = shard.seqNoStats().getMaxSeqNo();
|
|
|
+ logger.trace("{} starting: [{}], ending: [{}]", shard.shardId(), startingSeqNo, endingSeqNo);
|
|
|
+ // the start recovery request is initialized with the starting sequence number set to the target shard's local checkpoint plus one
|
|
|
+ if (startingSeqNo - 1 <= endingSeqNo) {
|
|
|
+ logger.trace(
|
|
|
+ "{} waiting for all operations in the range [{}, {}] to complete",
|
|
|
+ shard.shardId(),
|
|
|
+ startingSeqNo,
|
|
|
+ endingSeqNo);
|
|
|
+ /*
|
|
|
+ * We need to wait for all operations up to the current max to complete, otherwise we can not guarantee that all
|
|
|
+ * operations in the required range will be available for replaying from the translog of the source.
|
|
|
+ */
|
|
|
+ cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo));
|
|
|
+
|
|
|
+ final LocalCheckpointTracker tracker = new LocalCheckpointTracker(shard.indexSettings(), startingSeqNo, startingSeqNo - 1);
|
|
|
+ final Translog.Snapshot snapshot = translogView.snapshot();
|
|
|
+ Translog.Operation operation;
|
|
|
+ while ((operation = snapshot.next()) != null) {
|
|
|
+ if (operation.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) {
|
|
|
+ tracker.markSeqNoAsCompleted(operation.seqNo());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return tracker.getCheckpoint() >= endingSeqNo;
|
|
|
+ } else {
|
|
|
+ // norelease this can currently happen if a snapshot restore rolls the primary back to a previous commit point; in this
|
|
|
+ // situation the local checkpoint on the replica can be far in advance of the maximum sequence number on the primary violating
|
|
|
+ // all assumptions regarding local and global checkpoints
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Perform phase1 of the recovery operations. Once this {@link IndexCommit}
|
|
|
* snapshot has been performed no commit operations (files being fsync'd)
|
|
@@ -237,7 +293,7 @@ public class RecoverySourceHandler {
|
|
|
response.phase1ExistingFileSizes.add(md.length());
|
|
|
existingTotalSize += md.length();
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
- logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exists in local store and has checksum [{}]," +
|
|
|
+ logger.trace("[{}][{}] recovery [phase1] to {}: not recovering [{}], exist in local store and has checksum [{}]," +
|
|
|
" size [{}]",
|
|
|
indexName, shardId, request.targetNode(), md.name(), md.checksum(), md.length());
|
|
|
}
|
|
@@ -252,7 +308,7 @@ public class RecoverySourceHandler {
|
|
|
"[{}], local [{}]",
|
|
|
indexName, shardId, request.targetNode(), md.name(), request.metadataSnapshot().asMap().get(md.name()), md);
|
|
|
} else {
|
|
|
- logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exists in remote",
|
|
|
+ logger.trace("[{}][{}] recovery [phase1] to {}: recovering [{}], does not exist in remote",
|
|
|
indexName, shardId, request.targetNode(), md.name());
|
|
|
}
|
|
|
response.phase1FileNames.add(md.name());
|
|
@@ -329,8 +385,6 @@ public class RecoverySourceHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- prepareTargetForTranslog(translogView.totalOperations(), shard.segmentStats(false).getMaxUnsafeAutoIdTimestamp());
|
|
|
-
|
|
|
logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime());
|
|
|
response.phase1Time = stopWatch.totalTime().millis();
|
|
|
} catch (Exception e) {
|
|
@@ -340,14 +394,12 @@ public class RecoverySourceHandler {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
-
|
|
|
- protected void prepareTargetForTranslog(final int totalTranslogOps, long maxUnsafeAutoIdTimestamp) throws IOException {
|
|
|
+ void prepareTargetForTranslog(final int totalTranslogOps, final long maxUnsafeAutoIdTimestamp) throws IOException {
|
|
|
StopWatch stopWatch = new StopWatch().start();
|
|
|
logger.trace("{} recovery [phase1] to {}: prepare remote engine for translog", request.shardId(), request.targetNode());
|
|
|
final long startEngineStart = stopWatch.totalTime().millis();
|
|
|
- // Send a request preparing the new shard's translog to receive
|
|
|
- // operations. This ensures the shard engine is started and disables
|
|
|
- // garbage collection (not the JVM's GC!) of tombstone deletes
|
|
|
+ // Send a request preparing the new shard's translog to receive operations. This ensures the shard engine is started and disables
|
|
|
+ // garbage collection (not the JVM's GC!) of tombstone deletes.
|
|
|
cancellableThreads.executeIO(() -> recoveryTarget.prepareForTranslogOperations(totalTranslogOps, maxUnsafeAutoIdTimestamp));
|
|
|
stopWatch.stop();
|
|
|
|
|
@@ -357,31 +409,34 @@ public class RecoverySourceHandler {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Perform phase2 of the recovery process
|
|
|
+ * Perform phase two of the recovery process.
|
|
|
* <p>
|
|
|
- * Phase2 takes a snapshot of the current translog *without* acquiring the
|
|
|
- * write lock (however, the translog snapshot is a point-in-time view of
|
|
|
- * the translog). It then sends each translog operation to the target node
|
|
|
- * so it can be replayed into the new shard.
|
|
|
+ * Phase two uses a snapshot of the current translog *without* acquiring the write lock (however, the translog snapshot is
|
|
|
+ * point-in-time view of the translog). It then sends each translog operation to the target node so it can be replayed into the new
|
|
|
+ * shard.
|
|
|
+ *
|
|
|
+ * @param snapshot a snapshot of the translog
|
|
|
*/
|
|
|
- public void phase2(Translog.Snapshot snapshot) {
|
|
|
+ void phase2(final Translog.Snapshot snapshot) throws IOException {
|
|
|
if (shard.state() == IndexShardState.CLOSED) {
|
|
|
throw new IndexShardClosedException(request.shardId());
|
|
|
}
|
|
|
cancellableThreads.checkForCancel();
|
|
|
|
|
|
- StopWatch stopWatch = new StopWatch().start();
|
|
|
+ final StopWatch stopWatch = new StopWatch().start();
|
|
|
|
|
|
logger.trace("{} recovery [phase2] to {}: sending transaction log operations", request.shardId(), request.targetNode());
|
|
|
- // Send all the snapshot's translog operations to the target
|
|
|
- int totalOperations = sendSnapshot(snapshot);
|
|
|
+
|
|
|
+ // send all the snapshot's translog operations to the target
|
|
|
+ final int totalOperations = sendSnapshot(request.startingSeqNo(), snapshot);
|
|
|
+
|
|
|
stopWatch.stop();
|
|
|
logger.trace("{} recovery [phase2] to {}: took [{}]", request.shardId(), request.targetNode(), stopWatch.totalTime());
|
|
|
response.phase2Time = stopWatch.totalTime().millis();
|
|
|
response.phase2Operations = totalOperations;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
+ /*
|
|
|
* finalizes the recovery process
|
|
|
*/
|
|
|
public void finalizeRecovery() {
|
|
@@ -410,7 +465,7 @@ public class RecoverySourceHandler {
|
|
|
logger.trace("[{}][{}] performing relocation hand-off to {}", indexName, shardId, request.targetNode());
|
|
|
cancellableThreads.execute(() -> shard.relocated("to " + request.targetNode()));
|
|
|
}
|
|
|
- /**
|
|
|
+ /*
|
|
|
* if the recovery process fails after setting the shard state to RELOCATED, both relocation source and
|
|
|
* target are failed (see {@link IndexShard#updateRoutingEntry}).
|
|
|
*/
|
|
@@ -421,77 +476,73 @@ public class RecoverySourceHandler {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Send the given snapshot's operations to this handler's target node.
|
|
|
+ * Send the given snapshot's operations with a sequence number greater than the specified staring sequence number to this handler's
|
|
|
+ * target node.
|
|
|
* <p>
|
|
|
- * Operations are bulked into a single request depending on an operation
|
|
|
- * count limit or size-in-bytes limit
|
|
|
+ * Operations are bulked into a single request depending on an operation count limit or size-in-bytes limit.
|
|
|
*
|
|
|
+ * @param startingSeqNo the sequence number for which only operations with a sequence number greater than this will be sent
|
|
|
+ * @param snapshot the translog snapshot to replay operations from
|
|
|
* @return the total number of translog operations that were sent
|
|
|
+ * @throws IOException if an I/O exception occurred reading the translog snapshot
|
|
|
*/
|
|
|
- protected int sendSnapshot(final Translog.Snapshot snapshot) {
|
|
|
+ protected int sendSnapshot(final long startingSeqNo, final Translog.Snapshot snapshot) throws IOException {
|
|
|
int ops = 0;
|
|
|
long size = 0;
|
|
|
int totalOperations = 0;
|
|
|
final List<Translog.Operation> operations = new ArrayList<>();
|
|
|
- Translog.Operation operation;
|
|
|
- try {
|
|
|
- operation = snapshot.next(); // this ex should bubble up
|
|
|
- } catch (IOException ex) {
|
|
|
- throw new ElasticsearchException("failed to get next operation from translog", ex);
|
|
|
- }
|
|
|
|
|
|
- if (operation == null) {
|
|
|
- logger.trace("[{}][{}] no translog operations to send to {}",
|
|
|
- indexName, shardId, request.targetNode());
|
|
|
+ if (snapshot.totalOperations() == 0) {
|
|
|
+ logger.trace("[{}][{}] no translog operations to send to {}", indexName, shardId, request.targetNode());
|
|
|
}
|
|
|
- while (operation != null) {
|
|
|
+
|
|
|
+ // send operations in batches
|
|
|
+ Translog.Operation operation;
|
|
|
+ while ((operation = snapshot.next()) != null) {
|
|
|
if (shard.state() == IndexShardState.CLOSED) {
|
|
|
throw new IndexShardClosedException(request.shardId());
|
|
|
}
|
|
|
cancellableThreads.checkForCancel();
|
|
|
+ // we have to send older ops for which no sequence number was assigned, and any ops after the starting sequence number
|
|
|
+ if (operation.seqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO || operation.seqNo() < startingSeqNo) continue;
|
|
|
operations.add(operation);
|
|
|
- ops += 1;
|
|
|
+ ops++;
|
|
|
size += operation.estimateSize();
|
|
|
totalOperations++;
|
|
|
|
|
|
- // Check if this request is past bytes threshold, and
|
|
|
- // if so, send it off
|
|
|
+ // check if this request is past bytes threshold, and if so, send it off
|
|
|
if (size >= chunkSizeInBytes) {
|
|
|
-
|
|
|
- // don't throttle translog, since we lock for phase3 indexing,
|
|
|
- // so we need to move it as fast as possible. Note, since we
|
|
|
- // index docs to replicas while the index files are recovered
|
|
|
- // the lock can potentially be removed, in which case, it might
|
|
|
- // make sense to re-enable throttling in this phase
|
|
|
cancellableThreads.execute(() -> recoveryTarget.indexTranslogOperations(operations, snapshot.totalOperations()));
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
logger.trace("[{}][{}] sent batch of [{}][{}] (total: [{}]) translog operations to {}",
|
|
|
- indexName, shardId, ops, new ByteSizeValue(size),
|
|
|
- snapshot.totalOperations(),
|
|
|
- request.targetNode());
|
|
|
+ indexName,
|
|
|
+ shardId,
|
|
|
+ ops,
|
|
|
+ new ByteSizeValue(size),
|
|
|
+ snapshot.totalOperations(),
|
|
|
+ request.targetNode());
|
|
|
}
|
|
|
-
|
|
|
ops = 0;
|
|
|
size = 0;
|
|
|
operations.clear();
|
|
|
}
|
|
|
- try {
|
|
|
- operation = snapshot.next(); // this ex should bubble up
|
|
|
- } catch (IOException ex) {
|
|
|
- throw new ElasticsearchException("failed to get next operation from translog", ex);
|
|
|
- }
|
|
|
}
|
|
|
- // send the leftover
|
|
|
+
|
|
|
+ // send the leftover operations
|
|
|
if (!operations.isEmpty()) {
|
|
|
cancellableThreads.execute(() -> recoveryTarget.indexTranslogOperations(operations, snapshot.totalOperations()));
|
|
|
-
|
|
|
}
|
|
|
+
|
|
|
if (logger.isTraceEnabled()) {
|
|
|
logger.trace("[{}][{}] sent final batch of [{}][{}] (total: [{}]) translog operations to {}",
|
|
|
- indexName, shardId, ops, new ByteSizeValue(size),
|
|
|
- snapshot.totalOperations(),
|
|
|
- request.targetNode());
|
|
|
+ indexName,
|
|
|
+ shardId,
|
|
|
+ ops,
|
|
|
+ new ByteSizeValue(size),
|
|
|
+ snapshot.totalOperations(),
|
|
|
+ request.targetNode());
|
|
|
}
|
|
|
+
|
|
|
return totalOperations;
|
|
|
}
|
|
|
|