Replica start peer recovery with safe commit (#28181)

Today a replica starts a peer-recovery with the last commit. If the last
commit is not a safe commit, a replica will immediately fallback to the
file based sync which is more expensive than the sequence based
recovery. This commit modifies the peer-recovery in replica to start
with a safe commit. Moreover we can keep the existing translog on the
target if the recovery is sequence based recovery.

Relates #10708
This commit is contained in:
Nhat Nguyen 2018-01-12 19:09:31 -05:00 committed by GitHub
parent f2db2a02e2
commit 095f31b80e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 197 additions and 134 deletions

View File

@ -1512,6 +1512,11 @@ public abstract class Engine implements Closeable {
*/ */
public abstract Engine recoverFromTranslog() throws IOException; public abstract Engine recoverFromTranslog() throws IOException;
/**
* Do not replay translog operations, but make the engine be ready.
*/
public abstract void skipTranslogRecovery();
/** /**
* Returns <code>true</code> iff this engine is currently recovering from translog. * Returns <code>true</code> iff this engine is currently recovering from translog.
*/ */

View File

@ -401,6 +401,15 @@ public class InternalEngine extends Engine {
return this; return this;
} }
@Override
public void skipTranslogRecovery() {
if (openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
throw new IllegalStateException("Can't skip translog recovery with open mode: " + openMode);
}
assert pendingTranslogRecovery.get() : "translogRecovery is not pending but should be";
pendingTranslogRecovery.set(false); // we are good - now we can commit
}
private IndexCommit getStartingCommitPoint() throws IOException { private IndexCommit getStartingCommitPoint() throws IOException {
if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
final long lastSyncedGlobalCheckpoint = translog.getLastSyncedGlobalCheckpoint(); final long lastSyncedGlobalCheckpoint = translog.getLastSyncedGlobalCheckpoint();

View File

@ -1304,9 +1304,20 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
* opens the engine on top of the existing lucene engine and translog. * opens the engine on top of the existing lucene engine and translog.
* Operations from the translog will be replayed to bring lucene up to date. * Operations from the translog will be replayed to bring lucene up to date.
**/ **/
public void openIndexAndTranslog() throws IOException { public void openIndexAndRecoveryFromTranslog() throws IOException {
assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE; assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE;
innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, false); innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, false);
getEngine().recoverFromTranslog();
}
/**
* Opens the engine on top of the existing lucene engine and translog.
* The translog is kept but its operations won't be replayed.
*/
public void openIndexAndSkipTranslogRecovery() throws IOException {
assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.PEER;
innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, false);
getEngine().skipTranslogRecovery();
} }
private void innerOpenEngineAndTranslog(final EngineConfig.OpenMode openMode, final boolean forceNewHistoryUUID) throws IOException { private void innerOpenEngineAndTranslog(final EngineConfig.OpenMode openMode, final boolean forceNewHistoryUUID) throws IOException {
@ -1339,13 +1350,12 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
globalCheckpointTracker.updateGlobalCheckpointOnReplica(Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()), globalCheckpointTracker.updateGlobalCheckpointOnReplica(Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()),
"read from translog checkpoint"); "read from translog checkpoint");
} }
Engine newEngine = createNewEngine(config); createNewEngine(config);
verifyNotClosed(); verifyNotClosed();
if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) {
// We set active because we are now writing operations to the engine; this way, if we go idle after some time and become inactive, // We set active because we are now writing operations to the engine; this way, if we go idle after some time and become inactive,
// we still give sync'd flush a chance to run: // we still give sync'd flush a chance to run:
active.set(true); active.set(true);
newEngine.recoverFromTranslog();
} }
assertSequenceNumbersInCommit(); assertSequenceNumbersInCommit();
assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage();

View File

@ -401,7 +401,7 @@ final class StoreRecovery {
logger.debug("failed to list file details", e); logger.debug("failed to list file details", e);
} }
if (indexShouldExists) { if (indexShouldExists) {
indexShard.openIndexAndTranslog(); indexShard.openIndexAndRecoveryFromTranslog();
indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm());
} else { } else {
indexShard.createIndexAndTranslog(); indexShard.createIndexAndTranslog();

View File

@ -21,6 +21,8 @@ package org.elasticsearch.indices.recovery;
import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.logging.log4j.util.Supplier; import org.apache.logging.log4j.util.Supplier;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.RateLimiter; import org.apache.lucene.store.RateLimiter;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
@ -39,6 +41,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.CancellableThreads;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.engine.CombinedDeletionPolicy;
import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.engine.RecoveryEngineException;
import org.elasticsearch.index.mapper.MapperException; import org.elasticsearch.index.mapper.MapperException;
import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbers;
@ -60,6 +63,7 @@ import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -108,8 +112,8 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
FileChunkTransportRequestHandler()); FileChunkTransportRequestHandler());
transportService.registerRequestHandler(Actions.CLEAN_FILES, RecoveryCleanFilesRequest::new, ThreadPool.Names.GENERIC, new transportService.registerRequestHandler(Actions.CLEAN_FILES, RecoveryCleanFilesRequest::new, ThreadPool.Names.GENERIC, new
CleanFilesRequestHandler()); CleanFilesRequestHandler());
transportService.registerRequestHandler(Actions.PREPARE_TRANSLOG, RecoveryPrepareForTranslogOperationsRequest::new, ThreadPool transportService.registerRequestHandler(Actions.PREPARE_TRANSLOG, ThreadPool.Names.GENERIC,
.Names.GENERIC, new PrepareForTranslogOperationsRequestHandler()); RecoveryPrepareForTranslogOperationsRequest::new, new PrepareForTranslogOperationsRequestHandler());
transportService.registerRequestHandler(Actions.TRANSLOG_OPS, RecoveryTranslogOperationsRequest::new, ThreadPool.Names.GENERIC, transportService.registerRequestHandler(Actions.TRANSLOG_OPS, RecoveryTranslogOperationsRequest::new, ThreadPool.Names.GENERIC,
new TranslogOperationsRequestHandler()); new TranslogOperationsRequestHandler());
transportService.registerRequestHandler(Actions.FINALIZE, RecoveryFinalizeRecoveryRequest::new, ThreadPool.Names.GENERIC, new transportService.registerRequestHandler(Actions.FINALIZE, RecoveryFinalizeRecoveryRequest::new, ThreadPool.Names.GENERIC, new
@ -353,7 +357,9 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) {
try { try {
final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation()); final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation());
final SequenceNumbers.CommitInfo seqNoStats = recoveryTarget.store().loadSeqNoInfo(null); final List<IndexCommit> existingCommits = DirectoryReader.listCommits(recoveryTarget.store().directory());
final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, globalCheckpoint);
final SequenceNumbers.CommitInfo seqNoStats = recoveryTarget.store().loadSeqNoInfo(safeCommit);
if (seqNoStats.maxSeqNo <= globalCheckpoint) { if (seqNoStats.maxSeqNo <= globalCheckpoint) {
assert seqNoStats.localCheckpoint <= globalCheckpoint; assert seqNoStats.localCheckpoint <= globalCheckpoint;
/* /*
@ -387,7 +393,7 @@ public class PeerRecoveryTargetService extends AbstractComponent implements Inde
public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, TransportChannel channel) throws Exception { public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, TransportChannel channel) throws Exception {
try (RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId() try (RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId()
)) { )) {
recoveryRef.target().prepareForTranslogOperations(request.totalTranslogOps()); recoveryRef.target().prepareForTranslogOperations(request.deleteLocalTranslog(), request.totalTranslogOps());
} }
channel.sendResponse(TransportResponse.Empty.INSTANCE); channel.sendResponse(TransportResponse.Empty.INSTANCE);
} }

View File

@ -28,19 +28,33 @@ import org.elasticsearch.transport.TransportRequest;
import java.io.IOException; import java.io.IOException;
public class RecoveryPrepareForTranslogOperationsRequest extends TransportRequest { class RecoveryPrepareForTranslogOperationsRequest extends TransportRequest {
private long recoveryId; private final long recoveryId;
private ShardId shardId; private final ShardId shardId;
private int totalTranslogOps = RecoveryState.Translog.UNKNOWN; private final int totalTranslogOps;
private final boolean deleteLocalTranslog;
public RecoveryPrepareForTranslogOperationsRequest() { RecoveryPrepareForTranslogOperationsRequest(long recoveryId, ShardId shardId, int totalTranslogOps, boolean deleteLocalTranslog) {
}
RecoveryPrepareForTranslogOperationsRequest(long recoveryId, ShardId shardId, int totalTranslogOps) {
this.recoveryId = recoveryId; this.recoveryId = recoveryId;
this.shardId = shardId; this.shardId = shardId;
this.totalTranslogOps = totalTranslogOps; this.totalTranslogOps = totalTranslogOps;
this.deleteLocalTranslog = deleteLocalTranslog;
}
RecoveryPrepareForTranslogOperationsRequest(StreamInput in) throws IOException {
super.readFrom(in);
recoveryId = in.readLong();
shardId = ShardId.readShardId(in);
totalTranslogOps = in.readVInt();
if (in.getVersion().before(Version.V_6_0_0_alpha1)) {
in.readLong(); // maxUnsafeAutoIdTimestamp
}
if (in.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
deleteLocalTranslog = in.readBoolean();
} else {
deleteLocalTranslog = true;
}
} }
public long recoveryId() { public long recoveryId() {
@ -55,15 +69,11 @@ public class RecoveryPrepareForTranslogOperationsRequest extends TransportReques
return totalTranslogOps; return totalTranslogOps;
} }
@Override /**
public void readFrom(StreamInput in) throws IOException { * Whether or not the recover target should delete its local translog
super.readFrom(in); */
recoveryId = in.readLong(); boolean deleteLocalTranslog() {
shardId = ShardId.readShardId(in); return deleteLocalTranslog;
totalTranslogOps = in.readVInt();
if (in.getVersion().before(Version.V_6_0_0_alpha1)) {
in.readLong(); // maxUnsafeAutoIdTimestamp
}
} }
@Override @Override
@ -75,5 +85,8 @@ public class RecoveryPrepareForTranslogOperationsRequest extends TransportReques
if (out.getVersion().before(Version.V_6_0_0_alpha1)) { if (out.getVersion().before(Version.V_6_0_0_alpha1)) {
out.writeLong(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP); // maxUnsafeAutoIdTimestamp out.writeLong(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP); // maxUnsafeAutoIdTimestamp
} }
if (out.getVersion().onOrAfter(Version.V_7_0_0_alpha1)) {
out.writeBoolean(deleteLocalTranslog);
}
} }
} }

View File

@ -150,9 +150,9 @@ public class RecoverySourceHandler {
final long startingSeqNo; final long startingSeqNo;
final long requiredSeqNoRangeStart; final long requiredSeqNoRangeStart;
final boolean isSequenceNumberBasedRecoveryPossible = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO &&
isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery(); isTargetSameHistory() && isTranslogReadyForSequenceNumberBasedRecovery();
if (isSequenceNumberBasedRecoveryPossible) { if (isSequenceNumberBasedRecovery) {
logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo());
startingSeqNo = request.startingSeqNo(); startingSeqNo = request.startingSeqNo();
requiredSeqNoRangeStart = startingSeqNo; requiredSeqNoRangeStart = startingSeqNo;
@ -188,7 +188,8 @@ public class RecoverySourceHandler {
runUnderPrimaryPermit(() -> shard.initiateTracking(request.targetAllocationId())); runUnderPrimaryPermit(() -> shard.initiateTracking(request.targetAllocationId()));
try { try {
prepareTargetForTranslog(translog.estimateTotalOperationsFromMinSeq(startingSeqNo)); // For a sequence based recovery, the target can keep its local translog
prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, translog.estimateTotalOperationsFromMinSeq(startingSeqNo));
} catch (final Exception e) { } catch (final Exception e) {
throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e);
} }
@ -421,13 +422,13 @@ public class RecoverySourceHandler {
} }
} }
void prepareTargetForTranslog(final int totalTranslogOps) throws IOException { void prepareTargetForTranslog(final boolean createNewTranslog, final int totalTranslogOps) throws IOException {
StopWatch stopWatch = new StopWatch().start(); StopWatch stopWatch = new StopWatch().start();
logger.trace("recovery [phase1]: prepare remote engine for translog"); logger.trace("recovery [phase1]: prepare remote engine for translog");
final long startEngineStart = stopWatch.totalTime().millis(); 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 // 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. // garbage collection (not the JVM's GC!) of tombstone deletes.
cancellableThreads.executeIO(() -> recoveryTarget.prepareForTranslogOperations(totalTranslogOps)); cancellableThreads.executeIO(() -> recoveryTarget.prepareForTranslogOperations(createNewTranslog, totalTranslogOps));
stopWatch.stop(); stopWatch.stop();
response.startTime = stopWatch.totalTime().millis() - startEngineStart; response.startTime = stopWatch.totalTime().millis() - startEngineStart;

View File

@ -362,10 +362,14 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget
/*** Implementation of {@link RecoveryTargetHandler } */ /*** Implementation of {@link RecoveryTargetHandler } */
@Override @Override
public void prepareForTranslogOperations(int totalTranslogOps) throws IOException { public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException {
state().getTranslog().totalOperations(totalTranslogOps); state().getTranslog().totalOperations(totalTranslogOps);
// TODO: take the local checkpoint from store as global checkpoint, once we know it's safe if (createNewTranslog) {
indexShard().openIndexAndCreateTranslog(false, SequenceNumbers.UNASSIGNED_SEQ_NO); // TODO: Assigns the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2
indexShard().openIndexAndCreateTranslog(false, SequenceNumbers.UNASSIGNED_SEQ_NO);
} else {
indexShard().openIndexAndSkipTranslogRecovery();
}
} }
@Override @Override

View File

@ -32,10 +32,10 @@ public interface RecoveryTargetHandler {
/** /**
* Prepares the target to receive translog operations, after all file have been copied * Prepares the target to receive translog operations, after all file have been copied
* * @param createNewTranslog whether or not to delete the local translog on the target
* @param totalTranslogOps total translog operations expected to be sent * @param totalTranslogOps total translog operations expected to be sent
*/ */
void prepareForTranslogOperations(int totalTranslogOps) throws IOException; void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException;
/** /**
* The finalize request refreshes the engine now that new segments are available, enables garbage collection of tombstone files, and * The finalize request refreshes the engine now that new segments are available, enables garbage collection of tombstone files, and

View File

@ -76,9 +76,9 @@ public class RemoteRecoveryTargetHandler implements RecoveryTargetHandler {
} }
@Override @Override
public void prepareForTranslogOperations(int totalTranslogOps) throws IOException { public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException {
transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG, transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG,
new RecoveryPrepareForTranslogOperationsRequest(recoveryId, shardId, totalTranslogOps), new RecoveryPrepareForTranslogOperationsRequest(recoveryId, shardId, totalTranslogOps, createNewTranslog),
TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(),
EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); EmptyTransportResponseHandler.INSTANCE_SAME).txGet();
} }

View File

@ -31,7 +31,9 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
@ -226,7 +228,6 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
final IndexShard oldPrimary = shards.getPrimary(); final IndexShard oldPrimary = shards.getPrimary();
final IndexShard newPrimary = shards.getReplicas().get(0); final IndexShard newPrimary = shards.getReplicas().get(0);
final IndexShard replica = shards.getReplicas().get(1); final IndexShard replica = shards.getReplicas().get(1);
boolean expectSeqNoRecovery = true;
if (randomBoolean()) { if (randomBoolean()) {
// simulate docs that were inflight when primary failed, these will be rolled back // simulate docs that were inflight when primary failed, these will be rolled back
final int rollbackDocs = randomIntBetween(1, 5); final int rollbackDocs = randomIntBetween(1, 5);
@ -239,7 +240,6 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
} }
if (randomBoolean()) { if (randomBoolean()) {
oldPrimary.flush(new FlushRequest(index.getName())); oldPrimary.flush(new FlushRequest(index.getName()));
expectSeqNoRecovery = false;
} }
} }
@ -252,9 +252,30 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
equalTo(totalDocs - 1L)); equalTo(totalDocs - 1L));
// index some more // index some more
totalDocs += shards.indexDocs(randomIntBetween(0, 5)); int moreDocs = shards.indexDocs(randomIntBetween(0, 5));
totalDocs += moreDocs;
// As a replica keeps a safe commit, the file-based recovery only happens if the required translog
// for the sequence based recovery are not fully retained and extra documents were added to the primary.
boolean expectSeqNoRecovery = (moreDocs == 0 || randomBoolean());
int uncommittedOpsOnPrimary = 0;
if (expectSeqNoRecovery == false) {
IndexMetaData.Builder builder = IndexMetaData.builder(newPrimary.indexSettings().getIndexMetaData());
builder.settings(Settings.builder().put(newPrimary.indexSettings().getSettings())
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_AGE_SETTING.getKey(), "-1")
.put(IndexSettings.INDEX_TRANSLOG_RETENTION_SIZE_SETTING.getKey(), "-1")
);
newPrimary.indexSettings().updateIndexMetaData(builder.build());
newPrimary.onSettingsChanged();
shards.syncGlobalCheckpoint();
newPrimary.flush(new FlushRequest());
uncommittedOpsOnPrimary = shards.indexDocs(randomIntBetween(0, 10));
totalDocs += uncommittedOpsOnPrimary;
}
if (randomBoolean()) { if (randomBoolean()) {
uncommittedOpsOnPrimary = 0;
shards.syncGlobalCheckpoint();
newPrimary.flush(new FlushRequest()); newPrimary.flush(new FlushRequest());
} }
@ -269,7 +290,7 @@ public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestC
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs - committedDocs)); assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs - committedDocs));
} else { } else {
assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty())); assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs)); assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedOpsOnPrimary));
} }
// roll back the extra ops in the replica // roll back the extra ops in the replica

View File

@ -22,7 +22,6 @@ import org.apache.logging.log4j.Logger;
import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
@ -2109,7 +2108,7 @@ public class IndexShardTests extends IndexShardTestCase {
shard.prepareForIndexRecovery(); shard.prepareForIndexRecovery();
// Shard is still inactive since we haven't started recovering yet // Shard is still inactive since we haven't started recovering yet
assertFalse(shard.isActive()); assertFalse(shard.isActive());
shard.openIndexAndTranslog(); shard.openIndexAndRecoveryFromTranslog();
// Shard should now be active since we did recover: // Shard should now be active since we did recover:
assertTrue(shard.isActive()); assertTrue(shard.isActive());
closeShards(shard); closeShards(shard);
@ -2137,8 +2136,8 @@ public class IndexShardTests extends IndexShardTestCase {
new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> { new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> {
}) { }) {
@Override @Override
public void prepareForTranslogOperations(int totalTranslogOps) throws IOException { public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException {
super.prepareForTranslogOperations(totalTranslogOps); super.prepareForTranslogOperations(createNewTranslog, totalTranslogOps);
// Shard is still inactive since we haven't started recovering yet // Shard is still inactive since we haven't started recovering yet
assertFalse(replica.isActive()); assertFalse(replica.isActive());
@ -2186,8 +2185,8 @@ public class IndexShardTests extends IndexShardTestCase {
}) { }) {
// we're only checking that listeners are called when the engine is open, before there is no point // we're only checking that listeners are called when the engine is open, before there is no point
@Override @Override
public void prepareForTranslogOperations(int totalTranslogOps) throws IOException { public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException {
super.prepareForTranslogOperations(totalTranslogOps); super.prepareForTranslogOperations(createNewTranslog, totalTranslogOps);
assertListenerCalled.accept(replica); assertListenerCalled.accept(replica);
} }

View File

@ -19,103 +19,63 @@
package org.elasticsearch.indices.recovery; package org.elasticsearch.indices.recovery;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.IndexShardTestCase;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogWriter;
import java.io.IOException;
import java.nio.channels.FileChannel;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { public class PeerRecoveryTargetServiceTests extends IndexShardTestCase {
public void testGetStartingSeqNo() throws Exception { public void testGetStartingSeqNo() throws Exception {
IndexShard replica = newShard(false); final IndexShard replica = newShard(false);
final AtomicReference<Path> translogLocation = new AtomicReference<>();
RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null) {
@Override
Path translogLocation() {
return translogLocation.get();
}
};
try { try {
recoveryEmptyReplica(replica); // Empty store
int docs = randomIntBetween(1, 10); {
final String index = replica.shardId().getIndexName(); recoveryEmptyReplica(replica);
long seqNo = 0; final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null);
for (int i = 0; i < docs; i++) { assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(0L));
replica.applyIndexOperationOnReplica(seqNo++, 1, VersionType.EXTERNAL, recoveryTarget.decRef();
IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, false, }
SourceToParse.source(index, "type", "doc_" + i, new BytesArray("{}"), XContentType.JSON), // Last commit is good - use it.
update -> {}); final long initDocs = scaledRandomIntBetween(1, 10);
if (rarely()) { {
// insert a gap for (int i = 0; i < initDocs; i++) {
seqNo++; indexDoc(replica, "doc", Integer.toString(i));
} if (randomBoolean()) {
flushShard(replica);
}
}
flushShard(replica);
replica.updateGlobalCheckpointOnReplica(initDocs - 1, "test");
replica.getTranslog().sync();
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null);
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(initDocs));
recoveryTarget.decRef();
}
// Global checkpoint does not advance, last commit is not good - use the previous commit
final int moreDocs = randomIntBetween(1, 10);
{
for (int i = 0; i < moreDocs; i++) {
indexDoc(replica, "doc", Long.toString(i));
if (randomBoolean()) {
flushShard(replica);
}
}
flushShard(replica);
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null);
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(initDocs));
recoveryTarget.decRef();
}
// Advances the global checkpoint, a safe commit also advances
{
replica.updateGlobalCheckpointOnReplica(initDocs + moreDocs - 1, "test");
replica.getTranslog().sync();
final RecoveryTarget recoveryTarget = new RecoveryTarget(replica, null, null, null);
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(initDocs + moreDocs));
recoveryTarget.decRef();
} }
final long maxSeqNo = replica.seqNoStats().getMaxSeqNo();
final long localCheckpoint = replica.getLocalCheckpoint();
translogLocation.set(replica.getTranslog().location());
final Translog translog = replica.getTranslog();
final String translogUUID = translog.getTranslogUUID();
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(0L));
translogLocation.set(writeTranslog(replica.shardId(), translogUUID, translog.currentFileGeneration(), maxSeqNo - 1));
// commit is good, global checkpoint is at least max *committed* which is NO_OPS_PERFORMED
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(0L));
replica.flush(new FlushRequest());
translogLocation.set(replica.getTranslog().location());
// commit is not good, global checkpoint is below max
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO));
translogLocation.set(writeTranslog(replica.shardId(), translogUUID, translog.currentFileGeneration(), maxSeqNo));
// commit is good, global checkpoint is above max
assertThat(PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget), equalTo(localCheckpoint + 1));
} finally { } finally {
closeShards(replica); closeShards(replica);
recoveryTarget.decRef();
} }
} }
private Path writeTranslog(
final ShardId shardId,
final String translogUUID,
final long generation,
final long globalCheckpoint
) throws IOException {
final Path tempDir = createTempDir();
final Path resolve = tempDir.resolve(Translog.getFilename(generation));
Files.createFile(tempDir.resolve(Translog.CHECKPOINT_FILE_NAME));
try (TranslogWriter ignored = TranslogWriter.create(
shardId,
translogUUID,
generation,
resolve,
FileChannel::open,
TranslogConfig.DEFAULT_BUFFER_SIZE, generation, globalCheckpoint, () -> globalCheckpoint, () -> generation)) {}
return tempDir;
}
} }

View File

@ -423,7 +423,7 @@ public class RecoverySourceHandlerTests extends ESTestCase {
} }
@Override @Override
void prepareTargetForTranslog(final int totalTranslogOps) throws IOException { void prepareTargetForTranslog(final boolean createNewTranslog, final int totalTranslogOps) throws IOException {
prepareTargetForTranslogCalled.set(true); prepareTargetForTranslogCalled.set(true);
} }

View File

@ -41,6 +41,7 @@ import org.elasticsearch.index.replication.ESIndexLevelReplicationTestCase;
import org.elasticsearch.index.replication.RecoveryDuringReplicationTests; import org.elasticsearch.index.replication.RecoveryDuringReplicationTests;
import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbers;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.translog.SnapshotMatchers;
import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
@ -271,4 +272,38 @@ public class RecoveryTests extends ESIndexLevelReplicationTestCase {
assertThat(maxSeqNo, lessThanOrEqualTo(globalCheckpoint)); assertThat(maxSeqNo, lessThanOrEqualTo(globalCheckpoint));
closeShards(primaryShard, replicaShard); closeShards(primaryShard, replicaShard);
} }
public void testSequenceBasedRecoveryKeepsTranslog() throws Exception {
try (ReplicationGroup shards = createGroup(1)) {
shards.startAll();
final IndexShard replica = shards.getReplicas().get(0);
final int initDocs = scaledRandomIntBetween(0, 20);
int uncommittedDocs = 0;
for (int i = 0; i < initDocs; i++) {
shards.indexDocs(1);
uncommittedDocs++;
if (randomBoolean()) {
shards.syncGlobalCheckpoint();
shards.flush();
uncommittedDocs = 0;
}
}
shards.removeReplica(replica);
final int moreDocs = shards.indexDocs(scaledRandomIntBetween(0, 20));
if (randomBoolean()) {
shards.flush();
}
replica.close("test", randomBoolean());
replica.store().close();
final IndexShard newReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId());
shards.recoverReplica(newReplica);
try (Translog.Snapshot snapshot = newReplica.getTranslog().newSnapshot()) {
assertThat("Sequence based recovery should keep existing translog", snapshot, SnapshotMatchers.size(initDocs + moreDocs));
}
assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(uncommittedDocs + moreDocs));
assertThat(newReplica.recoveryState().getIndex().fileDetails(), empty());
}
}
} }