From 404b8dc86fca05efef43933b6041664f4eae0203 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 3 Jan 2017 18:53:44 -0500 Subject: [PATCH 01/41] Introduce sequence number-based recovery This commit introduces sequence-number-based recovery. When a replica has fallen out of sync, rather than performing a file-based recovery we first attempt to replay operations since the last local checkpoint on the replica. To do this, at the start of recovery the replica tells the primary what its local checkpoint is. The primary will then wait for all operations between that local checkpoint and the current maximum sequence number to complete; this is to ensure that there are no gaps in the operations that will be replayed from the primary to the replica. This is a best-effort attempt as we currently have no guarantees on the primary that these operations will be available; if we are not able to replay all operations in the desired range, we just fallback to file-based recovery. Later work will strengthen the guarantees. --- .../resources/checkstyle_suppressions.xml | 1 - .../common/settings/IndexScopedSettings.java | 4 +- .../util/concurrent/AbstractRefCounted.java | 1 + .../elasticsearch/index/engine/Engine.java | 3 +- .../index/engine/InternalEngine.java | 55 +++-- ...vice.java => GlobalCheckpointTracker.java} | 33 +-- ...rvice.java => LocalCheckpointTracker.java} | 74 +++--- .../elasticsearch/index/seqno/SeqNoStats.java | 2 +- .../index/seqno/SequenceNumbers.java | 60 +++++ .../index/seqno/SequenceNumbersService.java | 48 ++-- .../elasticsearch/index/shard/IndexShard.java | 18 +- .../org/elasticsearch/index/store/Store.java | 8 + .../index/translog/MultiSnapshot.java | 1 + .../index/translog/Translog.java | 5 + .../cluster/IndicesClusterStateService.java | 4 +- .../recovery/PeerRecoveryTargetService.java | 217 +++++++++++++----- .../recovery/RecoveriesCollection.java | 11 +- .../recovery/RecoverySourceHandler.java | 214 +++++++++++------ .../indices/recovery/RecoveryTarget.java | 64 +++++- .../SharedFSRecoverySourceHandler.java | 18 +- .../recovery/StartRecoveryRequest.java | 44 +++- .../index/engine/InternalEngineTests.java | 32 +-- .../ESIndexLevelReplicationTestCase.java | 45 +++- .../RecoveryDuringReplicationTests.java | 67 +++++- .../index/seqno/GlobalCheckpointTests.java | 113 +++++---- ....java => LocalCheckpointTrackerTests.java} | 138 +++++++---- .../recovery/IndexPrimaryRelocationIT.java | 10 +- .../recovery/RecoverySourceHandlerTests.java | 146 ++++++++---- .../recovery/StartRecoveryRequestTests.java | 27 ++- .../recovery/RecoveriesCollectionTests.java | 4 +- .../elasticsearch/backwards/IndexingIT.java | 2 +- .../index/shard/IndexShardTestCase.java | 83 +++++-- 32 files changed, 1091 insertions(+), 461 deletions(-) rename core/src/main/java/org/elasticsearch/index/seqno/{GlobalCheckpointService.java => GlobalCheckpointTracker.java} (87%) rename core/src/main/java/org/elasticsearch/index/seqno/{LocalCheckpointService.java => LocalCheckpointTracker.java} (75%) create mode 100644 core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java rename core/src/test/java/org/elasticsearch/index/seqno/{LocalCheckpointServiceTests.java => LocalCheckpointTrackerTests.java} (57%) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index b540be685654d..a9bd1373693bc 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -424,7 +424,6 @@ - diff --git a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 959fe1849ea0d..440eaa914fbd6 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -36,7 +36,7 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.FieldMapper; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.seqno.LocalCheckpointService; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.FsDirectoryService; import org.elasticsearch.index.store.Store; @@ -113,7 +113,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.ALLOW_UNMAPPED, IndexSettings.INDEX_CHECK_ON_STARTUP, IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL, - LocalCheckpointService.SETTINGS_BIT_ARRAYS_SIZE, + LocalCheckpointTracker.SETTINGS_BIT_ARRAYS_SIZE, IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, IndexSettings.MAX_SLICES_PER_SCROLL, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/AbstractRefCounted.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/AbstractRefCounted.java index d5b44ed4dfbe9..e0b8aea178c70 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/AbstractRefCounted.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/AbstractRefCounted.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.elasticsearch.common.util.concurrent; import org.apache.lucene.store.AlreadyClosedException; diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 663a10791b6da..97a4850879cc6 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -361,7 +361,7 @@ public Operation.TYPE getOperationType() { void setTranslogLocation(Translog.Location translogLocation) { if (freeze.get() == null) { - assert failure == null : "failure has to be null to set translog location"; + assert failure == null || translogLocation == null: "failure has to be null to set translog location"; this.translogLocation = translogLocation; } else { throw new IllegalStateException("result is already frozen"); @@ -379,6 +379,7 @@ void setTook(long took) { void freeze() { freeze.set(true); } + } public static class IndexResult extends Result { diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index a18ca7f280ee0..e92da51d2d333 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -63,6 +63,7 @@ import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; @@ -119,8 +120,6 @@ public class InternalEngine extends Engine { private final IndexThrottle throttle; private final SequenceNumbersService seqNoService; - static final String LOCAL_CHECKPOINT_KEY = "local_checkpoint"; - static final String MAX_SEQ_NO = "max_seq_no"; // How many callers are currently requesting index throttling. Currently there are only two situations where we do this: when merges // are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling @@ -365,7 +364,7 @@ private Translog.TranslogGeneration loadTranslogIdFromCommit(IndexWriter writer) private static SeqNoStats loadSeqNoStatsFromLuceneAndTranslog( final TranslogConfig translogConfig, final IndexWriter indexWriter) throws IOException { - long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()); + final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()); return loadSeqNoStatsFromLucene(globalCheckpoint, indexWriter); } @@ -378,20 +377,7 @@ private static SeqNoStats loadSeqNoStatsFromLuceneAndTranslog( * @return the sequence number stats */ private static SeqNoStats loadSeqNoStatsFromLucene(final long globalCheckpoint, final IndexWriter indexWriter) { - long maxSeqNo = SequenceNumbersService.NO_OPS_PERFORMED; - long localCheckpoint = SequenceNumbersService.NO_OPS_PERFORMED; - for (Map.Entry entry : indexWriter.getLiveCommitData()) { - final String key = entry.getKey(); - if (key.equals(LOCAL_CHECKPOINT_KEY)) { - assert localCheckpoint == SequenceNumbersService.NO_OPS_PERFORMED; - localCheckpoint = Long.parseLong(entry.getValue()); - } else if (key.equals(MAX_SEQ_NO)) { - assert maxSeqNo == SequenceNumbersService.NO_OPS_PERFORMED : localCheckpoint; - maxSeqNo = Long.parseLong(entry.getValue()); - } - } - - return new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint); + return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, indexWriter.getLiveCommitData()); } private SearcherManager createSearcherManager() throws EngineException { @@ -684,13 +670,20 @@ private IndexResult innerIndex(Index index) throws IOException { final IndexResult indexResult; if (checkVersionConflictResult.isPresent()) { indexResult = checkVersionConflictResult.get(); + // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication + if (indexResult.hasFailure()) { + location = null; + } else { + final Translog.NoOp operation = new Translog.NoOp(seqNo, index.primaryTerm(), "version conflict during recovery"); + location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; + } } else { // no version conflict if (index.origin() == Operation.Origin.PRIMARY) { seqNo = seqNoService().generateSeqNo(); } - /** + /* * Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -707,12 +700,11 @@ private IndexResult innerIndex(Index index) throws IOException { update(index.uid(), index.docs(), indexWriter); } indexResult = new IndexResult(updatedVersion, seqNo, deleted); - location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY - ? translog.add(new Translog.Index(index, indexResult)) - : null; versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); - indexResult.setTranslogLocation(location); + final Translog.Index operation = new Translog.Index(index, indexResult); + location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; } + indexResult.setTranslogLocation(location); indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); return indexResult; @@ -816,21 +808,26 @@ private DeleteResult innerDelete(Delete delete) throws IOException { final DeleteResult deleteResult; if (result.isPresent()) { deleteResult = result.get(); + // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication + if (deleteResult.hasFailure()) { + location = null; + } else { + final Translog.NoOp operation = new Translog.NoOp(seqNo, delete.primaryTerm(), "version conflict during recovery"); + location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; + } } else { if (delete.origin() == Operation.Origin.PRIMARY) { seqNo = seqNoService().generateSeqNo(); } - updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); deleteResult = new DeleteResult(updatedVersion, seqNo, found); - location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY - ? translog.add(new Translog.Delete(delete, deleteResult)) - : null; versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); - deleteResult.setTranslogLocation(location); + final Translog.Delete operation = new Translog.Delete(delete, deleteResult); + location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; } + deleteResult.setTranslogLocation(location); deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); return deleteResult; @@ -1552,11 +1549,11 @@ private void commitIndexWriter(IndexWriter writer, Translog translog, String syn final Map commitData = new HashMap<>(6); commitData.put(Translog.TRANSLOG_GENERATION_KEY, translogFileGen); commitData.put(Translog.TRANSLOG_UUID_KEY, translogUUID); - commitData.put(LOCAL_CHECKPOINT_KEY, localCheckpoint); + commitData.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, localCheckpoint); if (syncId != null) { commitData.put(Engine.SYNC_COMMIT_ID, syncId); } - commitData.put(MAX_SEQ_NO, Long.toString(seqNoService().getMaxSeqNo())); + commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(seqNoService().getMaxSeqNo())); if (logger.isTraceEnabled()) { logger.trace("committing writer with commit data [{}]", commitData); } diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointService.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java similarity index 87% rename from core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointService.java rename to core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 68eaf86f4f8a0..514fc985ce007 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -22,25 +22,25 @@ import com.carrotsearch.hppc.ObjectLongHashMap; import com.carrotsearch.hppc.ObjectLongMap; import com.carrotsearch.hppc.cursors.ObjectLongCursor; +import org.apache.logging.log4j.Logger; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.shard.AbstractIndexShardComponent; -import org.elasticsearch.index.shard.ShardId; import java.util.HashSet; +import java.util.Locale; import java.util.Set; import static org.elasticsearch.index.seqno.SequenceNumbersService.UNASSIGNED_SEQ_NO; /** - * A shard component that is responsible of tracking the global checkpoint. The global checkpoint is the highest sequence number for which - * all lower (or equal) sequence number have been processed on all shards that are currently active. Since shards count as "active" when the - * master starts them, and before this primary shard has been notified of this fact, we also include shards that have completed recovery. - * These shards have received all old operations via the recovery mechanism and are kept up to date by the various replications actions. - * The set of shards that are taken into account for the global checkpoint calculation are called the "in-sync shards". + * This class is responsible of tracking the global checkpoint. The global checkpoint is the highest sequence number for which all lower (or + * equal) sequence number have been processed on all shards that are currently active. Since shards count as "active" when the master starts + * them, and before this primary shard has been notified of this fact, we also include shards that have completed recovery. These shards + * have received all old operations via the recovery mechanism and are kept up to date by the various replications actions. The set of + * shards that are taken into account for the global checkpoint calculation are called the "in-sync shards". *

* The global checkpoint is maintained by the primary shard and is replicated to all the replicas (via {@link GlobalCheckpointSyncAction}). */ -public class GlobalCheckpointService extends AbstractIndexShardComponent { +public class GlobalCheckpointTracker { /* * This map holds the last known local checkpoint for every active shard and initializing shard copies that has been brought up to speed @@ -62,20 +62,22 @@ public class GlobalCheckpointService extends AbstractIndexShardComponent { */ private long globalCheckpoint; + private final Logger logger; + /** - * Initialize the global checkpoint service. The specified global checkpoint should be set to the last known global checkpoint for this - * shard, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO}. + * Initialize the global checkpoint service. The specified global checkpoint should be set to the last known global checkpoint, or + * {@link SequenceNumbersService#UNASSIGNED_SEQ_NO}. * - * @param shardId the shard this service is tracking local checkpoints for * @param indexSettings the index settings * @param globalCheckpoint the last known global checkpoint for this shard, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} + * @param logger a component logger */ - GlobalCheckpointService(final ShardId shardId, final IndexSettings indexSettings, final long globalCheckpoint) { - super(shardId, indexSettings); + GlobalCheckpointTracker(final IndexSettings indexSettings, final long globalCheckpoint, final Logger logger) { assert globalCheckpoint >= UNASSIGNED_SEQ_NO : "illegal initial global checkpoint: " + globalCheckpoint; inSyncLocalCheckpoints = new ObjectLongHashMap<>(1 + indexSettings.getNumberOfReplicas()); assignedAllocationIds = new HashSet<>(1 + indexSettings.getNumberOfReplicas()); this.globalCheckpoint = globalCheckpoint; + this.logger = logger; } /** @@ -127,8 +129,9 @@ synchronized boolean updateCheckpointOnPrimary() { minCheckpoint = Math.min(cp.value, minCheckpoint); } if (minCheckpoint < globalCheckpoint) { - throw new IllegalStateException(shardId + " new global checkpoint [" + minCheckpoint - + "] is lower than previous one [" + globalCheckpoint + "]"); + final String message = + String.format(Locale.ROOT, "new global checkpoint [%d] is lower than previous one [%d]", minCheckpoint, globalCheckpoint); + throw new IllegalStateException(message); } if (globalCheckpoint != minCheckpoint) { logger.trace("global checkpoint updated to [{}]", minCheckpoint); diff --git a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java similarity index 75% rename from core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java rename to core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java index 7da833cf866b0..140b26b450868 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java @@ -20,18 +20,17 @@ package org.elasticsearch.index.seqno; import org.apache.lucene.util.FixedBitSet; +import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.shard.AbstractIndexShardComponent; -import org.elasticsearch.index.shard.ShardId; import java.util.LinkedList; /** - * This class generates sequences numbers and keeps track of the so called local checkpoint - the highest number for which all previous - * sequence numbers have been processed (inclusive). + * 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). */ -public class LocalCheckpointService extends AbstractIndexShardComponent { +public class LocalCheckpointTracker { /** * We keep a bit for each sequence number that is still pending. To optimize allocation, we do so in multiple arrays allocating them on @@ -67,17 +66,14 @@ public class LocalCheckpointService extends AbstractIndexShardComponent { private volatile long nextSeqNo; /** - * Initialize the local checkpoint service. The {@code maxSeqNo} should be set to the last sequence number assigned by this shard, or - * {@link SequenceNumbersService#NO_OPS_PERFORMED} and {@code localCheckpoint} should be set to the last known local checkpoint for this - * shard, or {@link SequenceNumbersService#NO_OPS_PERFORMED}. - * - * @param shardId the shard this service is providing tracking local checkpoints for - * @param indexSettings the index settings - * @param maxSeqNo the last sequence number assigned by this shard, or {@link SequenceNumbersService#NO_OPS_PERFORMED} - * @param localCheckpoint the last known local checkpoint for this shard, or {@link SequenceNumbersService#NO_OPS_PERFORMED} + * Initialize the local checkpoint service. The {@code maxSeqNo} should be set to the last sequence number assigned, or + * {@link SequenceNumbersService#NO_OPS_PERFORMED} and {@code localCheckpoint} should be set to the last known local checkpoint, + * or {@link SequenceNumbersService#NO_OPS_PERFORMED}. + * @param indexSettings the index settings + * @param maxSeqNo the last sequence number assigned, or {@link SequenceNumbersService#NO_OPS_PERFORMED} + * @param localCheckpoint the last known local checkpoint, or {@link SequenceNumbersService#NO_OPS_PERFORMED} */ - LocalCheckpointService(final ShardId shardId, final IndexSettings indexSettings, final long maxSeqNo, final long localCheckpoint) { - super(shardId, indexSettings); + public LocalCheckpointTracker(final IndexSettings indexSettings, final long maxSeqNo, final long localCheckpoint) { if (localCheckpoint < 0 && localCheckpoint != SequenceNumbersService.NO_OPS_PERFORMED) { throw new IllegalArgumentException( "local checkpoint must be non-negative or [" + SequenceNumbersService.NO_OPS_PERFORMED + "] " @@ -107,7 +103,7 @@ synchronized long generateSeqNo() { * * @param seqNo the sequence number to mark as completed */ - synchronized void markSeqNoAsCompleted(final long seqNo) { + public synchronized void markSeqNoAsCompleted(final long seqNo) { // make sure we track highest seen sequence number if (seqNo >= nextSeqNo) { nextSeqNo = seqNo + 1; @@ -142,10 +138,25 @@ long getMaxSeqNo() { return nextSeqNo - 1; } + /** + * Waits for all operations up to the provided sequence number to complete. + * + * @param seqNo the sequence number that the checkpoint must advance to before this method returns + * @throws InterruptedException if the thread was interrupted while blocking on the condition + */ + @SuppressForbidden(reason = "Object#wait") + synchronized void waitForOpsToComplete(final long seqNo) throws InterruptedException { + while (checkpoint < seqNo) { + // notified by updateCheckpoint + this.wait(); + } + } + /** * 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() { assert Thread.holdsLock(this); assert checkpoint < firstProcessedSeqNo + bitArraysSize - 1 : @@ -154,19 +165,24 @@ assert getBitSetForSeqNo(checkpoint + 1) == processedSeqNo.getFirst() : "checkpoint + 1 doesn't point to the first bit set (o.w. current bit set is completed and shouldn't be there)"; assert getBitSetForSeqNo(checkpoint + 1).get(seqNoToBitSetOffset(checkpoint + 1)) : "updateCheckpoint is called but the bit following the checkpoint is not set"; - // keep it simple for now, get the checkpoint one by one; in the future we can optimize and read words - FixedBitSet current = processedSeqNo.getFirst(); - do { - checkpoint++; - // the checkpoint always falls in the first bit set or just before. If it falls - // on the last bit of the current bit set, we can clean it. - if (checkpoint == firstProcessedSeqNo + bitArraysSize - 1) { - processedSeqNo.removeFirst(); - firstProcessedSeqNo += bitArraysSize; - assert checkpoint - firstProcessedSeqNo < bitArraysSize; - current = processedSeqNo.peekFirst(); - } - } while (current != null && current.get(seqNoToBitSetOffset(checkpoint + 1))); + try { + // keep it simple for now, get the checkpoint one by one; in the future we can optimize and read words + FixedBitSet current = processedSeqNo.getFirst(); + do { + checkpoint++; + // the checkpoint always falls in the first bit set or just before. If it falls + // on the last bit of the current bit set, we can clean it. + if (checkpoint == firstProcessedSeqNo + bitArraysSize - 1) { + processedSeqNo.removeFirst(); + firstProcessedSeqNo += bitArraysSize; + assert checkpoint - firstProcessedSeqNo < bitArraysSize; + current = processedSeqNo.peekFirst(); + } + } while (current != null && current.get(seqNoToBitSetOffset(checkpoint + 1))); + } finally { + // notifies waiters in waitForOpsToComplete + this.notifyAll(); + } } /** diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java index b18bfc09400cf..12d82f0813ba3 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SeqNoStats.java @@ -30,7 +30,7 @@ public class SeqNoStats implements ToXContent, Writeable { private static final String SEQ_NO = "seq_no"; - private static final String MAX_SEQ_NO = "max"; + private static final String MAX_SEQ_NO = "max_seq_no"; private static final String LOCAL_CHECKPOINT = "local_checkpoint"; private static final String GLOBAL_CHECKPOINT = "global_checkpoint"; diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java new file mode 100644 index 0000000000000..c3950e1012acc --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java @@ -0,0 +1,60 @@ +/* + * 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.index.seqno; + +import java.util.Map; + +/** + * A utility class for handling sequence numbers. + */ +public class SequenceNumbers { + + public static final String LOCAL_CHECKPOINT_KEY = "local_checkpoint"; + public static final String MAX_SEQ_NO = "max_seq_no"; + + /** + * Reads the sequence number stats from the commit data (maximum sequence number and local checkpoint) and uses the specified global + * checkpoint. + * + * @param globalCheckpoint the global checkpoint to use + * @param commitData the commit data + * @return the sequence number stats + */ + public static SeqNoStats loadSeqNoStatsFromLuceneCommit( + final long globalCheckpoint, + final Iterable> commitData) { + long maxSeqNo = SequenceNumbersService.NO_OPS_PERFORMED; + long localCheckpoint = SequenceNumbersService.NO_OPS_PERFORMED; + + for (final Map.Entry entry : commitData) { + final String key = entry.getKey(); + if (key.equals(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) { + assert localCheckpoint == SequenceNumbersService.NO_OPS_PERFORMED : localCheckpoint; + localCheckpoint = Long.parseLong(entry.getValue()); + } else if (key.equals(SequenceNumbers.MAX_SEQ_NO)) { + assert maxSeqNo == SequenceNumbersService.NO_OPS_PERFORMED : maxSeqNo; + maxSeqNo = Long.parseLong(entry.getValue()); + } + } + + return new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint); + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index 9fed5f9cf7ec6..cc43363e13b2d 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -40,8 +40,8 @@ public class SequenceNumbersService extends AbstractIndexShardComponent { */ public static final long NO_OPS_PERFORMED = -1L; - private final LocalCheckpointService localCheckpointService; - private final GlobalCheckpointService globalCheckpointService; + private final LocalCheckpointTracker localCheckpointTracker; + private final GlobalCheckpointTracker globalCheckpointTracker; /** * Initialize the sequence number service. The {@code maxSeqNo} should be set to the last sequence number assigned by this shard, or @@ -62,8 +62,8 @@ public SequenceNumbersService( final long localCheckpoint, final long globalCheckpoint) { super(shardId, indexSettings); - localCheckpointService = new LocalCheckpointService(shardId, indexSettings, maxSeqNo, localCheckpoint); - globalCheckpointService = new GlobalCheckpointService(shardId, indexSettings, globalCheckpoint); + localCheckpointTracker = new LocalCheckpointTracker(indexSettings, maxSeqNo, localCheckpoint); + globalCheckpointTracker = new GlobalCheckpointTracker(indexSettings, globalCheckpoint, logger); } /** @@ -73,26 +73,36 @@ public SequenceNumbersService( * @return the next assigned sequence number */ public long generateSeqNo() { - return localCheckpointService.generateSeqNo(); + return localCheckpointTracker.generateSeqNo(); } /** - * The maximum sequence number issued so far. See {@link LocalCheckpointService#getMaxSeqNo()} for additional details. + * The maximum sequence number issued so far. See {@link LocalCheckpointTracker#getMaxSeqNo()} for additional details. * * @return the maximum sequence number */ public long getMaxSeqNo() { - return localCheckpointService.getMaxSeqNo(); + return localCheckpointTracker.getMaxSeqNo(); + } + + /** + * Waits for all operations up to the provided sequence number to complete. + * + * @param seqNo the sequence number that the checkpoint must advance to before this method returns + * @throws InterruptedException if the thread was interrupted while blocking on the condition + */ + public void waitForOpsToComplete(final long seqNo) throws InterruptedException { + localCheckpointTracker.waitForOpsToComplete(seqNo); } /** * Marks the processing of the provided sequence number as completed as updates the checkpoint if possible. - * See {@link LocalCheckpointService#markSeqNoAsCompleted(long)} for additional details. + * See {@link LocalCheckpointTracker#markSeqNoAsCompleted(long)} for additional details. * * @param seqNo the sequence number to mark as completed */ public void markSeqNoAsCompleted(final long seqNo) { - localCheckpointService.markSeqNoAsCompleted(seqNo); + localCheckpointTracker.markSeqNoAsCompleted(seqNo); } /** @@ -106,23 +116,23 @@ public SeqNoStats stats() { /** * Notifies the service to update the local checkpoint for the shard with the provided allocation ID. See - * {@link GlobalCheckpointService#updateLocalCheckpoint(String, long)} for details. + * {@link GlobalCheckpointTracker#updateLocalCheckpoint(String, long)} for details. * * @param allocationId the allocation ID of the shard to update the local checkpoint for * @param checkpoint the local checkpoint for the shard */ public void updateLocalCheckpointForShard(final String allocationId, final long checkpoint) { - globalCheckpointService.updateLocalCheckpoint(allocationId, checkpoint); + globalCheckpointTracker.updateLocalCheckpoint(allocationId, checkpoint); } /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. See - * {@link GlobalCheckpointService#markAllocationIdAsInSync(String)} for additional details. + * {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String)} for additional details. * * @param allocationId the allocation ID of the shard to mark as in-sync */ public void markAllocationIdAsInSync(final String allocationId) { - globalCheckpointService.markAllocationIdAsInSync(allocationId); + globalCheckpointTracker.markAllocationIdAsInSync(allocationId); } /** @@ -131,7 +141,7 @@ public void markAllocationIdAsInSync(final String allocationId) { * @return the local checkpoint */ public long getLocalCheckpoint() { - return localCheckpointService.getCheckpoint(); + return localCheckpointTracker.getCheckpoint(); } /** @@ -140,7 +150,7 @@ public long getLocalCheckpoint() { * @return the global checkpoint */ public long getGlobalCheckpoint() { - return globalCheckpointService.getCheckpoint(); + return globalCheckpointTracker.getCheckpoint(); } /** @@ -150,7 +160,7 @@ public long getGlobalCheckpoint() { * active allocations is not known. */ public boolean updateGlobalCheckpointOnPrimary() { - return globalCheckpointService.updateCheckpointOnPrimary(); + return globalCheckpointTracker.updateCheckpointOnPrimary(); } /** @@ -159,18 +169,18 @@ public boolean updateGlobalCheckpointOnPrimary() { * @param checkpoint the global checkpoint */ public void updateGlobalCheckpointOnReplica(final long checkpoint) { - globalCheckpointService.updateCheckpointOnReplica(checkpoint); + globalCheckpointTracker.updateCheckpointOnReplica(checkpoint); } /** * Notifies the service of the current allocation IDs in the cluster state. See - * {@link GlobalCheckpointService#updateAllocationIdsFromMaster(Set, Set)} for details. + * {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. * * @param activeAllocationIds the allocation IDs of the currently active shard copies * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies */ public void updateAllocationIdsFromMaster(final Set activeAllocationIds, final Set initializingAllocationIds) { - globalCheckpointService.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds); + globalCheckpointTracker.updateAllocationIdsFromMaster(activeAllocationIds, initializingAllocationIds); } } diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 070ce17ba2064..362ae57582534 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -101,7 +101,7 @@ import org.elasticsearch.index.refresh.RefreshStats; import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; -import org.elasticsearch.index.seqno.GlobalCheckpointService; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; @@ -1369,7 +1369,7 @@ public void writeIndexingBuffer() { /** * Notifies the service to update the local checkpoint for the shard with the provided allocation ID. See - * {@link GlobalCheckpointService#updateLocalCheckpoint(String, long)} for details. + * {@link GlobalCheckpointTracker#updateLocalCheckpoint(String, long)} for details. * * @param allocationId the allocation ID of the shard to update the local checkpoint for * @param checkpoint the local checkpoint for the shard @@ -1379,9 +1379,19 @@ public void updateLocalCheckpointForShard(final String allocationId, final long getEngine().seqNoService().updateLocalCheckpointForShard(allocationId, checkpoint); } + /** + * Waits for all operations up to the provided sequence number to complete. + * + * @param seqNo the sequence number that the checkpoint must advance to before this method returns + * @throws InterruptedException if the thread was interrupted while blocking on the condition + */ + public void waitForOpsToComplete(final long seqNo) throws InterruptedException { + getEngine().seqNoService().waitForOpsToComplete(seqNo); + } + /** * Marks the shard with the provided allocation ID as in-sync with the primary shard. See - * {@link GlobalCheckpointService#markAllocationIdAsInSync(String)} for additional details. + * {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String)} for additional details. * * @param allocationId the allocation ID of the shard to mark as in-sync */ @@ -1431,7 +1441,7 @@ public void updateGlobalCheckpointOnReplica(final long checkpoint) { /** * Notifies the service of the current allocation IDs in the cluster state. See - * {@link GlobalCheckpointService#updateAllocationIdsFromMaster(Set, Set)} for details. + * {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. * * @param activeAllocationIds the allocation IDs of the currently active shard copies * @param initializingAllocationIds the allocation IDs of the currently initializing shard copies diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java index fcc079d7909d7..7d569d947c3b2 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -24,6 +24,7 @@ import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.index.IndexFormatTooNewException; @@ -75,6 +76,8 @@ import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; @@ -207,6 +210,11 @@ private static SegmentInfos readSegmentsInfo(IndexCommit commit, Directory direc } + public SeqNoStats loadSeqNoStats(final long globalCheckpoint) throws IOException { + final IndexCommit indexCommit = DirectoryReader.listCommits(directory).get(0); + return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, indexCommit.getUserData().entrySet()); + } + final void ensureOpen() { if (this.refCounter.refCount() <= 0) { throw new AlreadyClosedException("store is already closed"); diff --git a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java index 7b1a05e1ac1e2..180b69233a58b 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java +++ b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java @@ -57,4 +57,5 @@ public Translog.Operation next() throws IOException { } return null; } + } diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index bdbce03bda1d9..e1d22378b8159 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -778,6 +778,8 @@ public static Type fromId(byte id) { Source getSource(); + long seqNo(); + /** * Reads the type and the operation from the given stream. The operatino must be written with * {@link Operation#writeType(Operation, StreamOutput)} @@ -922,6 +924,7 @@ public BytesReference source() { return this.source; } + @Override public long seqNo() { return seqNo; } @@ -1072,6 +1075,7 @@ public Term uid() { return this.uid; } + @Override public long seqNo() { return seqNo; } @@ -1147,6 +1151,7 @@ public static class NoOp implements Operation { private final long primaryTerm; private final String reason; + @Override public long seqNo() { return seqNo; } diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index f0d72f6c4c5d2..2307a71171411 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -53,7 +53,7 @@ import org.elasticsearch.index.IndexComponent; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.seqno.GlobalCheckpointService; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -739,7 +739,7 @@ public interface Shard { /** * Notifies the service of the current allocation ids in the cluster state. - * See {@link GlobalCheckpointService#updateAllocationIdsFromMaster(Set, Set)} for details. + * See {@link GlobalCheckpointTracker#updateAllocationIdsFromMaster(Set, Set)} for details. * * @param activeAllocationIds the allocation ids of the currently active shard copies * @param initializingAllocationIds the allocation ids of the currently initializing shard copies diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 894399e851e05..3145cd81ad736 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -41,6 +41,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexShard; @@ -48,6 +49,7 @@ import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.recovery.RecoveriesCollection.RecoveryRef; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.threadpool.ThreadPool; @@ -59,8 +61,11 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.List; +import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; @@ -124,11 +129,11 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh } /** - * cancel all ongoing recoveries for the given shard, if their status match a predicate + * Cancel all ongoing recoveries for the given shard. * - * @param reason reason for cancellation - * @param shardId shardId for which to cancel recoveries - * @return true if a recovery was cancelled + * @param reason reason for cancellation + * @param shardId shard ID for which to cancel recoveries + * @return {@code true} if a recovery was cancelled */ public boolean cancelRecoveriesForShard(ShardId shardId, String reason) { return onGoingRecoveries.cancelRecoveriesForShard(shardId, reason); @@ -152,8 +157,26 @@ protected void retryRecovery(final long recoveryId, final String reason, TimeVal retryRecovery(recoveryId, retryAfter, activityTimeout); } + protected void retryRecovery( + final long recoveryId, + final String reason, + final TimeValue retryAfter, + final TimeValue activityTimeout, + final Function supplier) { + logger.trace("will retry recovery with id [{}] in [{}] (reason [{}])", recoveryId, retryAfter, reason); + retryRecovery(recoveryId, retryAfter, activityTimeout, supplier); + } + private void retryRecovery(final long recoveryId, TimeValue retryAfter, TimeValue activityTimeout) { - RecoveryTarget newTarget = onGoingRecoveries.resetRecovery(recoveryId, activityTimeout); + retryRecovery(recoveryId, retryAfter, activityTimeout, RecoveryTarget::retry); + } + + private void retryRecovery( + final long recoveryId, + final TimeValue retryAfter, + final TimeValue activityTimeout, + final Function supplier) { + RecoveryTarget newTarget = onGoingRecoveries.resetRecovery(recoveryId, activityTimeout, supplier); if (newTarget != null) { threadPool.schedule(retryAfter, ThreadPool.Names.GENERIC, new RecoveryRunner(newTarget.recoveryId())); } @@ -169,50 +192,15 @@ private void doRecovery(final long recoveryId) { logger.trace("not running recovery with id [{}] - can't find it (probably finished)", recoveryId); return; } - RecoveryTarget recoveryTarget = recoveryRef.target(); + final RecoveryTarget recoveryTarget = recoveryRef.target(); assert recoveryTarget.sourceNode() != null : "can't do a recovery without a source node"; - logger.trace("collecting local files for {}", recoveryTarget.sourceNode()); - Store.MetadataSnapshot metadataSnapshot; - try { - if (recoveryTarget.indexShard().indexSettings().isOnSharedFilesystem()) { - // we are not going to copy any files, so don't bother listing files, potentially running - // into concurrency issues with the primary changing files underneath us. - metadataSnapshot = Store.MetadataSnapshot.EMPTY; - } else { - metadataSnapshot = recoveryTarget.indexShard().snapshotStoreMetadata(); - } - logger.trace("{} local file count: [{}]", recoveryTarget, metadataSnapshot.size()); - } catch (org.apache.lucene.index.IndexNotFoundException e) { - // happens on an empty folder. no need to log - logger.trace("{} shard folder empty, recover all files", recoveryTarget); - metadataSnapshot = Store.MetadataSnapshot.EMPTY; - } catch (IOException e) { - logger.warn("error while listing local files, recover as if there are none", e); - metadataSnapshot = Store.MetadataSnapshot.EMPTY; - } catch (Exception e) { - // this will be logged as warning later on... - logger.trace("unexpected error while listing local files, failing recovery", e); - onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), - new RecoveryFailedException(recoveryTarget.state(), "failed to list local files", e), true); - return; - } - - try { - logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); - recoveryTarget.indexShard().prepareForIndexRecovery(); + final Optional maybeRequest = getStartRecoveryRequest(recoveryTarget); + if (!maybeRequest.isPresent()) return; + else request = maybeRequest.get(); - request = new StartRecoveryRequest(recoveryTarget.shardId(), recoveryTarget.sourceNode(), - clusterService.localNode(), metadataSnapshot, recoveryTarget.state().getPrimary(), recoveryTarget.recoveryId()); - cancellableThreads = recoveryTarget.CancellableThreads(); - timer = recoveryTarget.state().getTimer(); - } catch (Exception e) { - // this will be logged as warning later on... - logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e); - onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), - new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e), true); - return; - } + cancellableThreads = recoveryTarget.cancellableThreads(); + timer = recoveryTarget.state().getTimer(); } try { @@ -227,7 +215,6 @@ public RecoveryResponse newInstance() { } }).txGet())); final RecoveryResponse recoveryResponse = responseHolder.get(); - assert responseHolder != null; final TimeValue recoveryTime = new TimeValue(timer.time()); // do this through ongoing recoveries to remove it from the collection onGoingRecoveries.markRecoveryAsDone(recoveryId); @@ -286,22 +273,29 @@ public RecoveryResponse newInstance() { if (cause instanceof IllegalIndexShardStateException || cause instanceof IndexNotFoundException || cause instanceof ShardNotFoundException) { // if the target is not ready yet, retry - retryRecovery(recoveryId, "remote shard not ready", recoverySettings.retryDelayStateSync(), + retryRecovery( + recoveryId, + "remote shard not ready", + recoverySettings.retryDelayStateSync(), recoverySettings.activityTimeout()); return; } if (cause instanceof DelayRecoveryException) { - retryRecovery(recoveryId, cause, recoverySettings.retryDelayStateSync(), - recoverySettings.activityTimeout()); + retryRecovery(recoveryId, cause, recoverySettings.retryDelayStateSync(), recoverySettings.activityTimeout()); return; } if (cause instanceof ConnectTransportException) { logger.debug("delaying recovery of {} for [{}] due to networking error [{}]", request.shardId(), recoverySettings.retryDelayNetwork(), cause.getMessage()); - retryRecovery(recoveryId, cause.getMessage(), recoverySettings.retryDelayNetwork(), - recoverySettings.activityTimeout()); + retryRecovery(recoveryId, cause.getMessage(), recoverySettings.retryDelayNetwork(), recoverySettings.activityTimeout()); + return; + } + + if (sequenceNumberBasedRecoveryFailed(cause)) { + final String header = getSequenceNumberBasedRecoveryFailedHeader((ElasticsearchException) cause); + retryRecovery(recoveryId, header, TimeValue.ZERO, recoverySettings.activityTimeout(), RecoveryTarget::fileRecoveryRetry); return; } @@ -310,10 +304,129 @@ public RecoveryResponse newInstance() { new RecoveryFailedException(request, "source shard is closed", cause), false); return; } + onGoingRecoveries.failRecovery(recoveryId, new RecoveryFailedException(request, e), true); } } + /** + * Obtain the {@link RecoverySourceHandler#SEQUENCE_NUMBER_BASED_RECOVERY_FAILED} header if it is present, otherwise {@code null}. + * + * @param ex the exception to inspect + * @return the header if it exists, otherwise {@code null} + */ + private static String getSequenceNumberBasedRecoveryFailedHeader(final ElasticsearchException ex) { + final List header = ex.getHeader(RecoverySourceHandler.SEQUENCE_NUMBER_BASED_RECOVERY_FAILED); + if (header != null && !header.isEmpty()) { + return header.get(0); + } else { + return null; + } + } + + /** + * Determine if the specified throwable represents an exception from a failed sequence number-based recovery. + * + * @param cause the throwable to inspect + * @return {@code true} iff the specified throwable represents an exception from a failed sequence number-based recovery + */ + public static boolean sequenceNumberBasedRecoveryFailed(final Throwable cause) { + if (cause instanceof ElasticsearchException) { + return getSequenceNumberBasedRecoveryFailedHeader((ElasticsearchException) cause) != null; + } + return false; + } + + @SuppressWarnings("OptionalUsedAsFieldOrParameterType") + private static Optional EMPTY_METADATA_SNAPSHOT = Optional.of(Store.MetadataSnapshot.EMPTY); + + /** + * Obtains a snapshot of the store metadata for the recovery target, or an empty {@link Optional} if obtaining the store metadata + * failed. + * + * @param recoveryTarget the target of the recovery + * @return a snapshot of the store metdata, or an empty {@link Optional} + */ + private Optional getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) { + try { + if (recoveryTarget.indexShard().indexSettings().isOnSharedFilesystem()) { + // we are not going to copy any files, so don't bother listing files, potentially running into concurrency issues with the + // primary changing files underneath us + return EMPTY_METADATA_SNAPSHOT; + } else { + return Optional.of(recoveryTarget.indexShard().snapshotStoreMetadata()); + } + } catch (org.apache.lucene.index.IndexNotFoundException e) { + // happens on an empty folder. no need to log + logger.trace("{} shard folder empty, recover all files", recoveryTarget); + return EMPTY_METADATA_SNAPSHOT; + } catch (final IOException e) { + logger.warn("error while listing local files, recover as if there are none", e); + return EMPTY_METADATA_SNAPSHOT; + } catch (final Exception e) { + // this will be logged as warning later on... + logger.trace("unexpected error while listing local files, failing recovery", e); + onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), + new RecoveryFailedException(recoveryTarget.state(), "failed to list local files", e), true); + return Optional.empty(); + } + } + + /** + * Prepare the start recovery request, returning an empty {@link Optional} instance if preparing the start request failed. + * + * @param recoveryTarget the target of the recovery + * @return a start recovery request, or an empty {@link Optional} + */ + private Optional getStartRecoveryRequest(final RecoveryTarget recoveryTarget) { + final StartRecoveryRequest request; + logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + + final Optional metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget); + if (!metadataSnapshot.isPresent()) return Optional.empty(); + logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.get().size()); + + try { + final long startingSeqNo; + if (metadataSnapshot.get().size() > 0 && recoveryTarget.canPerformSeqNoBasedRecovery()) { + startingSeqNo = getStartingSeqNo(recoveryTarget); + logger.trace( + "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", + recoveryTarget.shardId(), + startingSeqNo, + recoveryTarget.sourceNode()); + } else { + logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + } + + logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); + recoveryTarget.indexShard().prepareForIndexRecovery(); + + request = new StartRecoveryRequest( + recoveryTarget.shardId(), + recoveryTarget.sourceNode(), + clusterService.localNode(), + metadataSnapshot.get(), + recoveryTarget.state().getPrimary(), + recoveryTarget.recoveryId(), + startingSeqNo); + + } catch (final Exception e) { + // this will be logged as warning later on... + logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e); + onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), + new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e), true); + return Optional.empty(); + } + return Optional.of(request); + } + + public static long getStartingSeqNo(RecoveryTarget recoveryTarget) throws IOException { + final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.indexShard().shardPath().resolveTranslog()); + return recoveryTarget.store().loadSeqNoStats(globalCheckpoint).getLocalCheckpoint() + 1; + } + public interface RecoveryListener { void onRecoveryDone(RecoveryState state); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index aed2325610875..0a05bb58b7841 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; /** * This class holds a collection of all on going recoveries on the current node (i.e., the node is the target node @@ -82,14 +83,16 @@ private void startRecoveryInternal(RecoveryTarget recoveryTarget, TimeValue acti new RecoveryMonitor(recoveryTarget.recoveryId(), recoveryTarget.lastAccessTime(), activityTimeout)); } - /** * Resets the recovery and performs a recovery restart on the currently recovering index shard * * @see IndexShard#performRecoveryRestart() * @return newly created RecoveryTarget */ - public RecoveryTarget resetRecovery(final long recoveryId, TimeValue activityTimeout) { + public RecoveryTarget resetRecovery( + final long recoveryId, + final TimeValue activityTimeout, + final Function supplier) { RecoveryTarget oldRecoveryTarget = null; final RecoveryTarget newRecoveryTarget; @@ -102,12 +105,12 @@ public RecoveryTarget resetRecovery(final long recoveryId, TimeValue activityTim return null; } - newRecoveryTarget = oldRecoveryTarget.retryCopy(); + newRecoveryTarget = supplier.apply(oldRecoveryTarget); startRecoveryInternal(newRecoveryTarget, activityTimeout); } // Closes the current recovery target - boolean successfulReset = oldRecoveryTarget.resetRecovery(newRecoveryTarget.CancellableThreads()); + boolean successfulReset = oldRecoveryTarget.resetRecovery(newRecoveryTarget.cancellableThreads()); if (successfulReset) { logger.trace("{} restarted recovery from {}, id [{}], previous id [{}]", newRecoveryTarget.shardId(), newRecoveryTarget.sourceNode(), newRecoveryTarget.recoveryId(), oldRecoveryTarget.recoveryId()); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index be055531813bb..28158071d90ed 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -41,6 +41,8 @@ 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; @@ -55,6 +57,7 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.List; +import java.util.Locale; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.StreamSupport; @@ -72,6 +75,8 @@ */ public class RecoverySourceHandler { + public static final String SEQUENCE_NUMBER_BASED_RECOVERY_FAILED = "sequence_number_based_recovery_failed"; + protected final Logger logger; // Shard that is going to be recovered (the "source") private final IndexShard shard; @@ -124,49 +129,89 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov * 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()); + try (final Translog.View translogView = shard.acquireTranslogView()) { + logger.trace("{} captured translog id [{}] for recovery", shard.shardId(), translogView.minTranslogGeneration()); final IndexCommit phase1Snapshot; try { phase1Snapshot = shard.acquireIndexCommit(false); - } catch (Exception e) { + } catch (final Exception e) { IOUtils.closeWhileHandlingException(translogView); - throw new RecoveryEngineException(shard.shardId(), 1, "Snapshot failed", e); + throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); } - try { - phase1(phase1Snapshot, translogView); - } catch (Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); - } finally { + final long startingSeqNo; + final long endingSeqNo; + if (request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO) { + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + endingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + try { + try { + phase1(phase1Snapshot, translogView); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); + } + try { + prepareTargetForTranslog(translogView.totalOperations()); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); + } + } finally { + try { + shard.releaseIndexCommit(phase1Snapshot); + } catch (final IOException ex) { + logger.warn("releasing snapshot caused exception", ex); + } + } + } else { + startingSeqNo = request.startingSeqNo(); + endingSeqNo = shard.seqNoStats().getMaxSeqNo(); + if (endingSeqNo < startingSeqNo) { + final String message = String.format( + Locale.ROOT, + "requested starting operation [%d] higher than source operation maximum [%d]", + startingSeqNo, + endingSeqNo); + logger.debug("{} {}", shard.shardId(), message); + final ElasticsearchException ex = new ElasticsearchException(message); + ex.addHeader(SEQUENCE_NUMBER_BASED_RECOVERY_FAILED, message); + throw ex; + } try { - shard.releaseIndexCommit(phase1Snapshot); - } catch (IOException ex) { - logger.warn("releasing snapshot caused exception", ex); + prepareTargetForTranslog(translogView.totalOperations()); + } catch (final Exception e) { + throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); } + // 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 + logger.trace( + "{} waiting for all operations in the range [{}, {}] to complete", + shard.shardId(), + startingSeqNo, + endingSeqNo); + cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); } - // engine was just started at the end of phase 1 + // 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()); + phase2(translogView.snapshot(), startingSeqNo, endingSeqNo); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); } @@ -186,6 +231,7 @@ public RecoveryResponse recoverToTarget() throws IOException { * checksum can be reused */ public void phase1(final IndexCommit snapshot, final Translog.View translogView) { + assert request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO; cancellableThreads.checkForCancel(); // Total size of segment files that are recovered long totalSize = 0; @@ -329,8 +375,6 @@ public void phase1(final IndexCommit snapshot, final Translog.View translogView) } } - prepareTargetForTranslog(translogView.totalOperations()); - logger.trace("[{}][{}] recovery [phase1] to {}: took [{}]", indexName, shardId, request.targetNode(), stopWatch.totalTime()); response.phase1Time = stopWatch.totalTime().millis(); } catch (Exception e) { @@ -340,16 +384,14 @@ public void phase1(final IndexCommit snapshot, final Translog.View translogView) } } - - protected void prepareTargetForTranslog(final int totalTranslogOps) throws IOException { + void prepareTargetForTranslog(final int totalTranslogOps) 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 - cancellableThreads.executeIO(() -> recoveryTarget.prepareForTranslogOperations(totalTranslogOps, - shard.segmentStats(false).getMaxUnsafeAutoIdTimestamp())); + // 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, shard.segmentStats(false).getMaxUnsafeAutoIdTimestamp())); stopWatch.stop(); response.startTime = stopWatch.totalTime().millis() - startEngineStart; @@ -358,31 +400,54 @@ protected void prepareTargetForTranslog(final int totalTranslogOps) throws IOExc } /** - * Perform phase2 of the recovery process + * Perform phase two of the recovery process. *

- * 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 takes 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. */ - public void phase2(Translog.Snapshot snapshot) { + void phase2(final Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); - StopWatch stopWatch = new StopWatch().start(); + final LocalCheckpointTracker tracker; + if (startingSeqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + tracker = new LocalCheckpointTracker(shard.indexSettings(), startingSeqNo, startingSeqNo - 1); + } else { + tracker = new LocalCheckpointTracker(shard.indexSettings(), Long.MAX_VALUE, Long.MAX_VALUE); + } + + 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(snapshot, tracker); + + // check to see if all operations in the required range were sent to the target + if (tracker.getCheckpoint() < endingSeqNo) { + final String message = String.format( + Locale.ROOT, + "sequence number-based recovery failed due to missing ops in range [%d, %d]; first missed op [%d]", + startingSeqNo, + endingSeqNo, + tracker.getCheckpoint() + 1); + logger.debug("{} {}", shard.shardId(), message); + final ElasticsearchException ex = new ElasticsearchException(message); + ex.setShard(shard.shardId()); + ex.addHeader(SEQUENCE_NUMBER_BASED_RECOVERY_FAILED, message); + throw ex; + } + 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() { @@ -411,7 +476,7 @@ public void finalizeRecovery() { 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}). */ @@ -424,78 +489,79 @@ public void finalizeRecovery() { /** * Send the given snapshot's operations to this handler's target node. *

- * 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 snapshot the translog snapshot to replay operations from + * @param tracker tracks the replayed operations * @return the total number of translog operations that were sent */ - protected int sendSnapshot(final Translog.Snapshot snapshot) { + protected int sendSnapshot(final Translog.Snapshot snapshot, final LocalCheckpointTracker tracker) { int ops = 0; long size = 0; int totalOperations = 0; final List 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 = getNextOperationFromSnapshot(snapshot)) != null) { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); operations.add(operation); - ops += 1; + ops++; size += operation.estimateSize(); totalOperations++; + tracker.markSeqNoAsCompleted(operation.seqNo()); - // 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; } + private Translog.Operation getNextOperationFromSnapshot(final Translog.Snapshot snapshot) { + try { + return snapshot.next(); + } catch (final IOException ex) { + throw new ElasticsearchException("failed to get next operation from translog", ex); + } + } + /** * Cancels the recovery and interrupts all eligible threads. */ diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 00dd019aac47a..70deaea48e3d1 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -49,6 +49,7 @@ import org.elasticsearch.index.translog.Translog; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -91,23 +92,49 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget // last time this status was accessed private volatile long lastAccessTime = System.nanoTime(); + private volatile boolean canPerformSeqNoBasedRecovery = true; + + public boolean canPerformSeqNoBasedRecovery() { + return canPerformSeqNoBasedRecovery; + } + + public void setCanPerformSeqNoBasedRecovery(final boolean canPerformSeqNoBasedRecovery) { + this.canPerformSeqNoBasedRecovery = canPerformSeqNoBasedRecovery; + } + // latch that can be used to blockingly wait for RecoveryTarget to be closed private final CountDownLatch closedLatch = new CountDownLatch(1); private final Map tempFileNames = ConcurrentCollections.newConcurrentMap(); /** - * creates a new recovery target object that represents a recovery to the provided indexShard + * Creates a new recovery target object that represents a recovery to the provided shard. * - * @param indexShard local shard where we want to recover to - * @param sourceNode source node of the recovery where we recover from - * @param listener called when recovery is completed / failed + * @param indexShard local shard where we want to recover to + * @param sourceNode source node of the recovery where we recover from + * @param listener called when recovery is completed/failed * @param ensureClusterStateVersionCallback callback to ensure that the current node is at least on a cluster state with the provided - * version. Necessary for primary relocation so that new primary knows about all other ongoing - * replica recoveries when replicating documents (see {@link RecoverySourceHandler}). + * version; necessary for primary relocation so that new primary knows about all other ongoing + * replica recoveries when replicating documents (see {@link RecoverySourceHandler}) */ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener, Callback ensureClusterStateVersionCallback) { + this(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, true); + } + + /** + * Creates a new recovery target object that represents a recovery to the provided shard. + * + * @param indexShard local shard where we want to recover to + * @param sourceNode source node of the recovery where we recover from + * @param listener called when recovery is completed/failed + * @param ensureClusterStateVersionCallback callback to ensure that the current node is at least on a cluster state with the provided + * version; necessary for primary relocation so that new primary knows about all other ongoing + * replica recoveries when replicating documents (see {@link RecoverySourceHandler}) + * @param canPerformSeqNoBasedRecovery whether or not sequence number-based recovery can be performed + */ + private RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener, + Callback ensureClusterStateVersionCallback, boolean canPerformSeqNoBasedRecovery) { super("recovery_status"); this.cancellableThreads = new CancellableThreads(); this.recoveryId = idGenerator.incrementAndGet(); @@ -122,13 +149,26 @@ public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecov // make sure the store is not released until we are done. store.incRef(); indexShard.recoveryStats().incCurrentAsTarget(); + this.canPerformSeqNoBasedRecovery = canPerformSeqNoBasedRecovery; + } + + /** + * Returns a fresh recovery target to retry recovery from the same source node onto the same shard and using the same listener. + * + * @return a copy of this recovery target + */ + public RecoveryTarget retry() { + return new RecoveryTarget(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, canPerformSeqNoBasedRecovery); } /** - * returns a fresh RecoveryTarget to retry recovery from the same source node onto the same IndexShard and using the same listener + * Returns a fresh recovery target to retry recovery from the same source node onto the same shard and using the same listener, but + * disabling sequence number-based recovery. + * + * @return a copy of this recovery target */ - public RecoveryTarget retryCopy() { - return new RecoveryTarget(this.indexShard, this.sourceNode, this.listener, this.ensureClusterStateVersionCallback); + public RecoveryTarget fileRecoveryRetry() { + return new RecoveryTarget(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, false); } public long recoveryId() { @@ -152,7 +192,7 @@ public RecoveryState state() { return indexShard.recoveryState(); } - public CancellableThreads CancellableThreads() { + public CancellableThreads cancellableThreads() { return cancellableThreads; } @@ -185,7 +225,7 @@ public void renameAllTempFiles() throws IOException { * Closes the current recovery target and waits up to a certain timeout for resources to be freed. * Returns true if resetting the recovery was successful, false if the recovery target is already cancelled / failed or marked as done. */ - boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOException { + public boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOException { if (finished.compareAndSet(false, true)) { try { logger.debug("reset of recovery with shard {} and id [{}]", shardId, recoveryId); @@ -220,7 +260,7 @@ boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOE * unless this object is in use (in which case it will be cleaned once all ongoing users call * {@link #decRef()} *

- * if {@link #CancellableThreads()} was used, the threads will be interrupted. + * if {@link #cancellableThreads()} was used, the threads will be interrupted. */ public void cancel(String reason) { if (finished.compareAndSet(false, true)) { diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index 591176f047a2f..b27687101fb37 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.Translog; @@ -29,17 +30,16 @@ import java.util.function.Supplier; /** - * A recovery handler that skips phase 1 as well as sending the snapshot. During phase 3 the shard is marked - * as relocated an closed to ensure that the engine is closed and the target can acquire the IW write lock. + * A recovery handler that skips phase one as well as sending the translog snapshot. */ public class SharedFSRecoverySourceHandler extends RecoverySourceHandler { private final IndexShard shard; private final StartRecoveryRequest request; - public SharedFSRecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, StartRecoveryRequest request, - Supplier currentClusterStateVersionSupplier, - Function delayNewRecoveries, Logger logger) { + SharedFSRecoverySourceHandler(IndexShard shard, RecoveryTargetHandler recoveryTarget, StartRecoveryRequest request, + Supplier currentClusterStateVersionSupplier, + Function delayNewRecoveries, Logger logger) { super(shard, recoveryTarget, request, currentClusterStateVersionSupplier, delayNewRecoveries, -1, logger); this.shard = shard; this.request = request; @@ -49,7 +49,7 @@ public SharedFSRecoverySourceHandler(IndexShard shard, RecoveryTargetHandler rec public RecoveryResponse recoverToTarget() throws IOException { boolean engineClosed = false; try { - logger.trace("{} recovery [phase1] to {}: skipping phase 1 for shared filesystem", request.shardId(), request.targetNode()); + logger.trace("{} recovery [phase1] to {}: skipping phase1 for shared filesystem", request.shardId(), request.targetNode()); if (request.isPrimaryRelocation()) { logger.debug("[phase1] closing engine on primary for shared filesystem recovery"); try { @@ -82,9 +82,9 @@ public RecoveryResponse recoverToTarget() throws IOException { } @Override - protected int sendSnapshot(Translog.Snapshot snapshot) { - logger.trace("{} skipping recovery of translog snapshot on shared filesystem to: {}", - shard.shardId(), request.targetNode()); + protected int sendSnapshot(final Translog.Snapshot snapshot, final LocalCheckpointTracker tracker) { + logger.trace("{} skipping recovery of translog snapshot on shared filesystem to: {}", shard.shardId(), request.targetNode()); return 0; } + } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index bc8a73b562280..f5306d3e94b65 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -19,46 +19,60 @@ package org.elasticsearch.indices.recovery; +import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.transport.TransportRequest; import java.io.IOException; +/** + * Represents a request for starting a peer recovery. + */ public class StartRecoveryRequest extends TransportRequest { private long recoveryId; - private ShardId shardId; - private DiscoveryNode sourceNode; - private DiscoveryNode targetNode; - private Store.MetadataSnapshot metadataSnapshot; - private boolean primaryRelocation; + private long startingSeqNo; public StartRecoveryRequest() { } /** - * Start recovery request. + * Construct a request for starting a peer recovery. * - * @param sourceNode The node to recover from - * @param targetNode The node to recover to + * @param shardId the shard ID to recover + * @param sourceNode the source node to remover from + * @param targetNode the target node to recover to + * @param metadataSnapshot the Lucene metadata + * @param primaryRelocation whether or not the recovery is a primary relocation + * @param recoveryId the recovery ID + * @param startingSeqNo the starting sequence number */ - public StartRecoveryRequest(ShardId shardId, DiscoveryNode sourceNode, DiscoveryNode targetNode, Store.MetadataSnapshot metadataSnapshot, boolean primaryRelocation, long recoveryId) { + public StartRecoveryRequest(final ShardId shardId, + final DiscoveryNode sourceNode, + final DiscoveryNode targetNode, + final Store.MetadataSnapshot metadataSnapshot, + final boolean primaryRelocation, + final long recoveryId, + final long startingSeqNo) { this.recoveryId = recoveryId; this.shardId = shardId; this.sourceNode = sourceNode; this.targetNode = targetNode; this.metadataSnapshot = metadataSnapshot; this.primaryRelocation = primaryRelocation; + this.startingSeqNo = startingSeqNo; } public long recoveryId() { @@ -85,6 +99,10 @@ public Store.MetadataSnapshot metadataSnapshot() { return metadataSnapshot; } + public long startingSeqNo() { + return startingSeqNo; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -94,6 +112,11 @@ public void readFrom(StreamInput in) throws IOException { targetNode = new DiscoveryNode(in); metadataSnapshot = new Store.MetadataSnapshot(in); primaryRelocation = in.readBoolean(); + if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { + startingSeqNo = in.readZLong(); + } else { + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + } } @Override @@ -105,6 +128,9 @@ public void writeTo(StreamOutput out) throws IOException { targetNode.writeTo(out); metadataSnapshot.writeTo(out); out.writeBoolean(primaryRelocation); + if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { + out.writeZLong(startingSeqNo); + } } } diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f0ca8292f4f23..9b19d9a2ca967 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -107,8 +107,8 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; -import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; @@ -640,14 +640,14 @@ public SequenceNumbersService seqNoService() { assertThat(stats1.getGeneration(), greaterThan(0L)); assertThat(stats1.getId(), notNullValue()); assertThat(stats1.getUserData(), hasKey(Translog.TRANSLOG_GENERATION_KEY)); - assertThat(stats1.getUserData(), hasKey(InternalEngine.LOCAL_CHECKPOINT_KEY)); + assertThat(stats1.getUserData(), hasKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); assertThat( - Long.parseLong(stats1.getUserData().get(InternalEngine.LOCAL_CHECKPOINT_KEY)), + Long.parseLong(stats1.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); - assertThat(stats1.getUserData(), hasKey(InternalEngine.MAX_SEQ_NO)); + assertThat(stats1.getUserData(), hasKey(SequenceNumbers.MAX_SEQ_NO)); assertThat( - Long.parseLong(stats1.getUserData().get(InternalEngine.MAX_SEQ_NO)), + Long.parseLong(stats1.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); maxSeqNo.set(rarely() ? SequenceNumbersService.NO_OPS_PERFORMED : randomIntBetween(0, 1024)); @@ -669,9 +669,9 @@ public SequenceNumbersService seqNoService() { stats2.getUserData().get(Translog.TRANSLOG_GENERATION_KEY), not(equalTo(stats1.getUserData().get(Translog.TRANSLOG_GENERATION_KEY)))); assertThat(stats2.getUserData().get(Translog.TRANSLOG_UUID_KEY), equalTo(stats1.getUserData().get(Translog.TRANSLOG_UUID_KEY))); - assertThat(Long.parseLong(stats2.getUserData().get(InternalEngine.LOCAL_CHECKPOINT_KEY)), equalTo(localCheckpoint.get())); - assertThat(stats2.getUserData(), hasKey(InternalEngine.MAX_SEQ_NO)); - assertThat(Long.parseLong(stats2.getUserData().get(InternalEngine.MAX_SEQ_NO)), equalTo(maxSeqNo.get())); + assertThat(Long.parseLong(stats2.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), equalTo(localCheckpoint.get())); + assertThat(stats2.getUserData(), hasKey(SequenceNumbers.MAX_SEQ_NO)); + assertThat(Long.parseLong(stats2.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), equalTo(maxSeqNo.get())); } finally { IOUtils.close(engine); } @@ -1720,14 +1720,14 @@ public void testSeqNoAndCheckpoints() throws IOException { assertThat(initialEngine.seqNoService().stats().getGlobalCheckpoint(), equalTo(replicaLocalCheckpoint)); assertThat( - Long.parseLong(initialEngine.commitStats().getUserData().get(InternalEngine.LOCAL_CHECKPOINT_KEY)), + Long.parseLong(initialEngine.commitStats().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), equalTo(localCheckpoint)); initialEngine.getTranslog().sync(); // to guarantee the global checkpoint is written to the translog checkpoint assertThat( initialEngine.getTranslog().getLastSyncedGlobalCheckpoint(), equalTo(globalCheckpoint)); assertThat( - Long.parseLong(initialEngine.commitStats().getUserData().get(InternalEngine.MAX_SEQ_NO)), + Long.parseLong(initialEngine.commitStats().getUserData().get(SequenceNumbers.MAX_SEQ_NO)), equalTo(maxSeqNo)); } finally { @@ -1741,13 +1741,13 @@ public void testSeqNoAndCheckpoints() throws IOException { assertEquals(primarySeqNo, recoveringEngine.seqNoService().getMaxSeqNo()); assertThat( - Long.parseLong(recoveringEngine.commitStats().getUserData().get(InternalEngine.LOCAL_CHECKPOINT_KEY)), + Long.parseLong(recoveringEngine.commitStats().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), equalTo(primarySeqNo)); assertThat( recoveringEngine.getTranslog().getLastSyncedGlobalCheckpoint(), equalTo(globalCheckpoint)); assertThat( - Long.parseLong(recoveringEngine.commitStats().getUserData().get(InternalEngine.MAX_SEQ_NO)), + Long.parseLong(recoveringEngine.commitStats().getUserData().get(SequenceNumbers.MAX_SEQ_NO)), // after recovering from translog, all docs have been flushed to Lucene segments, so here we will assert // 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 @@ -1812,11 +1812,11 @@ public void run() { long prevMaxSeqNo = SequenceNumbersService.NO_OPS_PERFORMED; for (IndexCommit commit : DirectoryReader.listCommits(store.directory())) { Map userData = commit.getUserData(); - long localCheckpoint = userData.containsKey(InternalEngine.LOCAL_CHECKPOINT_KEY) ? - Long.parseLong(userData.get(InternalEngine.LOCAL_CHECKPOINT_KEY)) : + long localCheckpoint = userData.containsKey(SequenceNumbers.LOCAL_CHECKPOINT_KEY) ? + Long.parseLong(userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)) : SequenceNumbersService.NO_OPS_PERFORMED; - long maxSeqNo = userData.containsKey(InternalEngine.MAX_SEQ_NO) ? - Long.parseLong(userData.get(InternalEngine.MAX_SEQ_NO)) : + long maxSeqNo = userData.containsKey(SequenceNumbers.MAX_SEQ_NO) ? + Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)) : SequenceNumbersService.UNASSIGNED_SEQ_NO; // local checkpoint and max seq no shouldn't go backwards assertThat(localCheckpoint, greaterThanOrEqualTo(prevLocalCheckpoint)); diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 02b6eca43a338..31b5a5cd6f4c5 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.replication; import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.DocWriteResponse; @@ -35,8 +36,11 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingHelper; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.collect.Iterators; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; @@ -169,14 +173,45 @@ public void startPrimary() throws IOException { updateAllocationIDsOnPrimary(); } + private final Runnable replicaGlobalCheckpointSyncer = () -> { + throw new AssertionError("replicas can not sync global checkpoint"); + }; + public synchronized IndexShard addReplica() throws IOException { - final IndexShard replica = newShard(shardId, false, "s" + replicaId.incrementAndGet(), indexMetaData, - () -> { throw new AssertionError("replicas can't sync global checkpoint"); }, null); + final IndexShard replica = + newShard(shardId, false, "s" + replicaId.incrementAndGet(), indexMetaData, replicaGlobalCheckpointSyncer, null); replicas.add(replica); updateAllocationIDsOnPrimary(); return replica; } + public synchronized IndexShard addReplica(IndexShard replica) throws IOException { + replica.close("add", false); + IOUtils.close(replica.store()); + ShardRouting shardRouting = TestShardRouting.newShardRouting( + shardId, + replica.routingEntry().currentNodeId(), + false, ShardRoutingState.INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE); + + IndexShard newReplica = newShard(shardRouting, replica.shardPath(), indexMetaData, null, replicaGlobalCheckpointSyncer); + replicas.add(newReplica); + updateAllocationIDsOnPrimary(); + return newReplica; + } + + public synchronized List getReplicas() { + return Collections.unmodifiableList(replicas); + } + + synchronized boolean removeReplica(IndexShard replica) { + final boolean removed = replicas.remove(replica); + if (removed) { + updateAllocationIDsOnPrimary(); + } + return removed; + } + public void recoverReplica(IndexShard replica) throws IOException { recoverReplica(replica, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, version -> {})); } @@ -186,8 +221,10 @@ public void recoverReplica(IndexShard replica, BiFunction targetSupplier, - boolean markAsRecovering) throws IOException { + public void recoverReplica( + IndexShard replica, + BiFunction targetSupplier, + boolean markAsRecovering) throws IOException { ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering); updateAllocationIDsOnPrimary(); } diff --git a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 93b20633cf152..ee01aee64bd4a 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.replication; import org.apache.logging.log4j.Logger; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.store.Store; @@ -34,6 +35,9 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.not; + public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestCase { public void testIndexingDuringFileRecovery() throws Exception { @@ -57,11 +61,71 @@ public void testIndexingDuringFileRecovery() throws Exception { } } + public void testRecoveryOfDisconnectedReplica() throws Exception { + try (final ReplicationGroup shards = createGroup(1)) { + shards.startAll(); + int docs = shards.indexDocs(randomInt(50)); + shards.flush(); + shards.getPrimary().updateGlobalCheckpointOnPrimary(); + final IndexShard originalReplica = shards.getReplicas().get(0); + boolean replicaHasDocsSinceLastFlushedCheckpoint = false; + for (int i = 0; i < randomInt(2); i++) { + final int indexedDocs = shards.indexDocs(randomInt(5)); + docs += indexedDocs; + if (indexedDocs > 0) { + replicaHasDocsSinceLastFlushedCheckpoint = true; + } + + final boolean flush = randomBoolean(); + if (flush) { + originalReplica.flush(new FlushRequest()); + } + + final boolean sync = randomBoolean(); + if (sync) { + shards.getPrimary().updateGlobalCheckpointOnPrimary(); + if (flush) { + replicaHasDocsSinceLastFlushedCheckpoint = false; + } + } + } + + shards.removeReplica(originalReplica); + + final int missingOnReplica = shards.indexDocs(randomInt(5)); + docs += missingOnReplica; + replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0; + + if (randomBoolean()) { + shards.getPrimary().updateGlobalCheckpointOnPrimary(); + } + + final boolean flushPrimary = randomBoolean(); + if (flushPrimary) { + shards.flush(); + } + + final IndexShard recoveredReplica = shards.addReplica(originalReplica); + shards.recoverReplica(recoveredReplica); + if (flushPrimary && replicaHasDocsSinceLastFlushedCheckpoint) { + // replica has something to catch up with, but since we flushed the primary, we should fall back to full recovery + assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty())); + } else { + assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty()); + } + + docs += shards.indexDocs(randomInt(5)); + + shards.assertAllEqual(docs); + } + } + private static class BlockingTarget extends RecoveryTarget { + private final CountDownLatch recoveryBlocked; private final CountDownLatch releaseRecovery; private final RecoveryState.Stage stageToBlock; - public static final EnumSet SUPPORTED_STAGES = + static final EnumSet SUPPORTED_STAGES = EnumSet.of(RecoveryState.Stage.INDEX, RecoveryState.Stage.TRANSLOG, RecoveryState.Stage.FINALIZE); private final Logger logger; @@ -119,4 +183,5 @@ public void finalizeRecovery(long globalCheckpoint) { } } + } diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java index 8d8be2e402dc8..de7d05a56cdab 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java @@ -20,7 +20,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.junit.Before; @@ -42,19 +41,19 @@ public class GlobalCheckpointTests extends ESTestCase { - GlobalCheckpointService checkpointService; + GlobalCheckpointTracker tracker; @Override @Before public void setUp() throws Exception { super.setUp(); - checkpointService = new GlobalCheckpointService(new ShardId("test", "_na_", 0), - IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO); + tracker = + new GlobalCheckpointTracker(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO, logger); } public void testEmptyShards() { - assertFalse("checkpoint shouldn't be updated when the are no active shards", checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + assertFalse("checkpoint shouldn't be updated when the are no active shards", tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); } private final AtomicInteger aIdGenerator = new AtomicInteger(); @@ -81,7 +80,7 @@ public void testGlobalCheckpointUpdate() { // it is however nice not to assume this on this level and check we do the right thing. final long maxLocalCheckpoint = allocations.values().stream().min(Long::compare).orElse(UNASSIGNED_SEQ_NO); - assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); logger.info("--> using allocations"); allocations.keySet().forEach(aId -> { @@ -96,42 +95,42 @@ public void testGlobalCheckpointUpdate() { logger.info(" - [{}], local checkpoint [{}], [{}]", aId, allocations.get(aId), type); }); - checkpointService.updateAllocationIdsFromMaster(active, initializing); - initializing.forEach(aId -> checkpointService.markAllocationIdAsInSync(aId)); - allocations.keySet().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId))); + tracker.updateAllocationIdsFromMaster(active, initializing); + initializing.forEach(aId -> tracker.markAllocationIdAsInSync(aId)); + allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId))); - assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); - assertThat(checkpointService.updateCheckpointOnPrimary(), equalTo(maxLocalCheckpoint != UNASSIGNED_SEQ_NO)); - assertThat(checkpointService.getCheckpoint(), equalTo(maxLocalCheckpoint)); + assertThat(tracker.updateCheckpointOnPrimary(), equalTo(maxLocalCheckpoint != UNASSIGNED_SEQ_NO)); + assertThat(tracker.getCheckpoint(), equalTo(maxLocalCheckpoint)); // increment checkpoints active.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); initializing.forEach(aId -> allocations.put(aId, allocations.get(aId) + 1 + randomInt(4))); - allocations.keySet().forEach(aId -> checkpointService.updateLocalCheckpoint(aId, allocations.get(aId))); + allocations.keySet().forEach(aId -> tracker.updateLocalCheckpoint(aId, allocations.get(aId))); // now insert an unknown active/insync id , the checkpoint shouldn't change but a refresh should be requested. final String extraId = "extra_" + randomAsciiOfLength(5); // first check that adding it without the master blessing doesn't change anything. - checkpointService.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4)); - assertThat(checkpointService.getLocalCheckpointForAllocationId(extraId), equalTo(UNASSIGNED_SEQ_NO)); + tracker.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4)); + assertThat(tracker.getLocalCheckpointForAllocationId(extraId), equalTo(UNASSIGNED_SEQ_NO)); Set newActive = new HashSet<>(active); newActive.add(extraId); - checkpointService.updateAllocationIdsFromMaster(newActive, initializing); + tracker.updateAllocationIdsFromMaster(newActive, initializing); // we should ask for a refresh , but not update the checkpoint - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), equalTo(maxLocalCheckpoint)); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), equalTo(maxLocalCheckpoint)); // now notify for the new id - checkpointService.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4)); + tracker.updateLocalCheckpoint(extraId, maxLocalCheckpoint + 1 + randomInt(4)); // now it should be incremented - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), greaterThan(maxLocalCheckpoint)); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), greaterThan(maxLocalCheckpoint)); } public void testMissingActiveIdsPreventAdvance() { @@ -140,60 +139,60 @@ public void testMissingActiveIdsPreventAdvance() { final Map assigned = new HashMap<>(); assigned.putAll(active); assigned.putAll(initializing); - checkpointService.updateAllocationIdsFromMaster( + tracker.updateAllocationIdsFromMaster( new HashSet<>(randomSubsetOf(randomInt(active.size() - 1), active.keySet())), initializing.keySet()); - randomSubsetOf(initializing.keySet()).forEach(checkpointService::markAllocationIdAsInSync); - assigned.forEach(checkpointService::updateLocalCheckpoint); + randomSubsetOf(initializing.keySet()).forEach(tracker::markAllocationIdAsInSync); + assigned.forEach(tracker::updateLocalCheckpoint); // now mark all active shards - checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); + tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); // global checkpoint can't be advanced, but we need a sync - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); // update again - assigned.forEach(checkpointService::updateLocalCheckpoint); - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); + assigned.forEach(tracker::updateLocalCheckpoint); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); } public void testMissingInSyncIdsPreventAdvance() { final Map active = randomAllocationsWithLocalCheckpoints(0, 5); final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); - checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); - initializing.keySet().forEach(checkpointService::markAllocationIdAsInSync); + tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); + initializing.keySet().forEach(tracker::markAllocationIdAsInSync); randomSubsetOf(randomInt(initializing.size() - 1), - initializing.keySet()).forEach(aId -> checkpointService.updateLocalCheckpoint(aId, initializing.get(aId))); + initializing.keySet()).forEach(aId -> tracker.updateLocalCheckpoint(aId, initializing.get(aId))); - active.forEach(checkpointService::updateLocalCheckpoint); + active.forEach(tracker::updateLocalCheckpoint); // global checkpoint can't be advanced, but we need a sync - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), equalTo(UNASSIGNED_SEQ_NO)); // update again - initializing.forEach(checkpointService::updateLocalCheckpoint); - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); + initializing.forEach(tracker::updateLocalCheckpoint); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); } public void testInSyncIdsAreIgnoredIfNotValidatedByMaster() { final Map active = randomAllocationsWithLocalCheckpoints(1, 5); final Map initializing = randomAllocationsWithLocalCheckpoints(1, 5); final Map nonApproved = randomAllocationsWithLocalCheckpoints(1, 5); - checkpointService.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); - initializing.keySet().forEach(checkpointService::markAllocationIdAsInSync); - nonApproved.keySet().forEach(checkpointService::markAllocationIdAsInSync); + tracker.updateAllocationIdsFromMaster(active.keySet(), initializing.keySet()); + initializing.keySet().forEach(tracker::markAllocationIdAsInSync); + nonApproved.keySet().forEach(tracker::markAllocationIdAsInSync); List> allocations = Arrays.asList(active, initializing, nonApproved); Collections.shuffle(allocations, random()); - allocations.forEach(a -> a.forEach(checkpointService::updateLocalCheckpoint)); + allocations.forEach(a -> a.forEach(tracker::updateLocalCheckpoint)); // global checkpoint can be advanced, but we need a sync - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), not(equalTo(UNASSIGNED_SEQ_NO))); } public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { @@ -212,33 +211,33 @@ public void testInSyncIdsAreRemovedIfNotValidatedByMaster() { if (randomBoolean()) { allocations.putAll(initializingToBeRemoved); } - checkpointService.updateAllocationIdsFromMaster(active, initializing); + tracker.updateAllocationIdsFromMaster(active, initializing); if (randomBoolean()) { - initializingToStay.keySet().forEach(checkpointService::markAllocationIdAsInSync); + initializingToStay.keySet().forEach(tracker::markAllocationIdAsInSync); } else { - initializing.forEach(checkpointService::markAllocationIdAsInSync); + initializing.forEach(tracker::markAllocationIdAsInSync); } if (randomBoolean()) { - allocations.forEach(checkpointService::updateLocalCheckpoint); + allocations.forEach(tracker::updateLocalCheckpoint); } // global checkpoint may be advanced, but we need a sync in any case - assertTrue(checkpointService.updateCheckpointOnPrimary()); + assertTrue(tracker.updateCheckpointOnPrimary()); // now remove shards if (randomBoolean()) { - checkpointService.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); - allocations.forEach((aid, ckp) -> checkpointService.updateLocalCheckpoint(aid, ckp + 10L)); + tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); + allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L)); } else { - allocations.forEach((aid, ckp) -> checkpointService.updateLocalCheckpoint(aid, ckp + 10L)); - checkpointService.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); + allocations.forEach((aid, ckp) -> tracker.updateLocalCheckpoint(aid, ckp + 10L)); + tracker.updateAllocationIdsFromMaster(activeToStay.keySet(), initializingToStay.keySet()); } final long checkpoint = Stream.concat(activeToStay.values().stream(), initializingToStay.values().stream()) .min(Long::compare).get() + 10; // we added 10 to make sure it's advanced in the second time // global checkpoint is advanced and we need a sync - assertTrue(checkpointService.updateCheckpointOnPrimary()); - assertThat(checkpointService.getCheckpoint(), equalTo(checkpoint)); + assertTrue(tracker.updateCheckpointOnPrimary()); + assertThat(tracker.getCheckpoint(), equalTo(checkpoint)); } } diff --git a/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java b/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointTrackerTests.java similarity index 57% rename from core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java rename to core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointTrackerTests.java index d1f4c3987b2e2..74183670ecb1f 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/LocalCheckpointTrackerTests.java @@ -16,12 +16,13 @@ * specific language governing permissions and limitations * under the License. */ + package org.elasticsearch.index.seqno; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.Randomness; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.AbstractRunnable; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.junit.Before; @@ -31,16 +32,18 @@ import java.util.Collections; import java.util.List; import java.util.Set; +import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; -public class LocalCheckpointServiceTests extends ESTestCase { +public class LocalCheckpointTrackerTests extends ESTestCase { - private LocalCheckpointService checkpointService; + private LocalCheckpointTracker tracker; private final int SMALL_CHUNK_SIZE = 4; @@ -48,45 +51,47 @@ public class LocalCheckpointServiceTests extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); - checkpointService = getCheckpointService(); + tracker = getTracker(); } - private LocalCheckpointService getCheckpointService() { - return new LocalCheckpointService( - new ShardId("test", "_na_", 0), - IndexSettingsModule.newIndexSettings("test", - Settings.builder() - .put(LocalCheckpointService.SETTINGS_BIT_ARRAYS_SIZE.getKey(), SMALL_CHUNK_SIZE) - .build()), - SequenceNumbersService.NO_OPS_PERFORMED, - SequenceNumbersService.NO_OPS_PERFORMED); + private LocalCheckpointTracker getTracker() { + return new LocalCheckpointTracker( + IndexSettingsModule.newIndexSettings( + "test", + Settings + .builder() + .put(LocalCheckpointTracker.SETTINGS_BIT_ARRAYS_SIZE.getKey(), SMALL_CHUNK_SIZE) + .build()), + SequenceNumbersService.NO_OPS_PERFORMED, + SequenceNumbersService.NO_OPS_PERFORMED + ); } public void testSimplePrimary() { long seqNo1, seqNo2; - assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); - seqNo1 = checkpointService.generateSeqNo(); + assertThat(tracker.getCheckpoint(), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); + seqNo1 = tracker.generateSeqNo(); assertThat(seqNo1, equalTo(0L)); - checkpointService.markSeqNoAsCompleted(seqNo1); - assertThat(checkpointService.getCheckpoint(), equalTo(0L)); - seqNo1 = checkpointService.generateSeqNo(); - seqNo2 = checkpointService.generateSeqNo(); + tracker.markSeqNoAsCompleted(seqNo1); + assertThat(tracker.getCheckpoint(), equalTo(0L)); + seqNo1 = tracker.generateSeqNo(); + seqNo2 = tracker.generateSeqNo(); assertThat(seqNo1, equalTo(1L)); assertThat(seqNo2, equalTo(2L)); - checkpointService.markSeqNoAsCompleted(seqNo2); - assertThat(checkpointService.getCheckpoint(), equalTo(0L)); - checkpointService.markSeqNoAsCompleted(seqNo1); - assertThat(checkpointService.getCheckpoint(), equalTo(2L)); + tracker.markSeqNoAsCompleted(seqNo2); + assertThat(tracker.getCheckpoint(), equalTo(0L)); + tracker.markSeqNoAsCompleted(seqNo1); + assertThat(tracker.getCheckpoint(), equalTo(2L)); } public void testSimpleReplica() { - assertThat(checkpointService.getCheckpoint(), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); - checkpointService.markSeqNoAsCompleted(0L); - assertThat(checkpointService.getCheckpoint(), equalTo(0L)); - checkpointService.markSeqNoAsCompleted(2L); - assertThat(checkpointService.getCheckpoint(), equalTo(0L)); - checkpointService.markSeqNoAsCompleted(1L); - assertThat(checkpointService.getCheckpoint(), equalTo(2L)); + assertThat(tracker.getCheckpoint(), equalTo(SequenceNumbersService.NO_OPS_PERFORMED)); + tracker.markSeqNoAsCompleted(0L); + assertThat(tracker.getCheckpoint(), equalTo(0L)); + tracker.markSeqNoAsCompleted(2L); + assertThat(tracker.getCheckpoint(), equalTo(0L)); + tracker.markSeqNoAsCompleted(1L); + assertThat(tracker.getCheckpoint(), equalTo(2L)); } public void testSimpleOverFlow() { @@ -99,11 +104,11 @@ public void testSimpleOverFlow() { } Collections.shuffle(seqNoList, random()); for (Integer seqNo : seqNoList) { - checkpointService.markSeqNoAsCompleted(seqNo); + tracker.markSeqNoAsCompleted(seqNo); } - assertThat(checkpointService.checkpoint, equalTo(maxOps - 1L)); - assertThat(checkpointService.processedSeqNo.size(), equalTo(aligned ? 0 : 1)); - assertThat(checkpointService.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); + assertThat(tracker.checkpoint, equalTo(maxOps - 1L)); + assertThat(tracker.processedSeqNo.size(), equalTo(aligned ? 0 : 1)); + assertThat(tracker.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); } public void testConcurrentPrimary() throws InterruptedException { @@ -125,10 +130,10 @@ public void onFailure(Exception e) { protected void doRun() throws Exception { barrier.await(); for (int i = 0; i < opsPerThread; i++) { - long seqNo = checkpointService.generateSeqNo(); + long seqNo = tracker.generateSeqNo(); logger.info("[t{}] started [{}]", threadId, seqNo); if (seqNo != unFinishedSeq) { - checkpointService.markSeqNoAsCompleted(seqNo); + tracker.markSeqNoAsCompleted(seqNo); logger.info("[t{}] completed [{}]", threadId, seqNo); } } @@ -139,12 +144,12 @@ protected void doRun() throws Exception { for (Thread thread : threads) { thread.join(); } - assertThat(checkpointService.getMaxSeqNo(), equalTo(maxOps - 1L)); - assertThat(checkpointService.getCheckpoint(), equalTo(unFinishedSeq - 1L)); - checkpointService.markSeqNoAsCompleted(unFinishedSeq); - assertThat(checkpointService.getCheckpoint(), equalTo(maxOps - 1L)); - assertThat(checkpointService.processedSeqNo.size(), isOneOf(0, 1)); - assertThat(checkpointService.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); + assertThat(tracker.getMaxSeqNo(), equalTo(maxOps - 1L)); + assertThat(tracker.getCheckpoint(), equalTo(unFinishedSeq - 1L)); + tracker.markSeqNoAsCompleted(unFinishedSeq); + assertThat(tracker.getCheckpoint(), equalTo(maxOps - 1L)); + assertThat(tracker.processedSeqNo.size(), isOneOf(0, 1)); + assertThat(tracker.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); } public void testConcurrentReplica() throws InterruptedException { @@ -177,7 +182,7 @@ protected void doRun() throws Exception { Integer[] ops = seqNoPerThread[threadId]; for (int seqNo : ops) { if (seqNo != unFinishedSeq) { - checkpointService.markSeqNoAsCompleted(seqNo); + tracker.markSeqNoAsCompleted(seqNo); logger.info("[t{}] completed [{}]", threadId, seqNo); } } @@ -188,11 +193,48 @@ protected void doRun() throws Exception { for (Thread thread : threads) { thread.join(); } - assertThat(checkpointService.getMaxSeqNo(), equalTo(maxOps - 1L)); - assertThat(checkpointService.getCheckpoint(), equalTo(unFinishedSeq - 1L)); - checkpointService.markSeqNoAsCompleted(unFinishedSeq); - assertThat(checkpointService.getCheckpoint(), equalTo(maxOps - 1L)); - assertThat(checkpointService.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); + assertThat(tracker.getMaxSeqNo(), equalTo(maxOps - 1L)); + assertThat(tracker.getCheckpoint(), equalTo(unFinishedSeq - 1L)); + tracker.markSeqNoAsCompleted(unFinishedSeq); + assertThat(tracker.getCheckpoint(), equalTo(maxOps - 1L)); + assertThat(tracker.firstProcessedSeqNo, equalTo(((long) maxOps / SMALL_CHUNK_SIZE) * SMALL_CHUNK_SIZE)); + } + + public void testWaitForOpsToComplete() throws BrokenBarrierException, InterruptedException { + final int seqNo = randomIntBetween(0, 32); + final CyclicBarrier barrier = new CyclicBarrier(2); + final AtomicBoolean complete = new AtomicBoolean(); + final Thread thread = new Thread(() -> { + try { + // sychronize starting with the test thread + barrier.await(); + tracker.waitForOpsToComplete(seqNo); + complete.set(true); + // synchronize with the test thread checking if we are no longer waiting + barrier.await(); + } catch (BrokenBarrierException | InterruptedException e) { + throw new RuntimeException(e); + } + }); + + thread.start(); + + // synchronize starting with the waiting thread + barrier.await(); + + final List 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)); + assertFalse(complete.get()); + } + + tracker.markSeqNoAsCompleted(elements.get(elements.size() - 1)); + // synchronize with the waiting thread to mark that it is complete + barrier.await(); + assertTrue(complete.get()); + + thread.join(); } } diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java index 95067992eed7b..3e6135a954cdf 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java @@ -44,7 +44,7 @@ public class IndexPrimaryRelocationIT extends ESIntegTestCase { private static final int RELOCATION_COUNT = 25; - @TestLogging("_root:DEBUG,org.elasticsearch.action.delete:TRACE,org.elasticsearch.action.index:TRACE,index.shard:TRACE,org.elasticsearch.cluster.service:TRACE") + @TestLogging("_root:DEBUG,org.elasticsearch.action.delete:TRACE,org.elasticsearch.action.index:TRACE,org.elasticsearch.index.shard:TRACE,cluster.service:TRACE") public void testPrimaryRelocationWhileIndexing() throws Exception { internalCluster().ensureAtLeastNumDataNodes(randomIntBetween(2, 3)); client().admin().indices().prepareCreate("test") @@ -82,7 +82,12 @@ public void run() { .add(new MoveAllocationCommand("test", 0, relocationSource.getId(), relocationTarget.getId())) .execute().actionGet(); ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNoRelocatingShards(true).execute().actionGet(); - assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); + try { + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); + } catch (AssertionError e) { + logger.info("hi", e); + throw e; + } logger.info("--> [iteration {}] relocation complete", i); relocationSource = relocationTarget; if (indexingThread.isAlive() == false) { // indexing process aborted early, no need for more relocations as test has already failed @@ -97,4 +102,5 @@ public void run() { ElasticsearchAssertions.assertHitCount(client().prepareSearch("test")// extra paranoia ;) .setQuery(QueryBuilders.termQuery("auto", true)).get(), numAutoGenDocs.get()); } + } diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index f109bc4e37837..98da065cac0a5 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -41,6 +41,11 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.SegmentsStats; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRelocatedException; import org.elasticsearch.index.shard.IndexShardState; @@ -65,7 +70,9 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyBoolean; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -79,10 +86,14 @@ public void testSendFiles() throws Throwable { Settings settings = Settings.builder().put("indices.recovery.concurrent_streams", 1). put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); - StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - null, randomBoolean(), randomLong()); + final StartRecoveryRequest request = new StartRecoveryRequest( + shardId, + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + null, + randomBoolean(), + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : randomNonNegativeLong()); Store store = newStore(createTempDir()); RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, () -> 0L, e -> () -> {}, recoverySettings.getChunkSize().bytesAsInt(), logger); @@ -131,10 +142,15 @@ public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { Settings settings = Settings.builder().put("indices.recovery.concurrent_streams", 1). put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); - StartRecoveryRequest request = new StartRecoveryRequest(shardId, + final StartRecoveryRequest request = + new StartRecoveryRequest( + shardId, new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - null, randomBoolean(), randomLong()); + null, + randomBoolean(), + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); AtomicBoolean failedEngine = new AtomicBoolean(false); @@ -195,10 +211,15 @@ public void testHandleExceptinoOnSendSendFiles() throws Throwable { Settings settings = Settings.builder().put("indices.recovery.concurrent_streams", 1). put("indices.recovery.concurrent_small_file_streams", 1).build(); final RecoverySettings recoverySettings = new RecoverySettings(settings, service); - StartRecoveryRequest request = new StartRecoveryRequest(shardId, + final StartRecoveryRequest request = + new StartRecoveryRequest( + shardId, new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - null, randomBoolean(), randomLong()); + null, + randomBoolean(), + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); Path tempDir = createTempDir(); Store store = newStore(tempDir, false); AtomicBoolean failedEngine = new AtomicBoolean(false); @@ -254,18 +275,32 @@ protected void failEngine(IOException cause) { public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws IOException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); - StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - null, false, randomLong()); - IndexShard shard = mock(IndexShard.class); - Translog.View translogView = mock(Translog.View.class); + final StartRecoveryRequest request = + new StartRecoveryRequest( + shardId, + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + null, + false, + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); + final IndexShard shard = mock(IndexShard.class); + when(shard.seqNoStats()).thenReturn(mock(SeqNoStats.class)); + final Translog.View translogView = mock(Translog.View.class); when(shard.acquireTranslogView()).thenReturn(translogView); when(shard.state()).thenReturn(IndexShardState.RELOCATED); - AtomicBoolean phase1Called = new AtomicBoolean(); - AtomicBoolean phase2Called = new AtomicBoolean(); - RecoverySourceHandler handler = new RecoverySourceHandler(shard, null, request, () -> 0L, e -> () -> {}, - recoverySettings.getChunkSize().bytesAsInt(), logger) { + final AtomicBoolean phase1Called = new AtomicBoolean(); + final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean(); + final AtomicBoolean phase2Called = new AtomicBoolean(); + final RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + mock(RecoveryTargetHandler.class), + request, + () -> 0L, + e -> () -> { + }, + recoverySettings.getChunkSize().bytesAsInt(), + logger) { @Override public void phase1(final IndexCommit snapshot, final Translog.View translogView) { @@ -273,29 +308,44 @@ public void phase1(final IndexCommit snapshot, final Translog.View translogView) } @Override - public void phase2(Translog.Snapshot snapshot) { + void prepareTargetForTranslog(int totalTranslogOps) throws IOException { + prepareTargetForTranslogCalled.set(true); + } + + @Override + public void phase2(Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { phase2Called.set(true); } + }; - expectThrows(IndexShardRelocatedException.class, () -> handler.recoverToTarget()); - assertTrue(phase1Called.get()); + expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget); + // phase1 should only be attempted if we are not doing a sequence number-based recovery + assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertTrue(prepareTargetForTranslogCalled.get()); assertFalse(phase2Called.get()); } public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, InterruptedException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); - StartRecoveryRequest request = new StartRecoveryRequest(shardId, - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), - null, true, randomLong()); - AtomicBoolean phase1Called = new AtomicBoolean(); - AtomicBoolean phase2Called = new AtomicBoolean(); - AtomicBoolean ensureClusterStateVersionCalled = new AtomicBoolean(); - AtomicBoolean recoveriesDelayed = new AtomicBoolean(); - AtomicBoolean relocated = new AtomicBoolean(); - - IndexShard shard = mock(IndexShard.class); - Translog.View translogView = mock(Translog.View.class); + final StartRecoveryRequest request = + new StartRecoveryRequest( + shardId, + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT), + null, + true, + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); + final AtomicBoolean phase1Called = new AtomicBoolean(); + final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean(); + final AtomicBoolean phase2Called = new AtomicBoolean(); + final AtomicBoolean ensureClusterStateVersionCalled = new AtomicBoolean(); + final AtomicBoolean recoveriesDelayed = new AtomicBoolean(); + final AtomicBoolean relocated = new AtomicBoolean(); + + final IndexShard shard = mock(IndexShard.class); + when(shard.seqNoStats()).thenReturn(mock(SeqNoStats.class)); + final Translog.View translogView = mock(Translog.View.class); when(shard.acquireTranslogView()).thenReturn(translogView); when(shard.state()).then(i -> relocated.get() ? IndexShardState.RELOCATED : IndexShardState.STARTED); doAnswer(i -> { @@ -304,16 +354,17 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int return null; }).when(shard).relocated(any(String.class)); - RecoveryTargetHandler targetHandler = mock(RecoveryTargetHandler.class); - final Supplier currentClusterStateVersionSupplier = () -> { assertFalse(ensureClusterStateVersionCalled.get()); assertTrue(recoveriesDelayed.get()); ensureClusterStateVersionCalled.set(true); return 0L; }; + final Function delayNewRecoveries = s -> { - assertTrue(phase1Called.get()); + // phase1 should only be attempted if we are not doing a sequence number-based recovery + assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); assertFalse(recoveriesDelayed.get()); @@ -324,8 +375,14 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int }; }; - RecoverySourceHandler handler = new RecoverySourceHandler(shard, targetHandler, request, currentClusterStateVersionSupplier, - delayNewRecoveries, recoverySettings.getChunkSize().bytesAsInt(), logger) { + final RecoverySourceHandler handler = new RecoverySourceHandler( + shard, + mock(RecoveryTargetHandler.class), + request, + currentClusterStateVersionSupplier, + delayNewRecoveries, + recoverySettings.getChunkSize().bytesAsInt(), + logger) { @Override public void phase1(final IndexCommit snapshot, final Translog.View translogView) { @@ -333,13 +390,22 @@ public void phase1(final IndexCommit snapshot, final Translog.View translogView) } @Override - public void phase2(Translog.Snapshot snapshot) { + void prepareTargetForTranslog(int totalTranslogOps) throws IOException { + prepareTargetForTranslogCalled.set(true); + } + + @Override + public void phase2(final Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { phase2Called.set(true); } + }; + handler.recoverToTarget(); assertTrue(ensureClusterStateVersionCalled.get()); - assertTrue(phase1Called.get()); + // phase1 should only be attempted if we are not doing a sequence number-based recovery + assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); assertTrue(relocated.get()); assertFalse(recoveriesDelayed.get()); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java index 691b043e86aa0..e91eca4ad6d1d 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java @@ -23,6 +23,8 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.test.ESTestCase; @@ -36,33 +38,40 @@ import static org.hamcrest.Matchers.equalTo; public class StartRecoveryRequestTests extends ESTestCase { + public void testSerialization() throws Exception { - Version targetNodeVersion = randomVersion(random()); - StartRecoveryRequest outRequest = new StartRecoveryRequest( + final Version targetNodeVersion = randomVersion(random()); + final StartRecoveryRequest outRequest = new StartRecoveryRequest( new ShardId("test", "_na_", 0), new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), new DiscoveryNode("b", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), Store.MetadataSnapshot.EMPTY, randomBoolean(), - 1L - ); - ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); - OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); + randomNonNegativeLong(), + randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : randomNonNegativeLong()); + + final ByteArrayOutputStream outBuffer = new ByteArrayOutputStream(); + final OutputStreamStreamOutput out = new OutputStreamStreamOutput(outBuffer); out.setVersion(targetNodeVersion); outRequest.writeTo(out); - ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); + final ByteArrayInputStream inBuffer = new ByteArrayInputStream(outBuffer.toByteArray()); InputStreamStreamInput in = new InputStreamStreamInput(inBuffer); in.setVersion(targetNodeVersion); - StartRecoveryRequest inRequest = new StartRecoveryRequest(); + final StartRecoveryRequest inRequest = new StartRecoveryRequest(); inRequest.readFrom(in); assertThat(outRequest.shardId(), equalTo(inRequest.shardId())); assertThat(outRequest.sourceNode(), equalTo(inRequest.sourceNode())); assertThat(outRequest.targetNode(), equalTo(inRequest.targetNode())); assertThat(outRequest.metadataSnapshot().asMap(), equalTo(inRequest.metadataSnapshot().asMap())); - assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); assertThat(outRequest.isPrimaryRelocation(), equalTo(inRequest.isPrimaryRelocation())); + assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); + if (targetNodeVersion.onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { + assertThat(outRequest.startingSeqNo(), equalTo(inRequest.startingSeqNo())); + } else { + assertThat(SequenceNumbersService.UNASSIGNED_SEQ_NO, equalTo(inRequest.startingSeqNo())); + } } } diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java index bbc434120d864..2a5fe72760e86 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java @@ -126,10 +126,10 @@ public void testResetRecovery() throws Exception { IndexShard indexShard = recoveryTarget.indexShard(); Store store = recoveryTarget.store(); String tempFileName = recoveryTarget.getTempNameForFile("foobar"); - RecoveryTarget resetRecovery = collection.resetRecovery(recoveryId, TimeValue.timeValueMinutes(60)); + RecoveryTarget resetRecovery = collection.resetRecovery(recoveryId, TimeValue.timeValueMinutes(60), RecoveryTarget::retry); final long resetRecoveryId = resetRecovery.recoveryId(); assertNotSame(recoveryTarget, resetRecovery); - assertNotSame(recoveryTarget.CancellableThreads(), resetRecovery.CancellableThreads()); + assertNotSame(recoveryTarget.cancellableThreads(), resetRecovery.cancellableThreads()); assertSame(indexShard, resetRecovery.indexShard()); assertSame(store, resetRecovery.store()); assertEquals(referencesToStore, resetRecovery.store().refCount()); diff --git a/qa/backwards-5.0/src/test/java/org/elasticsearch/backwards/IndexingIT.java b/qa/backwards-5.0/src/test/java/org/elasticsearch/backwards/IndexingIT.java index 1e3ae7aaa9996..b5bd18aba5f14 100644 --- a/qa/backwards-5.0/src/test/java/org/elasticsearch/backwards/IndexingIT.java +++ b/qa/backwards-5.0/src/test/java/org/elasticsearch/backwards/IndexingIT.java @@ -176,7 +176,7 @@ private List buildShards(Nodes nodes, RestClient client) throws IOExcepti final Node node = nodes.getSafe(nodeId); final SeqNoStats seqNoStats; if (node.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { - Integer maxSeqNo = ObjectPath.evaluate(shard, "seq_no.max"); + Integer maxSeqNo = ObjectPath.evaluate(shard, "seq_no.max_seq_no"); Integer localCheckpoint = ObjectPath.evaluate(shard, "seq_no.local_checkpoint"); Integer globalCheckpoint = ObjectPath.evaluate(shard, "seq_no.global_checkpoint"); seqNoStats = new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 605b9026c2636..665acae5fbe12 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -25,6 +25,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.Bits; import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; @@ -40,6 +41,7 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.IndexSettings; @@ -48,12 +50,15 @@ import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.query.DisabledQueryCache; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.UidFieldMapper; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; @@ -76,10 +81,13 @@ import java.util.HashSet; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.function.BiFunction; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.not; /** * A base class for unit tests that need to create and shutdown {@link IndexShard} instances easily, @@ -377,32 +385,71 @@ protected void recoverReplica(IndexShard replica, IndexShard primary) throws IOE } /** - * Recovers a replica from the give primary, allow the user to supply a custom recovery target. - * A typical usage of a custome recovery target is to assert things in the various stages of recovery - * - * @param markAsRecovering set to false if you have already marked the replica as recovering + * Recovers a replica from the give primary, allow the user to supply a custom recovery target. A typical usage of a custom recovery + * target is to assert things in the various stages of recovery. + * @param replica the recovery target shard + * @param primary the recovery source shard + * @param targetSupplier supplies an instance of {@link RecoveryTarget} + * @param markAsRecovering set to {@code false} if the replica is marked as recovering */ - protected void recoverReplica(IndexShard replica, IndexShard primary, - BiFunction targetSupplier, - boolean markAsRecovering) - throws IOException { + protected final void recoverReplica(final IndexShard replica, + final IndexShard primary, + final BiFunction targetSupplier, + final boolean markAsRecovering) throws IOException { final DiscoveryNode pNode = getFakeDiscoNode(primary.routingEntry().currentNodeId()); final DiscoveryNode rNode = getFakeDiscoNode(replica.routingEntry().currentNodeId()); if (markAsRecovering) { - replica.markAsRecovering("remote", - new RecoveryState(replica.routingEntry(), pNode, rNode)); + replica.markAsRecovering("remote", new RecoveryState(replica.routingEntry(), pNode, rNode)); } else { assertEquals(replica.state(), IndexShardState.RECOVERING); } replica.prepareForIndexRecovery(); - RecoveryTarget recoveryTarget = targetSupplier.apply(replica, pNode); - StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), pNode, rNode, - getMetadataSnapshotOrEmpty(replica), false, 0); - RecoverySourceHandler recovery = new RecoverySourceHandler(primary, recoveryTarget, request, () -> 0L, e -> () -> { - }, - (int) ByteSizeUnit.MB.toKB(1), logger); - recovery.recoverToTarget(); - recoveryTarget.markAsDone(); + final RecoveryTarget recoveryTarget = targetSupplier.apply(replica, pNode); + + final Store.MetadataSnapshot snapshot = getMetadataSnapshotOrEmpty(replica); + final long startingSeqNo; + if (snapshot.size() > 0) { + startingSeqNo = PeerRecoveryTargetService.getStartingSeqNo(recoveryTarget); + } else { + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + } + + final StartRecoveryRequest request = + new StartRecoveryRequest(replica.shardId(), pNode, rNode, snapshot, false, 0, startingSeqNo); + final RecoverySourceHandler recovery = new RecoverySourceHandler( + primary, + recoveryTarget, + request, + () -> 0L, + e -> () -> {}, + (int) ByteSizeUnit.MB.toBytes(1), + logger); + try { + recovery.recoverToTarget(); + recoveryTarget.markAsDone(); + } catch (final RecoveryEngineException ree) { + if (PeerRecoveryTargetService.sequenceNumberBasedRecoveryFailed(ree.getCause())) { + assertThat(request.startingSeqNo(), not(equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO))); + final RecoveryTarget retryRecoveryTarget = targetSupplier.apply(replica, pNode); + recoveryTarget.resetRecovery(retryRecoveryTarget.cancellableThreads()); + replica.performRecoveryRestart(); + replica.prepareForIndexRecovery(); + final StartRecoveryRequest retryRequest = + new StartRecoveryRequest(replica.shardId(), pNode, rNode, snapshot, false, 0, SequenceNumbersService.UNASSIGNED_SEQ_NO); + final RecoverySourceHandler retryRecovery = new RecoverySourceHandler( + primary, + retryRecoveryTarget, + retryRequest, + () -> 0L, + e -> () -> {}, + (int) ByteSizeUnit.MB.toBytes(1), + logger); + retryRecovery.recoverToTarget(); + retryRecoveryTarget.markAsDone(); + } else { + throw ree; + } + } replica.updateRoutingEntry(ShardRoutingHelper.moveToStarted(replica.routingEntry())); } From d360a237f2a13b33d86565d347a43a4aaf150fcf Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 6 Jan 2017 14:05:59 -0500 Subject: [PATCH 02/41] Simplify sequence number-based recovery This commit simplifies sequence number-based recovery. Rather than execute a dance between the replica and the primary of having the replica request a sequence number-based recovery, then failling that recovery if it is not possible and having the replica request a second file-based recovery, we simply check on the primary side if a sequence number-based recovery is possible and immediately fallback to file-basd recovery if not. --- .../index/seqno/LocalCheckpointTracker.java | 3 +- .../recovery/PeerRecoveryTargetService.java | 58 +------- .../recovery/RecoveriesCollection.java | 7 +- .../recovery/RecoverySourceHandler.java | 131 +++++++----------- .../indices/recovery/RecoveryTarget.java | 49 +------ .../SharedFSRecoverySourceHandler.java | 2 +- .../recovery/IndexPrimaryRelocationIT.java | 7 +- .../recovery/RecoverySourceHandlerTests.java | 34 +++-- .../recovery/RecoveriesCollectionTests.java | 2 +- .../index/shard/IndexShardTestCase.java | 28 +--- 10 files changed, 92 insertions(+), 229 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java index 140b26b450868..06af7a2cecab4 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java @@ -69,7 +69,8 @@ public class LocalCheckpointTracker { * Initialize the local checkpoint service. The {@code maxSeqNo} should be set to the last sequence number assigned, or * {@link SequenceNumbersService#NO_OPS_PERFORMED} and {@code localCheckpoint} should be set to the last known local checkpoint, * or {@link SequenceNumbersService#NO_OPS_PERFORMED}. - * @param indexSettings the index settings + * + * @param indexSettings the index settings * @param maxSeqNo the last sequence number assigned, or {@link SequenceNumbersService#NO_OPS_PERFORMED} * @param localCheckpoint the last known local checkpoint, or {@link SequenceNumbersService#NO_OPS_PERFORMED} */ diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 3145cd81ad736..9adb2a74cdd69 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -157,26 +157,8 @@ protected void retryRecovery(final long recoveryId, final String reason, TimeVal retryRecovery(recoveryId, retryAfter, activityTimeout); } - protected void retryRecovery( - final long recoveryId, - final String reason, - final TimeValue retryAfter, - final TimeValue activityTimeout, - final Function supplier) { - logger.trace("will retry recovery with id [{}] in [{}] (reason [{}])", recoveryId, retryAfter, reason); - retryRecovery(recoveryId, retryAfter, activityTimeout, supplier); - } - - private void retryRecovery(final long recoveryId, TimeValue retryAfter, TimeValue activityTimeout) { - retryRecovery(recoveryId, retryAfter, activityTimeout, RecoveryTarget::retry); - } - - private void retryRecovery( - final long recoveryId, - final TimeValue retryAfter, - final TimeValue activityTimeout, - final Function supplier) { - RecoveryTarget newTarget = onGoingRecoveries.resetRecovery(recoveryId, activityTimeout, supplier); + private void retryRecovery(final long recoveryId, final TimeValue retryAfter, final TimeValue activityTimeout) { + RecoveryTarget newTarget = onGoingRecoveries.resetRecovery(recoveryId, activityTimeout); if (newTarget != null) { threadPool.schedule(retryAfter, ThreadPool.Names.GENERIC, new RecoveryRunner(newTarget.recoveryId())); } @@ -293,12 +275,6 @@ public RecoveryResponse newInstance() { return; } - if (sequenceNumberBasedRecoveryFailed(cause)) { - final String header = getSequenceNumberBasedRecoveryFailedHeader((ElasticsearchException) cause); - retryRecovery(recoveryId, header, TimeValue.ZERO, recoverySettings.activityTimeout(), RecoveryTarget::fileRecoveryRetry); - return; - } - if (cause instanceof AlreadyClosedException) { onGoingRecoveries.failRecovery(recoveryId, new RecoveryFailedException(request, "source shard is closed", cause), false); @@ -309,34 +285,6 @@ public RecoveryResponse newInstance() { } } - /** - * Obtain the {@link RecoverySourceHandler#SEQUENCE_NUMBER_BASED_RECOVERY_FAILED} header if it is present, otherwise {@code null}. - * - * @param ex the exception to inspect - * @return the header if it exists, otherwise {@code null} - */ - private static String getSequenceNumberBasedRecoveryFailedHeader(final ElasticsearchException ex) { - final List header = ex.getHeader(RecoverySourceHandler.SEQUENCE_NUMBER_BASED_RECOVERY_FAILED); - if (header != null && !header.isEmpty()) { - return header.get(0); - } else { - return null; - } - } - - /** - * Determine if the specified throwable represents an exception from a failed sequence number-based recovery. - * - * @param cause the throwable to inspect - * @return {@code true} iff the specified throwable represents an exception from a failed sequence number-based recovery - */ - public static boolean sequenceNumberBasedRecoveryFailed(final Throwable cause) { - if (cause instanceof ElasticsearchException) { - return getSequenceNumberBasedRecoveryFailedHeader((ElasticsearchException) cause) != null; - } - return false; - } - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") private static Optional EMPTY_METADATA_SNAPSHOT = Optional.of(Store.MetadataSnapshot.EMPTY); @@ -388,7 +336,7 @@ private Optional getStartRecoveryRequest(final RecoveryTar try { final long startingSeqNo; - if (metadataSnapshot.get().size() > 0 && recoveryTarget.canPerformSeqNoBasedRecovery()) { + if (metadataSnapshot.get().size() > 0) { startingSeqNo = getStartingSeqNo(recoveryTarget); logger.trace( "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index 0a05bb58b7841..4fb2e398e520a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -89,10 +89,7 @@ private void startRecoveryInternal(RecoveryTarget recoveryTarget, TimeValue acti * @see IndexShard#performRecoveryRestart() * @return newly created RecoveryTarget */ - public RecoveryTarget resetRecovery( - final long recoveryId, - final TimeValue activityTimeout, - final Function supplier) { + public RecoveryTarget resetRecovery(final long recoveryId, final TimeValue activityTimeout) { RecoveryTarget oldRecoveryTarget = null; final RecoveryTarget newRecoveryTarget; @@ -105,7 +102,7 @@ public RecoveryTarget resetRecovery( return null; } - newRecoveryTarget = supplier.apply(oldRecoveryTarget); + newRecoveryTarget = oldRecoveryTarget.retryCopy(); startRecoveryInternal(newRecoveryTarget, activityTimeout); } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 28158071d90ed..2f33069a6ca48 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -57,7 +57,6 @@ import java.io.OutputStream; import java.util.ArrayList; import java.util.List; -import java.util.Locale; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.StreamSupport; @@ -131,30 +130,22 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov public RecoveryResponse recoverToTarget() throws IOException { try (final Translog.View translogView = shard.acquireTranslogView()) { logger.trace("{} captured translog id [{}] for recovery", shard.shardId(), translogView.minTranslogGeneration()); - final IndexCommit phase1Snapshot; - try { - phase1Snapshot = shard.acquireIndexCommit(false); - } catch (final Exception e) { - IOUtils.closeWhileHandlingException(translogView); - throw new RecoveryEngineException(shard.shardId(), 1, "snapshot failed", e); - } - final long startingSeqNo; - final long endingSeqNo; - if (request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO) { - startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; - endingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + boolean isSequenceNumberBasedRecoveryPossible = request.startingSeqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO && + isTranslogReadyForSequenceNumberBasedRecovery(translogView); + + if (!isSequenceNumberBasedRecoveryPossible) { + final IndexCommit phase1Snapshot; try { - try { - phase1(phase1Snapshot, translogView); - } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); - } - try { - prepareTargetForTranslog(translogView.totalOperations()); - } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); - } + 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); @@ -162,33 +153,12 @@ public RecoveryResponse recoverToTarget() throws IOException { logger.warn("releasing snapshot caused exception", ex); } } - } else { - startingSeqNo = request.startingSeqNo(); - endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - if (endingSeqNo < startingSeqNo) { - final String message = String.format( - Locale.ROOT, - "requested starting operation [%d] higher than source operation maximum [%d]", - startingSeqNo, - endingSeqNo); - logger.debug("{} {}", shard.shardId(), message); - final ElasticsearchException ex = new ElasticsearchException(message); - ex.addHeader(SEQUENCE_NUMBER_BASED_RECOVERY_FAILED, message); - throw ex; - } - try { - prepareTargetForTranslog(translogView.totalOperations()); - } catch (final Exception e) { - throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); - } - // 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 - logger.trace( - "{} waiting for all operations in the range [{}, {}] to complete", - shard.shardId(), - startingSeqNo, - endingSeqNo); - cancellableThreads.execute(() -> shard.waitForOpsToComplete(endingSeqNo)); + } + + try { + prepareTargetForTranslog(translogView.totalOperations()); + } 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 @@ -211,7 +181,7 @@ public RecoveryResponse recoverToTarget() throws IOException { logger.trace("{} snapshot translog for recovery; current size is [{}]", shard.shardId(), translogView.totalOperations()); try { - phase2(translogView.snapshot(), startingSeqNo, endingSeqNo); + phase2(translogView.snapshot()); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); } @@ -221,6 +191,30 @@ public RecoveryResponse recoverToTarget() throws IOException { return response; } + boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) { + final long startingSeqNo = request.startingSeqNo(); + final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); + if (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 + logger.trace( + "{} waiting for all operations in the range [{}, {}] to complete", + shard.shardId(), + startingSeqNo, + endingSeqNo); + 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 = getNextOperationFromSnapshot(snapshot)) != null) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } + return tracker.getCheckpoint() >= endingSeqNo; + } + return false; + } + /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) @@ -231,7 +225,6 @@ public RecoveryResponse recoverToTarget() throws IOException { * checksum can be reused */ public void phase1(final IndexCommit snapshot, final Translog.View translogView) { - assert request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO; cancellableThreads.checkForCancel(); // Total size of segment files that are recovered long totalSize = 0; @@ -402,44 +395,24 @@ void prepareTargetForTranslog(final int totalTranslogOps) throws IOException { /** * Perform phase two of the recovery process. *

- * Phase two takes a snapshot of the current translog *without* acquiring the write lock (however, the translog snapshot is + * 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 */ - void phase2(final Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { + void phase2(final Translog.Snapshot snapshot) { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); - final LocalCheckpointTracker tracker; - if (startingSeqNo != SequenceNumbersService.UNASSIGNED_SEQ_NO) { - tracker = new LocalCheckpointTracker(shard.indexSettings(), startingSeqNo, startingSeqNo - 1); - } else { - tracker = new LocalCheckpointTracker(shard.indexSettings(), Long.MAX_VALUE, Long.MAX_VALUE); - } - 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 - final int totalOperations = sendSnapshot(snapshot, tracker); - - // check to see if all operations in the required range were sent to the target - if (tracker.getCheckpoint() < endingSeqNo) { - final String message = String.format( - Locale.ROOT, - "sequence number-based recovery failed due to missing ops in range [%d, %d]; first missed op [%d]", - startingSeqNo, - endingSeqNo, - tracker.getCheckpoint() + 1); - logger.debug("{} {}", shard.shardId(), message); - final ElasticsearchException ex = new ElasticsearchException(message); - ex.setShard(shard.shardId()); - ex.addHeader(SEQUENCE_NUMBER_BASED_RECOVERY_FAILED, message); - throw ex; - } + final int totalOperations = sendSnapshot(snapshot); stopWatch.stop(); logger.trace("{} recovery [phase2] to {}: took [{}]", request.shardId(), request.targetNode(), stopWatch.totalTime()); @@ -492,10 +465,9 @@ public void finalizeRecovery() { * Operations are bulked into a single request depending on an operation count limit or size-in-bytes limit. * * @param snapshot the translog snapshot to replay operations from - * @param tracker tracks the replayed operations * @return the total number of translog operations that were sent */ - protected int sendSnapshot(final Translog.Snapshot snapshot, final LocalCheckpointTracker tracker) { + protected int sendSnapshot(final Translog.Snapshot snapshot) { int ops = 0; long size = 0; int totalOperations = 0; @@ -516,7 +488,6 @@ protected int sendSnapshot(final Translog.Snapshot snapshot, final LocalCheckpoi ops++; size += operation.estimateSize(); totalOperations++; - tracker.markSeqNoAsCompleted(operation.seqNo()); // check if this request is past bytes threshold, and if so, send it off if (size >= chunkSizeInBytes) { diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 70deaea48e3d1..d9886efa07b21 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -92,16 +92,6 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget // last time this status was accessed private volatile long lastAccessTime = System.nanoTime(); - private volatile boolean canPerformSeqNoBasedRecovery = true; - - public boolean canPerformSeqNoBasedRecovery() { - return canPerformSeqNoBasedRecovery; - } - - public void setCanPerformSeqNoBasedRecovery(final boolean canPerformSeqNoBasedRecovery) { - this.canPerformSeqNoBasedRecovery = canPerformSeqNoBasedRecovery; - } - // latch that can be used to blockingly wait for RecoveryTarget to be closed private final CountDownLatch closedLatch = new CountDownLatch(1); @@ -117,24 +107,10 @@ public void setCanPerformSeqNoBasedRecovery(final boolean canPerformSeqNoBasedRe * version; necessary for primary relocation so that new primary knows about all other ongoing * replica recoveries when replicating documents (see {@link RecoverySourceHandler}) */ - public RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener, - Callback ensureClusterStateVersionCallback) { - this(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, true); - } - - /** - * Creates a new recovery target object that represents a recovery to the provided shard. - * - * @param indexShard local shard where we want to recover to - * @param sourceNode source node of the recovery where we recover from - * @param listener called when recovery is completed/failed - * @param ensureClusterStateVersionCallback callback to ensure that the current node is at least on a cluster state with the provided - * version; necessary for primary relocation so that new primary knows about all other ongoing - * replica recoveries when replicating documents (see {@link RecoverySourceHandler}) - * @param canPerformSeqNoBasedRecovery whether or not sequence number-based recovery can be performed - */ - private RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener, - Callback ensureClusterStateVersionCallback, boolean canPerformSeqNoBasedRecovery) { + public RecoveryTarget(final IndexShard indexShard, + final DiscoveryNode sourceNode, + final PeerRecoveryTargetService.RecoveryListener listener, + final Callback ensureClusterStateVersionCallback) { super("recovery_status"); this.cancellableThreads = new CancellableThreads(); this.recoveryId = idGenerator.incrementAndGet(); @@ -149,7 +125,6 @@ private RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerReco // make sure the store is not released until we are done. store.incRef(); indexShard.recoveryStats().incCurrentAsTarget(); - this.canPerformSeqNoBasedRecovery = canPerformSeqNoBasedRecovery; } /** @@ -157,18 +132,8 @@ private RecoveryTarget(IndexShard indexShard, DiscoveryNode sourceNode, PeerReco * * @return a copy of this recovery target */ - public RecoveryTarget retry() { - return new RecoveryTarget(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, canPerformSeqNoBasedRecovery); - } - - /** - * Returns a fresh recovery target to retry recovery from the same source node onto the same shard and using the same listener, but - * disabling sequence number-based recovery. - * - * @return a copy of this recovery target - */ - public RecoveryTarget fileRecoveryRetry() { - return new RecoveryTarget(indexShard, sourceNode, listener, ensureClusterStateVersionCallback, false); + public RecoveryTarget retryCopy() { + return new RecoveryTarget(indexShard, sourceNode, listener, ensureClusterStateVersionCallback); } public long recoveryId() { @@ -225,7 +190,7 @@ public void renameAllTempFiles() throws IOException { * Closes the current recovery target and waits up to a certain timeout for resources to be freed. * Returns true if resetting the recovery was successful, false if the recovery target is already cancelled / failed or marked as done. */ - public boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOException { + boolean resetRecovery(CancellableThreads newTargetCancellableThreads) throws IOException { if (finished.compareAndSet(false, true)) { try { logger.debug("reset of recovery with shard {} and id [{}]", shardId, recoveryId); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index b27687101fb37..d5cd4c9d92246 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -82,7 +82,7 @@ public RecoveryResponse recoverToTarget() throws IOException { } @Override - protected int sendSnapshot(final Translog.Snapshot snapshot, final LocalCheckpointTracker tracker) { + protected int sendSnapshot(final Translog.Snapshot snapshot) { logger.trace("{} skipping recovery of translog snapshot on shared filesystem to: {}", shard.shardId(), request.targetNode()); return 0; } diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java index 3e6135a954cdf..f1b1d0338c68d 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/IndexPrimaryRelocationIT.java @@ -82,12 +82,7 @@ public void run() { .add(new MoveAllocationCommand("test", 0, relocationSource.getId(), relocationTarget.getId())) .execute().actionGet(); ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForNoRelocatingShards(true).execute().actionGet(); - try { - assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); - } catch (AssertionError e) { - logger.info("hi", e); - throw e; - } + assertThat(clusterHealthResponse.isTimedOut(), equalTo(false)); logger.info("--> [iteration {}] relocation complete", i); relocationSource = relocationTarget; if (indexingThread.isAlive() == false) { // indexing process aborted early, no need for more relocations as test has already failed diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 98da065cac0a5..de39c5bb31974 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -41,10 +41,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.engine.SegmentsStats; -import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardRelocatedException; @@ -72,7 +69,6 @@ import static java.util.Collections.emptySet; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyBoolean; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -275,6 +271,8 @@ protected void failEngine(IOException cause) { public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws IOException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); + final boolean attemptSequenceNumberBasedRecovery = randomBoolean(); + final boolean isTranslogReadyForSequenceNumberBasedRecovery = attemptSequenceNumberBasedRecovery && randomBoolean(); final StartRecoveryRequest request = new StartRecoveryRequest( shardId, @@ -283,7 +281,7 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws I null, false, randomNonNegativeLong(), - randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); + attemptSequenceNumberBasedRecovery ? randomNonNegativeLong() : SequenceNumbersService.UNASSIGNED_SEQ_NO); final IndexShard shard = mock(IndexShard.class); when(shard.seqNoStats()).thenReturn(mock(SeqNoStats.class)); final Translog.View translogView = mock(Translog.View.class); @@ -302,6 +300,11 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Completed() throws I recoverySettings.getChunkSize().bytesAsInt(), logger) { + @Override + boolean isTranslogReadyForSequenceNumberBasedRecovery(Translog.View translogView) { + return isTranslogReadyForSequenceNumberBasedRecovery; + } + @Override public void phase1(final IndexCommit snapshot, final Translog.View translogView) { phase1Called.set(true); @@ -313,20 +316,22 @@ void prepareTargetForTranslog(int totalTranslogOps) throws IOException { } @Override - public void phase2(Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { + public void phase2(Translog.Snapshot snapshot) { phase2Called.set(true); } }; expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget); // phase1 should only be attempted if we are not doing a sequence number-based recovery - assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertFalse(phase2Called.get()); } public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, InterruptedException { final RecoverySettings recoverySettings = new RecoverySettings(Settings.EMPTY, service); + final boolean attemptSequenceNumberBasedRecovery = randomBoolean(); + final boolean isTranslogReadyForSequenceNumberBasedRecovery = attemptSequenceNumberBasedRecovery && randomBoolean(); final StartRecoveryRequest request = new StartRecoveryRequest( shardId, @@ -335,7 +340,7 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int null, true, randomNonNegativeLong(), - randomBoolean() ? SequenceNumbersService.UNASSIGNED_SEQ_NO : 0L); + attemptSequenceNumberBasedRecovery ? randomNonNegativeLong(): SequenceNumbersService.UNASSIGNED_SEQ_NO); final AtomicBoolean phase1Called = new AtomicBoolean(); final AtomicBoolean prepareTargetForTranslogCalled = new AtomicBoolean(); final AtomicBoolean phase2Called = new AtomicBoolean(); @@ -363,7 +368,7 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int final Function delayNewRecoveries = s -> { // phase1 should only be attempted if we are not doing a sequence number-based recovery - assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); @@ -384,18 +389,23 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int recoverySettings.getChunkSize().bytesAsInt(), logger) { + @Override + boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) { + return isTranslogReadyForSequenceNumberBasedRecovery; + } + @Override public void phase1(final IndexCommit snapshot, final Translog.View translogView) { phase1Called.set(true); } @Override - void prepareTargetForTranslog(int totalTranslogOps) throws IOException { + void prepareTargetForTranslog(final int totalTranslogOps) throws IOException { prepareTargetForTranslogCalled.set(true); } @Override - public void phase2(final Translog.Snapshot snapshot, final long startingSeqNo, final long endingSeqNo) { + public void phase2(final Translog.Snapshot snapshot) { phase2Called.set(true); } @@ -404,7 +414,7 @@ public void phase2(final Translog.Snapshot snapshot, final long startingSeqNo, f handler.recoverToTarget(); assertTrue(ensureClusterStateVersionCalled.get()); // phase1 should only be attempted if we are not doing a sequence number-based recovery - assertThat(phase1Called.get(), equalTo(request.startingSeqNo() == SequenceNumbersService.UNASSIGNED_SEQ_NO)); + assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); assertTrue(relocated.get()); diff --git a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java index 2a5fe72760e86..ca7290f5f7a85 100644 --- a/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java +++ b/core/src/test/java/org/elasticsearch/recovery/RecoveriesCollectionTests.java @@ -126,7 +126,7 @@ public void testResetRecovery() throws Exception { IndexShard indexShard = recoveryTarget.indexShard(); Store store = recoveryTarget.store(); String tempFileName = recoveryTarget.getTempNameForFile("foobar"); - RecoveryTarget resetRecovery = collection.resetRecovery(recoveryId, TimeValue.timeValueMinutes(60), RecoveryTarget::retry); + RecoveryTarget resetRecovery = collection.resetRecovery(recoveryId, TimeValue.timeValueMinutes(60)); final long resetRecoveryId = resetRecovery.recoveryId(); assertNotSame(recoveryTarget, resetRecovery); assertNotSame(recoveryTarget.cancellableThreads(), resetRecovery.cancellableThreads()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 665acae5fbe12..073f78186f970 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -424,32 +424,8 @@ protected final void recoverReplica(final IndexShard replica, e -> () -> {}, (int) ByteSizeUnit.MB.toBytes(1), logger); - try { - recovery.recoverToTarget(); - recoveryTarget.markAsDone(); - } catch (final RecoveryEngineException ree) { - if (PeerRecoveryTargetService.sequenceNumberBasedRecoveryFailed(ree.getCause())) { - assertThat(request.startingSeqNo(), not(equalTo(SequenceNumbersService.UNASSIGNED_SEQ_NO))); - final RecoveryTarget retryRecoveryTarget = targetSupplier.apply(replica, pNode); - recoveryTarget.resetRecovery(retryRecoveryTarget.cancellableThreads()); - replica.performRecoveryRestart(); - replica.prepareForIndexRecovery(); - final StartRecoveryRequest retryRequest = - new StartRecoveryRequest(replica.shardId(), pNode, rNode, snapshot, false, 0, SequenceNumbersService.UNASSIGNED_SEQ_NO); - final RecoverySourceHandler retryRecovery = new RecoverySourceHandler( - primary, - retryRecoveryTarget, - retryRequest, - () -> 0L, - e -> () -> {}, - (int) ByteSizeUnit.MB.toBytes(1), - logger); - retryRecovery.recoverToTarget(); - retryRecoveryTarget.markAsDone(); - } else { - throw ree; - } - } + recovery.recoverToTarget(); + recoveryTarget.markAsDone(); replica.updateRoutingEntry(ShardRoutingHelper.moveToStarted(replica.routingEntry())); } From b9200cf3e0efadcaceb13937d2c17c1ab1921b1b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 8 Jan 2017 12:57:14 -0500 Subject: [PATCH 03/41] Remove obsolete field from RecoverySourceHandler This commit removes a field that was left behind in a previous refactoring that rendered the field obsolete. --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 0c545d185352c..67925f0e12916 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -74,8 +74,6 @@ */ public class RecoverySourceHandler { - public static final String SEQUENCE_NUMBER_BASED_RECOVERY_FAILED = "sequence_number_based_recovery_failed"; - protected final Logger logger; // Shard that is going to be recovered (the "source") private final IndexShard shard; From 91e1ff07d1240917edecc8998df911860768647f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 8 Jan 2017 14:25:15 -0500 Subject: [PATCH 04/41] Handle translog missing while preparing recovery If a file-based recovery completes phase one successfully, but a network partition happens before the translog is opened, during the retry loop the recovery target will proceed to attempt a sequence-number-based recovery as the index files are present. However, as the translog was never opened it will be missing on disk leading to a no such file exception while preparing for a sequence-number-based recovery. We should not let this fail the recovery, but instead proceed to attempt another file-based recovery. --- .../recovery/PeerRecoveryTargetService.java | 37 ++++++++++++++----- 1 file changed, 28 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 9adb2a74cdd69..1e3016c198f5b 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -41,6 +41,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; @@ -338,16 +339,20 @@ private Optional getStartRecoveryRequest(final RecoveryTar final long startingSeqNo; if (metadataSnapshot.get().size() > 0) { startingSeqNo = getStartingSeqNo(recoveryTarget); - logger.trace( - "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", - recoveryTarget.shardId(), - startingSeqNo, - recoveryTarget.sourceNode()); } else { - logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; } + if (startingSeqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { + logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + } else { + logger.trace( + "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", + recoveryTarget.shardId(), + startingSeqNo, + recoveryTarget.sourceNode()); + } + logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); recoveryTarget.indexShard().prepareForIndexRecovery(); @@ -370,9 +375,23 @@ private Optional getStartRecoveryRequest(final RecoveryTar return Optional.of(request); } - public static long getStartingSeqNo(RecoveryTarget recoveryTarget) throws IOException { - final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.indexShard().shardPath().resolveTranslog()); - return recoveryTarget.store().loadSeqNoStats(globalCheckpoint).getLocalCheckpoint() + 1; + /** + * Get the starting sequence number for a sequence-number-based request. + * + * @param recoveryTarget the target of the recovery + * @return the starting sequence number or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} if obtaining the starting sequence number + * failed + */ + public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { + try { + final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.indexShard().shardPath().resolveTranslog()); + return recoveryTarget.store().loadSeqNoStats(globalCheckpoint).getLocalCheckpoint() + 1; + } catch (final IOException e) { + // this can happen, for example, if a phase one of the recovery completed successfully, a network partition happens before the + // translog on the recovery target is opened, the recovery enters a retry loop seeing now that the index files are on disk to + // proceeds to attempt a sequence-number-based recovery + return SequenceNumbersService.UNASSIGNED_SEQ_NO; + } } public interface RecoveryListener { From 1c14260d5212e98c936344df57abcc833cf2b93b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 8 Jan 2017 16:36:27 -0500 Subject: [PATCH 05/41] Skip adding operations without sequence number A version conflict exception can happen during recovery. If this operation is from an old primary, a sequence number will have not been assigned to the operation. In this case, we should skip adding a no-op to the translog. --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index e92da51d2d333..b247b801d4584 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -671,7 +671,7 @@ private IndexResult innerIndex(Index index) throws IOException { if (checkVersionConflictResult.isPresent()) { indexResult = checkVersionConflictResult.get(); // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication - if (indexResult.hasFailure()) { + if (indexResult.hasFailure() || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { location = null; } else { final Translog.NoOp operation = new Translog.NoOp(seqNo, index.primaryTerm(), "version conflict during recovery"); @@ -809,7 +809,7 @@ private DeleteResult innerDelete(Delete delete) throws IOException { if (result.isPresent()) { deleteResult = result.get(); // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication - if (deleteResult.hasFailure()) { + if (deleteResult.hasFailure() || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { location = null; } else { final Translog.NoOp operation = new Translog.NoOp(seqNo, delete.primaryTerm(), "version conflict during recovery"); From 8b0e5014b1e29976f9dd5acf11422b3d7873ef2f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 15 Jan 2017 15:33:12 -0500 Subject: [PATCH 06/41] Revert adding no-ops on version confict in replica This commit reverts adding no-ops to the translog when a version conflict exception arises on a replica. Instead, we will treat these as normal operations on a replica, but this will happen in another commit. --- .../index/engine/InternalEngine.java | 30 ++++++------------- 1 file changed, 9 insertions(+), 21 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index b247b801d4584..a59cfb712f906 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -670,20 +670,13 @@ private IndexResult innerIndex(Index index) throws IOException { final IndexResult indexResult; if (checkVersionConflictResult.isPresent()) { indexResult = checkVersionConflictResult.get(); - // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication - if (indexResult.hasFailure() || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { - location = null; - } else { - final Translog.NoOp operation = new Translog.NoOp(seqNo, index.primaryTerm(), "version conflict during recovery"); - location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; - } } else { // no version conflict if (index.origin() == Operation.Origin.PRIMARY) { seqNo = seqNoService().generateSeqNo(); } - /* + /** * Update the document's sequence number and primary term; the sequence number here is derived here from either the sequence * number service if this is on the primary, or the existing document's sequence number if this is on the replica. The * primary term here has already been set, see IndexShard#prepareIndex where the Engine$Index operation is created. @@ -700,11 +693,12 @@ private IndexResult innerIndex(Index index) throws IOException { update(index.uid(), index.docs(), indexWriter); } indexResult = new IndexResult(updatedVersion, seqNo, deleted); + location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + ? translog.add(new Translog.Index(index, indexResult)) + : null; versionMap.putUnderLock(index.uid().bytes(), new VersionValue(updatedVersion)); - final Translog.Index operation = new Translog.Index(index, indexResult); - location = index.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; + indexResult.setTranslogLocation(location); } - indexResult.setTranslogLocation(location); indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); return indexResult; @@ -808,13 +802,6 @@ private DeleteResult innerDelete(Delete delete) throws IOException { final DeleteResult deleteResult; if (result.isPresent()) { deleteResult = result.get(); - // norelease: this is not correct as this does not force an fsync, and we need to handle failures including replication - if (deleteResult.hasFailure() || seqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { - location = null; - } else { - final Translog.NoOp operation = new Translog.NoOp(seqNo, delete.primaryTerm(), "version conflict during recovery"); - location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; - } } else { if (delete.origin() == Operation.Origin.PRIMARY) { seqNo = seqNoService().generateSeqNo(); @@ -822,12 +809,13 @@ private DeleteResult innerDelete(Delete delete) throws IOException { updatedVersion = delete.versionType().updateVersion(currentVersion, expectedVersion); found = deleteIfFound(delete.uid(), currentVersion, deleted, versionValue); deleteResult = new DeleteResult(updatedVersion, seqNo, found); + location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY + ? translog.add(new Translog.Delete(delete, deleteResult)) + : null; versionMap.putUnderLock(delete.uid().bytes(), new DeleteVersionValue(updatedVersion, engineConfig.getThreadPool().estimatedTimeInMillis())); - final Translog.Delete operation = new Translog.Delete(delete, deleteResult); - location = delete.origin() != Operation.Origin.LOCAL_TRANSLOG_RECOVERY ? translog.add(operation) : null; + deleteResult.setTranslogLocation(location); } - deleteResult.setTranslogLocation(location); deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); return deleteResult; From dac513a3fdcb62272f3c0ccbb148c2bb608d2a4b Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 15 Jan 2017 15:36:55 -0500 Subject: [PATCH 07/41] Revert whitespace change in MultiSnapshot.java This commit reverts a whitespace change in MultiSnapshot.java. --- .../java/org/elasticsearch/index/translog/MultiSnapshot.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java index 180b69233a58b..7b1a05e1ac1e2 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java +++ b/core/src/main/java/org/elasticsearch/index/translog/MultiSnapshot.java @@ -57,5 +57,4 @@ public Translog.Operation next() throws IOException { } return null; } - } From 81a1e1c79bdda961e66cadb7298d1d8e908581bb Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 15 Jan 2017 15:45:05 -0500 Subject: [PATCH 08/41] Bubble up translog I/O exceptions during recovery When reading the translog on the source during peer recovery, if an I/O exception occurs it is wrapped in an unchecked exception. This is unnecessary as we can just let the I/O exception bubble all the way up. This commit does that. --- .../recovery/RecoverySourceHandler.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 67925f0e12916..dde55698fa7d5 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -189,7 +189,15 @@ public RecoveryResponse recoverToTarget() throws IOException { return response; } - boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) { + /** + * 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(); final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); if (startingSeqNo <= endingSeqNo) { @@ -205,7 +213,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl final LocalCheckpointTracker tracker = new LocalCheckpointTracker(shard.indexSettings(), startingSeqNo, startingSeqNo - 1); final Translog.Snapshot snapshot = translogView.snapshot(); Translog.Operation operation; - while ((operation = getNextOperationFromSnapshot(snapshot)) != null) { + while ((operation = snapshot.next()) != null) { tracker.markSeqNoAsCompleted(operation.seqNo()); } return tracker.getCheckpoint() >= endingSeqNo; @@ -398,7 +406,7 @@ void prepareTargetForTranslog(final int totalTranslogOps, final long maxUnsafeAu * * @param snapshot a snapshot of the translog */ - void phase2(final Translog.Snapshot snapshot) { + void phase2(final Translog.Snapshot snapshot) throws IOException { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } @@ -463,8 +471,9 @@ public void finalizeRecovery() { * * @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 Translog.Snapshot snapshot) throws IOException { int ops = 0; long size = 0; int totalOperations = 0; @@ -476,7 +485,7 @@ protected int sendSnapshot(final Translog.Snapshot snapshot) { // send operations in batches Translog.Operation operation; - while ((operation = getNextOperationFromSnapshot(snapshot)) != null) { + while ((operation = snapshot.next()) != null) { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } @@ -522,14 +531,6 @@ protected int sendSnapshot(final Translog.Snapshot snapshot) { return totalOperations; } - private Translog.Operation getNextOperationFromSnapshot(final Translog.Snapshot snapshot) { - try { - return snapshot.next(); - } catch (final IOException ex) { - throw new ElasticsearchException("failed to get next operation from translog", ex); - } - } - /** * Cancels the recovery and interrupts all eligible threads. */ From 8960522775d1415f4828dee34d934ef9c1f6330c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sun, 15 Jan 2017 22:08:37 -0500 Subject: [PATCH 09/41] Add assertion on number of recovered ops This commit adds an assertion to the number of ops recovered from the translog in the recovery of disconnected replica test. --- .../indices/recovery/RecoverySourceHandler.java | 13 ++++++++----- .../recovery/SharedFSRecoverySourceHandler.java | 2 +- .../replication/RecoveryDuringReplicationTests.java | 10 +++++++--- 3 files changed, 16 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index dde55698fa7d5..9ba424ec6b131 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -218,7 +218,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl } return tracker.getCheckpoint() >= endingSeqNo; } - return false; + return true; } /** @@ -417,7 +417,7 @@ void phase2(final Translog.Snapshot snapshot) throws IOException { logger.trace("{} recovery [phase2] to {}: sending transaction log operations", request.shardId(), request.targetNode()); // send all the snapshot's translog operations to the target - final int totalOperations = sendSnapshot(snapshot); + final int totalOperations = sendSnapshot(request.startingSeqNo(), snapshot); stopWatch.stop(); logger.trace("{} recovery [phase2] to {}: took [{}]", request.shardId(), request.targetNode(), stopWatch.totalTime()); @@ -465,15 +465,17 @@ public void finalizeRecovery() { } /** - * 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. *

* Operations are bulked into a single request depending on an operation count limit or size-in-bytes limit. * - * @param snapshot the translog snapshot to replay operations from + * @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) throws IOException { + protected int sendSnapshot(final long startingSeqNo, final Translog.Snapshot snapshot) throws IOException { int ops = 0; long size = 0; int totalOperations = 0; @@ -490,6 +492,7 @@ protected int sendSnapshot(final Translog.Snapshot snapshot) throws IOException throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); + if (operation.seqNo() < startingSeqNo) continue; operations.add(operation); ops++; size += operation.estimateSize(); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java index e8437b614e0b6..802576922f90e 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoverySourceHandler.java @@ -83,7 +83,7 @@ public RecoveryResponse recoverToTarget() throws IOException { } @Override - protected int sendSnapshot(final Translog.Snapshot snapshot) { + protected int sendSnapshot(final long startingSeqNo, final Translog.Snapshot snapshot) { logger.trace("{} skipping recovery of translog snapshot on shared filesystem to: {}", shard.shardId(), request.targetNode()); return 0; } diff --git a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index ee01aee64bd4a..2f565dbf56a9f 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -36,6 +36,7 @@ import java.util.concurrent.Future; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestCase { @@ -68,6 +69,7 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { shards.flush(); shards.getPrimary().updateGlobalCheckpointOnPrimary(); final IndexShard originalReplica = shards.getReplicas().get(0); + long replicaCommittedLocalCheckpoint = docs - 1; boolean replicaHasDocsSinceLastFlushedCheckpoint = false; for (int i = 0; i < randomInt(2); i++) { final int indexedDocs = shards.indexDocs(randomInt(5)); @@ -79,14 +81,13 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { final boolean flush = randomBoolean(); if (flush) { originalReplica.flush(new FlushRequest()); + replicaHasDocsSinceLastFlushedCheckpoint = false; + replicaCommittedLocalCheckpoint = docs - 1; } final boolean sync = randomBoolean(); if (sync) { shards.getPrimary().updateGlobalCheckpointOnPrimary(); - if (flush) { - replicaHasDocsSinceLastFlushedCheckpoint = false; - } } } @@ -112,6 +113,9 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty())); } else { assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty()); + assertThat( + recoveredReplica.recoveryState().getTranslog().recoveredOperations(), + equalTo(Math.toIntExact(docs - (replicaCommittedLocalCheckpoint + 1)))); } docs += shards.indexDocs(randomInt(5)); From b6e6cc3b7d3c7fdff0e862e984017d705f3b5230 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 16:04:30 -0500 Subject: [PATCH 10/41] Restore logger specific to GlobalCheckpointTracker This commit restores having a logger that is specific to GlobalCheckpointTracker so that its logging levels can be configured separately from SequenceNumbersService. --- .../index/seqno/GlobalCheckpointTracker.java | 12 ++++++------ .../index/seqno/SequenceNumbersService.java | 2 +- .../index/seqno/GlobalCheckpointTests.java | 6 +++++- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java index 514fc985ce007..fd9b2f4687fd6 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -24,6 +24,8 @@ import com.carrotsearch.hppc.cursors.ObjectLongCursor; import org.apache.logging.log4j.Logger; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.ShardId; import java.util.HashSet; import java.util.Locale; @@ -40,7 +42,7 @@ *

* The global checkpoint is maintained by the primary shard and is replicated to all the replicas (via {@link GlobalCheckpointSyncAction}). */ -public class GlobalCheckpointTracker { +public class GlobalCheckpointTracker extends AbstractIndexShardComponent { /* * This map holds the last known local checkpoint for every active shard and initializing shard copies that has been brought up to speed @@ -62,22 +64,20 @@ public class GlobalCheckpointTracker { */ private long globalCheckpoint; - private final Logger logger; - /** * Initialize the global checkpoint service. The specified global checkpoint should be set to the last known global checkpoint, or * {@link SequenceNumbersService#UNASSIGNED_SEQ_NO}. * + * @param shardId the shard ID * @param indexSettings the index settings * @param globalCheckpoint the last known global checkpoint for this shard, or {@link SequenceNumbersService#UNASSIGNED_SEQ_NO} - * @param logger a component logger */ - GlobalCheckpointTracker(final IndexSettings indexSettings, final long globalCheckpoint, final Logger logger) { + GlobalCheckpointTracker(final ShardId shardId, final IndexSettings indexSettings, final long globalCheckpoint) { + super(shardId, indexSettings); assert globalCheckpoint >= UNASSIGNED_SEQ_NO : "illegal initial global checkpoint: " + globalCheckpoint; inSyncLocalCheckpoints = new ObjectLongHashMap<>(1 + indexSettings.getNumberOfReplicas()); assignedAllocationIds = new HashSet<>(1 + indexSettings.getNumberOfReplicas()); this.globalCheckpoint = globalCheckpoint; - this.logger = logger; } /** diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java index cc43363e13b2d..a0ba1e850aad1 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java @@ -63,7 +63,7 @@ public SequenceNumbersService( final long globalCheckpoint) { super(shardId, indexSettings); localCheckpointTracker = new LocalCheckpointTracker(indexSettings, maxSeqNo, localCheckpoint); - globalCheckpointTracker = new GlobalCheckpointTracker(indexSettings, globalCheckpoint, logger); + globalCheckpointTracker = new GlobalCheckpointTracker(shardId, indexSettings, globalCheckpoint); } /** diff --git a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java index de7d05a56cdab..7d6dd25403b65 100644 --- a/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java +++ b/core/src/test/java/org/elasticsearch/index/seqno/GlobalCheckpointTests.java @@ -20,6 +20,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.junit.Before; @@ -48,7 +49,10 @@ public class GlobalCheckpointTests extends ESTestCase { public void setUp() throws Exception { super.setUp(); tracker = - new GlobalCheckpointTracker(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO, logger); + new GlobalCheckpointTracker( + new ShardId("test", "_na_", 0), + IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), + UNASSIGNED_SEQ_NO); } public void testEmptyShards() { From ab18bde863883549f42cf0f38820f6068bb74045 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 16:09:02 -0500 Subject: [PATCH 11/41] Use readLong/writeLong for serializing seq. no. When serializing sequence numbers in StartRecoveryRequest we use readZLong/writeZLong. This is inconsistent with how these are handled in other places. This commit makes StartRecoveryRequest consistent with the handling in other places. --- .../elasticsearch/indices/recovery/StartRecoveryRequest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index f5306d3e94b65..46a29d654640c 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -113,7 +113,7 @@ public void readFrom(StreamInput in) throws IOException { metadataSnapshot = new Store.MetadataSnapshot(in); primaryRelocation = in.readBoolean(); if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { - startingSeqNo = in.readZLong(); + startingSeqNo = in.readLong(); } else { startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; } @@ -129,7 +129,7 @@ public void writeTo(StreamOutput out) throws IOException { metadataSnapshot.writeTo(out); out.writeBoolean(primaryRelocation); if (out.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { - out.writeZLong(startingSeqNo); + out.writeLong(startingSeqNo); } } From cea70f4d4ac2455c714e1fc8b33e307c16aa55c8 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 16:13:04 -0500 Subject: [PATCH 12/41] Remove unneeded assertion disjunction in engine This commit removes an unneeded disjunction in an assertion in the engine. --- core/src/main/java/org/elasticsearch/index/engine/Engine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index 76aee3e46a54c..af2388eedef81 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -361,7 +361,7 @@ public Operation.TYPE getOperationType() { void setTranslogLocation(Translog.Location translogLocation) { if (freeze.get() == null) { - assert failure == null || translogLocation == null: "failure has to be null to set translog location"; + assert failure == null : "failure has to be null to set translog location"; this.translogLocation = translogLocation; } else { throw new IllegalStateException("result is already frozen"); From 34fbb378343f0f94e95e9026dd0b2492dc4fe23d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 16:27:32 -0500 Subject: [PATCH 13/41] Explicitly prepare shard for peer recovery This commit moves preparing a shard for peer recovery out of the method that prepares the recovery request as this was an odd place to change manipulate the state of the shard. --- .../indices/recovery/PeerRecoveryTargetService.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 1e3016c198f5b..5bb26b21acd82 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -184,6 +184,9 @@ private void doRecovery(final long recoveryId) { cancellableThreads = recoveryTarget.cancellableThreads(); timer = recoveryTarget.state().getTimer(); + + logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); + recoveryTarget.indexShard().prepareForIndexRecovery(); } try { @@ -353,9 +356,6 @@ private Optional getStartRecoveryRequest(final RecoveryTar recoveryTarget.sourceNode()); } - logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); - recoveryTarget.indexShard().prepareForIndexRecovery(); - request = new StartRecoveryRequest( recoveryTarget.shardId(), recoveryTarget.sourceNode(), From 1929c03d06ba1b63a2cdd521a8c527e4c1d2abe0 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 17:40:18 -0500 Subject: [PATCH 14/41] Load sequence number statistics from the store This commit loads sequence number statistics from the store directly when starting the engine instead of indirectly via the index writer. --- .../index/engine/InternalEngine.java | 33 ++----------------- 1 file changed, 3 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index f945447cf60a6..f8d2437d082c6 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -158,11 +158,12 @@ public InternalEngine(EngineConfig engineConfig) throws EngineException { switch (openMode) { case OPEN_INDEX_AND_TRANSLOG: writer = createWriter(false); - seqNoStats = loadSeqNoStatsFromLuceneAndTranslog(engineConfig.getTranslogConfig(), writer); + final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); + seqNoStats = store.loadSeqNoStats(globalCheckpoint); break; case OPEN_INDEX_CREATE_TRANSLOG: writer = createWriter(false); - seqNoStats = loadSeqNoStatsFromLucene(SequenceNumbersService.UNASSIGNED_SEQ_NO, writer); + seqNoStats = store.loadSeqNoStats(SequenceNumbersService.UNASSIGNED_SEQ_NO); break; case CREATE_INDEX_AND_TRANSLOG: writer = createWriter(true); @@ -352,34 +353,6 @@ private Translog.TranslogGeneration loadTranslogIdFromCommit(IndexWriter writer) return null; } - /** - * Reads the sequence number stats from the Lucene commit point (maximum sequence number and local checkpoint) and the translog - * checkpoint (global checkpoint). - * - * @param translogConfig the translog config (for the global checkpoint) - * @param indexWriter the index writer (for the Lucene commit point) - * @return the sequence number stats - * @throws IOException if an I/O exception occurred reading the Lucene commit point or the translog checkpoint - */ - private static SeqNoStats loadSeqNoStatsFromLuceneAndTranslog( - final TranslogConfig translogConfig, - final IndexWriter indexWriter) throws IOException { - final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()); - return loadSeqNoStatsFromLucene(globalCheckpoint, indexWriter); - } - - /** - * Reads the sequence number stats from the Lucene commit point (maximum sequence number and local checkpoint) and uses the - * specified global checkpoint. - * - * @param globalCheckpoint the global checkpoint to use - * @param indexWriter the index writer (for the Lucene commit point) - * @return the sequence number stats - */ - private static SeqNoStats loadSeqNoStatsFromLucene(final long globalCheckpoint, final IndexWriter indexWriter) { - return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, indexWriter.getLiveCommitData()); - } - private SearcherManager createSearcherManager() throws EngineException { boolean success = false; SearcherManager searcherManager = null; From c0169c29075a6f315c3467863bce987fbc3fd9cf Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 16 Jan 2017 17:50:58 -0500 Subject: [PATCH 15/41] Rename replication test case method This commit renames a method that is used in replication test cases to add a replica using existing shard data. --- .../replication/ESIndexLevelReplicationTestCase.java | 11 +++++------ .../replication/RecoveryDuringReplicationTests.java | 6 +++++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java index 60a7d6e50def8..bf1d020756558 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java @@ -50,6 +50,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; @@ -185,16 +186,14 @@ public synchronized IndexShard addReplica() throws IOException { return replica; } - public synchronized IndexShard addReplica(IndexShard replica) throws IOException { - replica.close("add", false); - IOUtils.close(replica.store()); - ShardRouting shardRouting = TestShardRouting.newShardRouting( + public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardPath, final String nodeId) throws IOException { + final ShardRouting shardRouting = TestShardRouting.newShardRouting( shardId, - replica.routingEntry().currentNodeId(), + nodeId, false, ShardRoutingState.INITIALIZING, RecoverySource.PeerRecoverySource.INSTANCE); - IndexShard newReplica = newShard(shardRouting, replica.shardPath(), indexMetaData, null, replicaGlobalCheckpointSyncer); + final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null, replicaGlobalCheckpointSyncer); replicas.add(newReplica); updateAllocationIDsOnPrimary(); return newReplica; diff --git a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 2f565dbf56a9f..ea603e100c7e0 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.replication; import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.index.shard.IndexShard; @@ -106,7 +107,10 @@ public void testRecoveryOfDisconnectedReplica() throws Exception { shards.flush(); } - final IndexShard recoveredReplica = shards.addReplica(originalReplica); + originalReplica.close("disconnected", false); + IOUtils.close(originalReplica.store()); + final IndexShard recoveredReplica = + shards.addReplicaWithExistingPath(originalReplica.shardPath(), originalReplica.routingEntry().currentNodeId()); shards.recoverReplica(recoveredReplica); if (flushPrimary && replicaHasDocsSinceLastFlushedCheckpoint) { // replica has something to catch up with, but since we flushed the primary, we should fall back to full recovery From 999ca91ef5ec3d70f636e0e1275ee3c73bf25549 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 17 Jan 2017 14:39:21 -0500 Subject: [PATCH 16/41] Iteration --- .../indices/recovery/RecoverySourceHandler.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 132141af72a5a..9c073a21a10f1 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -200,14 +200,17 @@ public RecoveryResponse recoverToTarget() throws IOException { boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View translogView) throws IOException { final long startingSeqNo = request.startingSeqNo(); final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - if (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 + logger.trace("starting: [{}], ending: [{}}", startingSeqNo, endingSeqNo); + 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); @@ -218,7 +221,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl } return tracker.getCheckpoint() >= endingSeqNo; } - return true; + return false; } /** @@ -492,7 +495,8 @@ protected int sendSnapshot(final long startingSeqNo, final Translog.Snapshot sna throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); - if (operation.seqNo() < startingSeqNo) continue; + // 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++; size += operation.estimateSize(); From 7281b75beed4d45a29fac03fd7952eba793d008f Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 17 Jan 2017 20:57:09 -0500 Subject: [PATCH 17/41] Add in-flight ops recovery test This commit adds a test for performing a sequence-number-based recovery with some ops that are in-flight when the recovery starts. --- .../indices/recovery/EvilRecoveryIT.java | 229 ++++++++++++++++++ 1 file changed, 229 insertions(+) create mode 100644 core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java new file mode 100644 index 0000000000000..eba9fcafccf00 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java @@ -0,0 +1,229 @@ +package org.elasticsearch.indices.recovery; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.analysis.Tokenizer; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.analysis.AnalyzerProvider; +import org.elasticsearch.index.analysis.AnalyzerScope; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.analysis.AnalysisModule; +import org.elasticsearch.plugins.AnalysisPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.InternalTestCluster; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicReference; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.hamcrest.Matchers.equalTo; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) +public class EvilRecoveryIT extends ESIntegTestCase { + + private static AtomicReference indexLatch = new AtomicReference<>(); + private static AtomicReference waitForOpsToCompleteLatch = new AtomicReference<>(); + + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(LatchAnalysisPlugin.class); + } + + public static class LatchAnalysisPlugin extends Plugin implements AnalysisPlugin { + + @Override + public Map>> getAnalyzers() { + return Collections.singletonMap("latch_analyzer", (a, b, c, d) -> new LatchAnalyzerProvider()); + } + + } + + static class LatchAnalyzerProvider implements AnalyzerProvider { + + @Override + public String name() { + return "latch_analyzer"; + } + + @Override + public AnalyzerScope scope() { + return AnalyzerScope.INDICES; + } + + @Override + public LatchAnalyzer get() { + return new LatchAnalyzer(); + } + + } + + static class LatchAnalyzer extends Analyzer { + + @Override + protected TokenStreamComponents createComponents(final String fieldName) { + return new TokenStreamComponents(new LatchTokenizer()); + } + + } + + static class LatchTokenizer extends Tokenizer { + + @Override + public final boolean incrementToken() throws IOException { + try { + if (indexLatch.get() != null) { + // latch that all exected operations are in the engine + indexLatch.get().countDown(); + } + + if (waitForOpsToCompleteLatch.get() != null) { + // latch that waits for the replica to restart and allows recovery to proceed + waitForOpsToCompleteLatch.get().await(); + } + + } catch (final InterruptedException e) { + throw new RuntimeException(e); + } + return false; + } + + } + + @Override + protected Settings nodeSettings(final int nodeOrdinal) { + final Settings nodeSettings = super.nodeSettings(nodeOrdinal); + final int processors = randomIntBetween(1, 4); + /* + * We have to do this to ensure that there are sufficiently many threads to accept the indexing requests, otherwise operations will + * instead be queued and never trip the latch that all operations are inside the engine. + */ + return Settings.builder().put(nodeSettings).put("processors", processors).put("thread_pool.bulk.size", 1 + processors).build(); + } + + /* + * This tests that sequence-number-based recoveries wait for in-flight operations to complete. The trick here is simple. We latch some + * in-flight operations inside the engine after sequence numbers are assigned. While these operations are latched, we restart a replica. + * Sequence-number-based recovery on this replica has to wait until these in-flight operations complete to proceed. We verify at the end + * of recovery that a file-based recovery was not completed, and that the expected number of operations was replayed via the translog. + */ + public void testRecoveryWaitsForOps() throws Exception { + final int docs = randomIntBetween(1, 64); + final int numberOfProcessors = EsExecutors.numberOfProcessors(nodeSettings(0)); + final int latchedDocs = randomIntBetween(1, 1 + numberOfProcessors); + + try { + internalCluster().startMasterOnlyNode(); + final String primaryNode = internalCluster().startDataOnlyNode(nodeSettings(0)); + + // prepare mapping that uses our latch analyzer + final XContentBuilder mapping = jsonBuilder(); + mapping.startObject(); + { + mapping.startObject("type"); + { + mapping.startObject("properties"); + { + mapping.startObject("foo"); + { + mapping.field("type", "text"); + mapping.field("analyzer", "latch_analyzer"); + mapping.endObject(); + } + mapping.endObject(); + } + mapping.endObject(); + } + mapping.endObject(); + } + + // create the index with our mapping + client() + .admin() + .indices() + .prepareCreate("index") + .addMapping("type", mapping) + .setSettings(Settings.builder().put("number_of_shards", 1)) + .get(); + + // start the replica node; we do this after creating the index so we can control which node is holds the primary shard + final String replicaNode = internalCluster().startDataOnlyNode(nodeSettings(1)); + ensureGreen(); + + // index some documents so that the replica will attempt a sequence-number-based recovery upon restart + for (int foo = 0; foo < docs; foo++) { + index(randomFrom(primaryNode, replicaNode), foo); + } + + if (randomBoolean()) { + client().admin().indices().flush(new FlushRequest()).get(); + } + + // start some in-flight operations that will get latched in the engine + final List threads = new ArrayList<>(); + indexLatch.set(new CountDownLatch(latchedDocs)); + waitForOpsToCompleteLatch.set(new CountDownLatch(1)); + for (int i = docs; i < docs + latchedDocs; i++) { + final int foo = i; + // we have to index through the primary since we are going to restart the replica + final Thread thread = new Thread(() -> index(primaryNode, foo)); + threads.add(thread); + thread.start(); + } + + // latch until all operations are inside the engine + indexLatch.get().await(); + + internalCluster().restartNode(replicaNode, new InternalTestCluster.RestartCallback()); + + final Index index = resolveIndex("index"); + + // wait until recovery starts + assertBusy(() -> { + final IndicesService primaryService = internalCluster().getInstance(IndicesService.class, primaryNode); + assertThat(primaryService.indexServiceSafe(index).getShard(0).recoveryStats().currentAsSource(), equalTo(1)); + final IndicesService replicaService = internalCluster().getInstance(IndicesService.class, replicaNode); + assertThat(replicaService.indexServiceSafe(index).getShard(0).recoveryStats().currentAsTarget(), equalTo(1)); + } + ); + + // unlatch the operations that are latched inside the engine + waitForOpsToCompleteLatch.get().countDown(); + + for (final Thread thread : threads) { + thread.join(); + } + + // recovery should complete successfully + ensureGreen(); + + // verify that a sequence-number-based recovery was completed + final org.elasticsearch.action.admin.indices.recovery.RecoveryResponse response = + client().admin().indices().prepareRecoveries("index").get(); + final List states = response.shardRecoveryStates().get("index"); + for (final RecoveryState state : states) { + if (state.getTargetNode().getName().equals(replicaNode)) { + assertThat(state.getTranslog().recoveredOperations(), equalTo(latchedDocs)); + assertThat(state.getIndex().recoveredFilesPercent(), equalTo(0f)); + } + } + } finally { + internalCluster().close(); + } + + } + + private void index(final String node, final int foo) { + client(node).prepareIndex("index", "type").setSource("{\"foo\":\"" + Integer.toString(foo) + "\"}").get(); + } + +} From 320301c36d79c1b86f9edab3ff83292b8269d059 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 18 Jan 2017 16:54:05 -0500 Subject: [PATCH 18/41] Add assertion on recoveries targeting old replicas This commit adds an assertion in the serialization of start recovery request that guards against a recovery from a new primary targeting an old replica. --- .../elasticsearch/indices/recovery/StartRecoveryRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index 46a29d654640c..430bfbe292579 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -115,7 +115,7 @@ public void readFrom(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { startingSeqNo = in.readLong(); } else { - startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + assert false : "a start recovery request should not be received from an old replica but was [" + targetNode.getVersion() + "]"; } } From 50d31911ec8cd2b0576814ec537d89ad01b67390 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 18 Jan 2017 17:39:01 -0500 Subject: [PATCH 19/41] Add defensive assertion on starting seq. no. This commit adds a defensive assertion when checking is the translog on the recovery source is ready for a sequence-number-based recovery. --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 9c073a21a10f1..b90d68c85019b 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -199,6 +199,7 @@ public RecoveryResponse recoverToTarget() throws IOException { */ 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: [{}}", startingSeqNo, endingSeqNo); if (startingSeqNo - 1 <= endingSeqNo) { From 2596dcd547af52f5da8af0e63d0064b448c7538c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 18 Jan 2017 17:47:06 -0500 Subject: [PATCH 20/41] Store#loadSeqNoStats iteration This commit iterates on Store#loadSeqStats, adding Javadocs and improving the loading of the commit data by ensuring that we load the latest commit point and obtain the user data directly. --- .../java/org/elasticsearch/index/store/Store.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java index 7d569d947c3b2..ea4097c13628c 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -210,9 +210,18 @@ private static SegmentInfos readSegmentsInfo(IndexCommit commit, Directory direc } + /** + * Loads the local checkpoint and the maximum sequence number from the latest Lucene commit point and returns the triplet of local and + * global checkpoints, and maximum sequence number as an instance of {@link SeqNoStats}. The global checkpoint must be provided + * externally as it is not stored in the commit point. + * + * @param globalCheckpoint the provided global checkpoint + * @return an instance of {@link SeqNoStats} populated with the local and global checkpoints, and the maximum sequence number + * @throws IOException if an I/O exception occurred reading the latest Lucene commit point from disk + */ public SeqNoStats loadSeqNoStats(final long globalCheckpoint) throws IOException { - final IndexCommit indexCommit = DirectoryReader.listCommits(directory).get(0); - return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, indexCommit.getUserData().entrySet()); + final Map userData = SegmentInfos.readLatestCommit(directory).getUserData(); + return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, userData.entrySet()); } final void ensureOpen() { From cc2002c00f8bd4240ef8f83db788ded2079846f3 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 19 Jan 2017 13:12:38 -0500 Subject: [PATCH 21/41] Simplify preparing start recovery request --- .../recovery/PeerRecoveryTargetService.java | 122 ++++++++---------- 1 file changed, 51 insertions(+), 71 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 5bb26b21acd82..c923a0ed9cecf 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -41,7 +41,6 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; -import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; @@ -62,11 +61,9 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; @@ -172,21 +169,24 @@ private void doRecovery(final long recoveryId) { try (RecoveryRef recoveryRef = onGoingRecoveries.getRecovery(recoveryId)) { if (recoveryRef == null) { - logger.trace("not running recovery with id [{}] - can't find it (probably finished)", recoveryId); + logger.trace("not running recovery with id [{}] - can not find it (probably finished)", recoveryId); return; } final RecoveryTarget recoveryTarget = recoveryRef.target(); - assert recoveryTarget.sourceNode() != null : "can't do a recovery without a source node"; - - final Optional maybeRequest = getStartRecoveryRequest(recoveryTarget); - if (!maybeRequest.isPresent()) return; - else request = maybeRequest.get(); - cancellableThreads = recoveryTarget.cancellableThreads(); timer = recoveryTarget.state().getTimer(); - - logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); - recoveryTarget.indexShard().prepareForIndexRecovery(); + try { + assert recoveryTarget.sourceNode() != null : "can not do a recovery without a source node"; + request = getStartRecoveryRequest(recoveryTarget); + logger.trace("{} preparing shard for peer recovery", recoveryTarget.shardId()); + recoveryTarget.indexShard().prepareForIndexRecovery(); + } catch (final Exception e) { + // this will be logged as warning later on... + logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e); + onGoingRecoveries.failRecovery(recoveryId, + new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e), true); + return; + } } try { @@ -289,90 +289,70 @@ public RecoveryResponse newInstance() { } } - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private static Optional EMPTY_METADATA_SNAPSHOT = Optional.of(Store.MetadataSnapshot.EMPTY); - /** - * Obtains a snapshot of the store metadata for the recovery target, or an empty {@link Optional} if obtaining the store metadata - * failed. + * Obtains a snapshot of the store metadata for the recovery target. * * @param recoveryTarget the target of the recovery - * @return a snapshot of the store metdata, or an empty {@link Optional} + * @return a snapshot of the store metdata */ - private Optional getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) { + private Store.MetadataSnapshot getStoreMetadataSnapshot(final RecoveryTarget recoveryTarget) { try { if (recoveryTarget.indexShard().indexSettings().isOnSharedFilesystem()) { // we are not going to copy any files, so don't bother listing files, potentially running into concurrency issues with the // primary changing files underneath us - return EMPTY_METADATA_SNAPSHOT; + return Store.MetadataSnapshot.EMPTY; } else { - return Optional.of(recoveryTarget.indexShard().snapshotStoreMetadata()); + return recoveryTarget.indexShard().snapshotStoreMetadata(); } - } catch (org.apache.lucene.index.IndexNotFoundException e) { + } catch (final org.apache.lucene.index.IndexNotFoundException e) { // happens on an empty folder. no need to log - logger.trace("{} shard folder empty, recover all files", recoveryTarget); - return EMPTY_METADATA_SNAPSHOT; + logger.trace("{} shard folder empty, recovering all files", recoveryTarget); + return Store.MetadataSnapshot.EMPTY; } catch (final IOException e) { - logger.warn("error while listing local files, recover as if there are none", e); - return EMPTY_METADATA_SNAPSHOT; - } catch (final Exception e) { - // this will be logged as warning later on... - logger.trace("unexpected error while listing local files, failing recovery", e); - onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), - new RecoveryFailedException(recoveryTarget.state(), "failed to list local files", e), true); - return Optional.empty(); + logger.warn("error while listing local files, recovering as if there are none", e); + return Store.MetadataSnapshot.EMPTY; } } /** - * Prepare the start recovery request, returning an empty {@link Optional} instance if preparing the start request failed. + * Prepare the start recovery request. * * @param recoveryTarget the target of the recovery - * @return a start recovery request, or an empty {@link Optional} + * @return a start recovery request */ - private Optional getStartRecoveryRequest(final RecoveryTarget recoveryTarget) { + private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recoveryTarget) { final StartRecoveryRequest request; logger.trace("{} collecting local files for [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); - final Optional metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget); - if (!metadataSnapshot.isPresent()) return Optional.empty(); - logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.get().size()); + final Store.MetadataSnapshot metadataSnapshot = getStoreMetadataSnapshot(recoveryTarget); + logger.trace("{} local file count [{}]", recoveryTarget.shardId(), metadataSnapshot.size()); - try { - final long startingSeqNo; - if (metadataSnapshot.get().size() > 0) { - startingSeqNo = getStartingSeqNo(recoveryTarget); - } else { - startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; - } - - if (startingSeqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { - logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); - } else { - logger.trace( - "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", - recoveryTarget.shardId(), - startingSeqNo, - recoveryTarget.sourceNode()); - } + final long startingSeqNo; + if (metadataSnapshot.size() > 0) { + startingSeqNo = getStartingSeqNo(recoveryTarget); + } else { + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; + } - request = new StartRecoveryRequest( + if (startingSeqNo == SequenceNumbersService.UNASSIGNED_SEQ_NO) { + logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); + } else { + logger.trace( + "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", recoveryTarget.shardId(), - recoveryTarget.sourceNode(), - clusterService.localNode(), - metadataSnapshot.get(), - recoveryTarget.state().getPrimary(), - recoveryTarget.recoveryId(), - startingSeqNo); - - } catch (final Exception e) { - // this will be logged as warning later on... - logger.trace("unexpected error while preparing shard for peer recovery, failing recovery", e); - onGoingRecoveries.failRecovery(recoveryTarget.recoveryId(), - new RecoveryFailedException(recoveryTarget.state(), "failed to prepare shard for recovery", e), true); - return Optional.empty(); + startingSeqNo, + recoveryTarget.sourceNode()); } - return Optional.of(request); + + request = new StartRecoveryRequest( + recoveryTarget.shardId(), + recoveryTarget.sourceNode(), + clusterService.localNode(), + metadataSnapshot, + recoveryTarget.state().getPrimary(), + recoveryTarget.recoveryId(), + startingSeqNo); + return request; } /** From 1e59f84799232018b2e4257c55771e1248d7a113 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 19 Jan 2017 13:25:55 -0500 Subject: [PATCH 22/41] Only mark assigned sequence numbers as completed --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index b90d68c85019b..2a8e46f749189 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -218,7 +218,9 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl final Translog.Snapshot snapshot = translogView.snapshot(); Translog.Operation operation; while ((operation = snapshot.next()) != null) { - tracker.markSeqNoAsCompleted(operation.seqNo()); + if (operation.seqNo() != SequenceNumbersService.UNASSIGNED_SEQ_NO) { + tracker.markSeqNoAsCompleted(operation.seqNo()); + } } return tracker.getCheckpoint() >= endingSeqNo; } From cc30114c9531b956f004f69a56f45ec6c8b40310 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 19 Jan 2017 13:54:34 -0500 Subject: [PATCH 23/41] Iteration --- .../indices/recovery/PeerRecoveryTargetService.java | 2 +- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index c923a0ed9cecf..bd78cc915e35e 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -368,7 +368,7 @@ public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { return recoveryTarget.store().loadSeqNoStats(globalCheckpoint).getLocalCheckpoint() + 1; } catch (final IOException e) { // this can happen, for example, if a phase one of the recovery completed successfully, a network partition happens before the - // translog on the recovery target is opened, the recovery enters a retry loop seeing now that the index files are on disk to + // translog on the recovery target is opened, the recovery enters a retry loop seeing now that the index files are on disk and // proceeds to attempt a sequence-number-based recovery return SequenceNumbersService.UNASSIGNED_SEQ_NO; } diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 2a8e46f749189..621705ccad430 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -201,7 +201,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl final long startingSeqNo = request.startingSeqNo(); assert startingSeqNo >= 0; final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - logger.trace("starting: [{}], ending: [{}}", startingSeqNo, endingSeqNo); + logger.trace("{} starting: [{}], ending: [{}}", shardId, startingSeqNo, endingSeqNo); if (startingSeqNo - 1 <= endingSeqNo) { logger.trace( "{} waiting for all operations in the range [{}, {}] to complete", From 781f2761ef2e0f7956340999121fa7aa6313666c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 19 Jan 2017 18:01:04 -0500 Subject: [PATCH 24/41] Add comment regarding starting seq. no. --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 621705ccad430..5fff8b503d26f 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -202,6 +202,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl assert startingSeqNo >= 0; final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); logger.trace("{} starting: [{}], ending: [{}}", 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", From 3a70bd7b3f5aac2fea7252d8f54cd1d1cf1a4c8d Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 19 Jan 2017 22:35:19 -0500 Subject: [PATCH 25/41] Add missing license header --- .../indices/recovery/EvilRecoveryIT.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java index eba9fcafccf00..7250f119887d6 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java @@ -1,3 +1,22 @@ +/* + * 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.indices.recovery; import org.apache.lucene.analysis.Analyzer; From 88fc8011921e108ff6b4eff5c0ea6a5d6866d77c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 20 Jan 2017 05:52:46 -0500 Subject: [PATCH 26/41] Fix start recovery request serialization test --- .../indices/recovery/StartRecoveryRequestTests.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java index e91eca4ad6d1d..a7f1604361a10 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java @@ -35,12 +35,13 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.test.VersionUtils.randomVersion; +import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.hamcrest.Matchers.equalTo; public class StartRecoveryRequestTests extends ESTestCase { public void testSerialization() throws Exception { - final Version targetNodeVersion = randomVersion(random()); + final Version targetNodeVersion = randomVersionBetween(random(), Version.V_6_0_0_alpha1_UNRELEASED, Version.CURRENT); final StartRecoveryRequest outRequest = new StartRecoveryRequest( new ShardId("test", "_na_", 0), new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), @@ -67,11 +68,7 @@ public void testSerialization() throws Exception { assertThat(outRequest.metadataSnapshot().asMap(), equalTo(inRequest.metadataSnapshot().asMap())); assertThat(outRequest.isPrimaryRelocation(), equalTo(inRequest.isPrimaryRelocation())); assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); - if (targetNodeVersion.onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { - assertThat(outRequest.startingSeqNo(), equalTo(inRequest.startingSeqNo())); - } else { - assertThat(SequenceNumbersService.UNASSIGNED_SEQ_NO, equalTo(inRequest.startingSeqNo())); - } + assertThat(outRequest.startingSeqNo(), equalTo(inRequest.startingSeqNo())); } } From 6a0b70e60b45298f380149ac415a27ea0dac7e42 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Fri, 20 Jan 2017 10:25:47 -0500 Subject: [PATCH 27/41] Revert start recovery request assertion --- .../indices/recovery/StartRecoveryRequest.java | 2 +- .../indices/recovery/StartRecoveryRequestTests.java | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java index 430bfbe292579..46a29d654640c 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/StartRecoveryRequest.java @@ -115,7 +115,7 @@ public void readFrom(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { startingSeqNo = in.readLong(); } else { - assert false : "a start recovery request should not be received from an old replica but was [" + targetNode.getVersion() + "]"; + startingSeqNo = SequenceNumbersService.UNASSIGNED_SEQ_NO; } } diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java index a7f1604361a10..c176b7c3d291c 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/StartRecoveryRequestTests.java @@ -41,7 +41,7 @@ public class StartRecoveryRequestTests extends ESTestCase { public void testSerialization() throws Exception { - final Version targetNodeVersion = randomVersionBetween(random(), Version.V_6_0_0_alpha1_UNRELEASED, Version.CURRENT); + final Version targetNodeVersion = randomVersion(random()); final StartRecoveryRequest outRequest = new StartRecoveryRequest( new ShardId("test", "_na_", 0), new DiscoveryNode("a", buildNewFakeTransportAddress(), emptyMap(), emptySet(), targetNodeVersion), @@ -68,7 +68,11 @@ public void testSerialization() throws Exception { assertThat(outRequest.metadataSnapshot().asMap(), equalTo(inRequest.metadataSnapshot().asMap())); assertThat(outRequest.isPrimaryRelocation(), equalTo(inRequest.isPrimaryRelocation())); assertThat(outRequest.recoveryId(), equalTo(inRequest.recoveryId())); - assertThat(outRequest.startingSeqNo(), equalTo(inRequest.startingSeqNo())); + if (targetNodeVersion.onOrAfter(Version.V_6_0_0_alpha1_UNRELEASED)) { + assertThat(outRequest.startingSeqNo(), equalTo(inRequest.startingSeqNo())); + } else { + assertThat(SequenceNumbersService.UNASSIGNED_SEQ_NO, equalTo(inRequest.startingSeqNo())); + } } } From 3571036a2887b50e8b11b0519f3e3bdcd3f0053a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 07:41:56 -0500 Subject: [PATCH 28/41] Add note about snapshot restore impact on recovery --- .../indices/recovery/RecoverySourceHandler.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 5fff8b503d26f..32c92e8e700c4 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -224,8 +224,12 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl } } 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; } - return false; } /** From da94b2803694777cf95339609d4b04d60dc837b9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 15:54:43 -0500 Subject: [PATCH 29/41] Rename EvilRecoveryIT to EvilPeerRecoveryIT --- .../recovery/{EvilRecoveryIT.java => EvilPeerRecoveryIT.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename core/src/test/java/org/elasticsearch/indices/recovery/{EvilRecoveryIT.java => EvilPeerRecoveryIT.java} (99%) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java similarity index 99% rename from core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java rename to core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java index 7250f119887d6..62a821f4939fd 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/EvilRecoveryIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java @@ -48,7 +48,7 @@ import static org.hamcrest.Matchers.equalTo; @ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, numDataNodes = 0) -public class EvilRecoveryIT extends ESIntegTestCase { +public class EvilPeerRecoveryIT extends ESIntegTestCase { private static AtomicReference indexLatch = new AtomicReference<>(); private static AtomicReference waitForOpsToCompleteLatch = new AtomicReference<>(); From e801c5b0db55093f70c52635ca1fff00939f1e10 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 16:25:10 -0500 Subject: [PATCH 30/41] Use bulk thread pool size for number of docs --- .../indices/recovery/EvilPeerRecoveryIT.java | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java index 62a821f4939fd..019478b4d5aea 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java @@ -34,6 +34,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.ArrayList; @@ -118,17 +119,6 @@ public final boolean incrementToken() throws IOException { } - @Override - protected Settings nodeSettings(final int nodeOrdinal) { - final Settings nodeSettings = super.nodeSettings(nodeOrdinal); - final int processors = randomIntBetween(1, 4); - /* - * We have to do this to ensure that there are sufficiently many threads to accept the indexing requests, otherwise operations will - * instead be queued and never trip the latch that all operations are inside the engine. - */ - return Settings.builder().put(nodeSettings).put("processors", processors).put("thread_pool.bulk.size", 1 + processors).build(); - } - /* * This tests that sequence-number-based recoveries wait for in-flight operations to complete. The trick here is simple. We latch some * in-flight operations inside the engine after sequence numbers are assigned. While these operations are latched, we restart a replica. @@ -137,9 +127,6 @@ protected Settings nodeSettings(final int nodeOrdinal) { */ public void testRecoveryWaitsForOps() throws Exception { final int docs = randomIntBetween(1, 64); - final int numberOfProcessors = EsExecutors.numberOfProcessors(nodeSettings(0)); - final int latchedDocs = randomIntBetween(1, 1 + numberOfProcessors); - try { internalCluster().startMasterOnlyNode(); final String primaryNode = internalCluster().startDataOnlyNode(nodeSettings(0)); @@ -188,7 +175,9 @@ public void testRecoveryWaitsForOps() throws Exception { } // start some in-flight operations that will get latched in the engine + final List threads = new ArrayList<>(); + final int latchedDocs = internalCluster().getInstance(ThreadPool.class, replicaNode).info(ThreadPool.Names.BULK).getMax(); indexLatch.set(new CountDownLatch(latchedDocs)); waitForOpsToCompleteLatch.set(new CountDownLatch(1)); for (int i = docs; i < docs + latchedDocs; i++) { From a4cf33ecbf48dc44a33f34ac13be5f1a931150b9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 16:26:46 -0500 Subject: [PATCH 31/41] Remove extraneous blank line --- .../org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java index 019478b4d5aea..41b20770b69d5 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/EvilPeerRecoveryIT.java @@ -175,7 +175,6 @@ public void testRecoveryWaitsForOps() throws Exception { } // start some in-flight operations that will get latched in the engine - final List threads = new ArrayList<>(); final int latchedDocs = internalCluster().getInstance(ThreadPool.class, replicaNode).info(ThreadPool.Names.BULK).getMax(); indexLatch.set(new CountDownLatch(latchedDocs)); From c16295acfa8f456de42c8cad767564f1d62f6e03 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 16:49:24 -0500 Subject: [PATCH 32/41] Temporarily enable trace logging on test --- .../java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 431b592fac986..39fc925593516 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -45,6 +45,7 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; @@ -384,6 +385,8 @@ public void testLatestVersionLoaded() throws Exception { assertThat(state.metaData().index("test").getAliases().get("test_alias").filter(), notNullValue()); } + // test fails with seed FE6770A74885D66E + @TestLogging("org.elasticsearch.indices.recovery:TRACE") public void testReusePeerRecovery() throws Exception { final Settings settings = Settings.builder() .put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) From 32c6702c6e569f67bb8aa7b382b652db47c86d06 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 19:24:09 -0500 Subject: [PATCH 33/41] More trace logging for test --- .../java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 39fc925593516..7ef0b3f1680bd 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -386,7 +386,7 @@ public void testLatestVersionLoaded() throws Exception { } // test fails with seed FE6770A74885D66E - @TestLogging("org.elasticsearch.indices.recovery:TRACE") + @TestLogging("org.elasticsearch.indices.recovery:TRACE,org.elasticsearch.index.shard.StoreRecovery:TRACE") public void testReusePeerRecovery() throws Exception { final Settings settings = Settings.builder() .put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) From 76f880771a08231e0c93f3f3d576595015ae1bba Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 23 Jan 2017 19:28:14 -0500 Subject: [PATCH 34/41] Fix shard ID in logging statement --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 32c92e8e700c4..1e68314bc2cb5 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -201,7 +201,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl final long startingSeqNo = request.startingSeqNo(); assert startingSeqNo >= 0; final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - logger.trace("{} starting: [{}], ending: [{}}", shardId, startingSeqNo, endingSeqNo); + 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( From adafa21eab67905490d6fdba8d9ca3d06153ee91 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Tue, 24 Jan 2017 22:21:56 -0500 Subject: [PATCH 35/41] Fix RFGIT#testReusePeerRecovery test bug --- .../recovery/PeerRecoveryTargetService.java | 4 +++- .../indices/recovery/RecoveryState.java | 20 +++++++++++++++++++ .../gateway/RecoveryFromGatewayIT.java | 7 ++++--- 3 files changed, 27 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index bd78cc915e35e..55af1c7d58a06 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -529,7 +529,9 @@ class FilesInfoRequestHandler implements TransportRequestHandler 0) { builder.dateField(Fields.STOP_TIME_IN_MILLIS, Fields.STOP_TIME, timer.stopTime); @@ -308,6 +327,7 @@ static final class Fields { static final String TYPE = "type"; static final String STAGE = "stage"; static final String PRIMARY = "primary"; + static final String SEQUENCE_NUMBER_BASED_RECOVERY = "sequence_number_based_recovery"; static final String START_TIME = "start_time"; static final String START_TIME_IN_MILLIS = "start_time_in_millis"; static final String STOP_TIME = "stop_time"; diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 7ef0b3f1680bd..39fb15eb47192 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -68,6 +68,7 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -385,8 +386,6 @@ public void testLatestVersionLoaded() throws Exception { assertThat(state.metaData().index("test").getAliases().get("test_alias").filter(), notNullValue()); } - // test fails with seed FE6770A74885D66E - @TestLogging("org.elasticsearch.indices.recovery:TRACE,org.elasticsearch.index.shard.StoreRecovery:TRACE") public void testReusePeerRecovery() throws Exception { final Settings settings = Settings.builder() .put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) @@ -465,7 +464,9 @@ public void testReusePeerRecovery() throws Exception { recovered += file.length(); } } - if (!recoveryState.getPrimary() && (useSyncIds == false)) { + if (recoveryState.getSequenceNumberBasedRecovery()) { + assertThat(recoveryState.getTranslog().recoveredOperations(), greaterThanOrEqualTo(0)); + } else if (!recoveryState.getPrimary() && (useSyncIds == false)) { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", recoveryState.getShardId().getId(), recoveryState.getSourceNode().getName(), recoveryState.getTargetNode().getName(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); From 270a68adb3a8d4ccb3fcbc8c917c738cdb15ea7a Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 25 Jan 2017 10:13:13 -0500 Subject: [PATCH 36/41] Cleanup --- .../indices/recovery/PeerRecoveryTargetService.java | 2 +- .../indices/recovery/RecoverySourceHandler.java | 2 +- .../org/elasticsearch/gateway/RecoveryFromGatewayIT.java | 1 - .../indices/recovery/RecoverySourceHandlerTests.java | 6 +++--- 4 files changed, 5 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 55af1c7d58a06..337e8bff2ea8d 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -338,7 +338,7 @@ private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recove logger.trace("{} preparing for file-based recovery from [{}]", recoveryTarget.shardId(), recoveryTarget.sourceNode()); } else { logger.trace( - "{} preparing for sequence number-based recovery starting at local checkpoint [{}] from [{}]", + "{} preparing for sequence-number-based recovery starting at local checkpoint [{}] from [{}]", recoveryTarget.shardId(), startingSeqNo, recoveryTarget.sourceNode()); diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 1e68314bc2cb5..7e30fa4ae1304 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -201,7 +201,7 @@ boolean isTranslogReadyForSequenceNumberBasedRecovery(final Translog.View transl final long startingSeqNo = request.startingSeqNo(); assert startingSeqNo >= 0; final long endingSeqNo = shard.seqNoStats().getMaxSeqNo(); - logger.trace("{} starting: [{}], ending: [{}}", shard.shardId(), startingSeqNo, endingSeqNo); + 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( diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 39fb15eb47192..bbec8df55fe88 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -45,7 +45,6 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster.RestartCallback; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 210981dedc42e..c28f5ff9ea0aa 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -325,7 +325,7 @@ public void phase2(Translog.Snapshot snapshot) { }; expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget); - // phase1 should only be attempted if we are not doing a sequence number-based recovery + // phase1 should only be attempted if we are not doing a sequence-number-based recovery assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertFalse(phase2Called.get()); @@ -371,7 +371,7 @@ public void testWaitForClusterStateOnPrimaryRelocation() throws IOException, Int }; final Function delayNewRecoveries = s -> { - // phase1 should only be attempted if we are not doing a sequence number-based recovery + // phase1 should only be attempted if we are not doing a sequence-number-based recovery assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); @@ -417,7 +417,7 @@ public void phase2(final Translog.Snapshot snapshot) { handler.recoverToTarget(); assertTrue(ensureClusterStateVersionCalled.get()); - // phase1 should only be attempted if we are not doing a sequence number-based recovery + // phase1 should only be attempted if we are not doing a sequence-number-based recovery assertThat(phase1Called.get(), equalTo(!isTranslogReadyForSequenceNumberBasedRecovery)); assertTrue(prepareTargetForTranslogCalled.get()); assertTrue(phase2Called.get()); From 2e67a0b6f02451c472dc3647d7c3e5ceefb66947 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 26 Jan 2017 10:13:39 -0500 Subject: [PATCH 37/41] Revert "Fix RFGIT#testReusePeerRecovery test bug" This reverts commit adafa21eab67905490d6fdba8d9ca3d06153ee91. --- .../recovery/PeerRecoveryTargetService.java | 4 +--- .../recovery/RecoverySourceHandler.java | 4 ++-- .../indices/recovery/RecoveryState.java | 20 ------------------- .../gateway/RecoveryFromGatewayIT.java | 5 +---- 4 files changed, 4 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 337e8bff2ea8d..23f6356ef6b8a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -529,9 +529,7 @@ class FilesInfoRequestHandler implements TransportRequestHandler 0) { builder.dateField(Fields.STOP_TIME_IN_MILLIS, Fields.STOP_TIME, timer.stopTime); @@ -327,7 +308,6 @@ static final class Fields { static final String TYPE = "type"; static final String STAGE = "stage"; static final String PRIMARY = "primary"; - static final String SEQUENCE_NUMBER_BASED_RECOVERY = "sequence_number_based_recovery"; static final String START_TIME = "start_time"; static final String START_TIME_IN_MILLIS = "start_time_in_millis"; static final String STOP_TIME = "stop_time"; diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index bbec8df55fe88..431b592fac986 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -67,7 +67,6 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; @@ -463,9 +462,7 @@ public void testReusePeerRecovery() throws Exception { recovered += file.length(); } } - if (recoveryState.getSequenceNumberBasedRecovery()) { - assertThat(recoveryState.getTranslog().recoveredOperations(), greaterThanOrEqualTo(0)); - } else if (!recoveryState.getPrimary() && (useSyncIds == false)) { + if (!recoveryState.getPrimary() && (useSyncIds == false)) { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", recoveryState.getShardId().getId(), recoveryState.getSourceNode().getName(), recoveryState.getTargetNode().getName(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); From 06a3785e15eda4862045066a8cd173048989eeaf Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 26 Jan 2017 13:07:43 -0500 Subject: [PATCH 38/41] Rewrite reuse peer recovery test --- .../gateway/RecoveryFromGatewayIT.java | 141 ++++++++---------- 1 file changed, 65 insertions(+), 76 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 431b592fac986..b55767291aa52 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; +import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -30,6 +31,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -45,17 +47,23 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster.RestartCallback; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; import java.util.stream.IntStream; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; @@ -384,85 +392,75 @@ public void testLatestVersionLoaded() throws Exception { assertThat(state.metaData().index("test").getAliases().get("test_alias").filter(), notNullValue()); } + @TestLogging("org.elasticsearch.indices.recovery:TRACE") public void testReusePeerRecovery() throws Exception { - final Settings settings = Settings.builder() - .put(MockFSIndexStore.INDEX_CHECK_INDEX_ON_CLOSE_SETTING.getKey(), false) - .put("gateway.recover_after_nodes", 4) - .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING.getKey(), 4) - .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING.getKey(), 4) - .put(MockFSDirectoryService.CRASH_INDEX_SETTING.getKey(), false).build(); - - internalCluster().startNodes(4, settings); - // prevent any rebalance actions during the peer recovery - // if we run into a relocation the reuse count will be 0 and this fails the test. We are testing here if - // we reuse the files on disk after full restarts for replicas. - assertAcked(prepareCreate("test").setSettings(Settings.builder() - .put(indexSettings()) - .put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE))); - ensureGreen(); + internalCluster().startMasterOnlyNode(); + final String primaryNode = internalCluster().startDataOnlyNode(nodeSettings(0)); + + // create the index with our mapping + client(primaryNode) + .admin() + .indices() + .prepareCreate("test") + .setSettings(Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 1)) + .get(); + logger.info("--> indexing docs"); - for (int i = 0; i < 1000; i++) { - client().prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); - if ((i % 200) == 0) { - client().admin().indices().prepareFlush().execute().actionGet(); - } + for (int i = 0; i < randomIntBetween(1, 1024); i++) { + client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); } - if (randomBoolean()) { - client().admin().indices().prepareFlush().execute().actionGet(); - } - logger.info("Running Cluster Health"); + + client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + + // start the replica node; we do this after indexing so a file-based recovery is triggered to ensure the files are identical + final String replicaNode = internalCluster().startDataOnlyNode(nodeSettings(1)); ensureGreen(); - client().admin().indices().prepareForceMerge("test").setMaxNumSegments(100).get(); // just wait for merges - client().admin().indices().prepareFlush().setForce(true).get(); - - boolean useSyncIds = randomBoolean(); - if (useSyncIds == false) { - logger.info("--> disabling allocation while the cluster is shut down"); - - // Disable allocations while we are closing nodes - client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(Settings.builder() - .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), EnableAllocationDecider.Allocation.NONE)) - .get(); - logger.info("--> full cluster restart"); - internalCluster().fullRestart(); - - logger.info("--> waiting for cluster to return to green after first shutdown"); - ensureGreen(); - } else { - logger.info("--> trying to sync flush"); - assertEquals(client().admin().indices().prepareSyncedFlush("test").get().failedShards(), 0); - assertSyncIdsNotNull(); + + final RecoveryResponse initialRecoveryReponse = client().admin().indices().prepareRecoveries("test").get(); + final Set files = new HashSet<>(); + for (final RecoveryState recoveryState : initialRecoveryReponse.shardRecoveryStates().get("test")) { + if (recoveryState.getTargetNode().getName().equals(replicaNode)) { + for (final RecoveryState.File file : recoveryState.getIndex().fileDetails()) { + files.add(file.name()); + } + break; + } } - logger.info("--> disabling allocation while the cluster is shut down{}", useSyncIds ? "" : " a second time"); - // Disable allocations while we are closing nodes - client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(Settings.builder() - .put(EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING.getKey(), EnableAllocationDecider.Allocation.NONE)) - .get(); + logger.info("--> restart replica node"); - Map primaryTerms = assertAndCapturePrimaryTerms(null); + internalCluster().restartNode(replicaNode, new RestartCallback() { + @Override + public Settings onNodeStopped(String nodeName) throws Exception { + // index some more documents; we expect to reuse the files that already exist on the replica + for (int i = 0; i < randomIntBetween(1, 1024); i++) { + client(primaryNode).prepareIndex("test", "type").setSource("field", "value").execute().actionGet(); + } - logger.info("--> full cluster restart"); - internalCluster().fullRestart(); + // prevent a sequence-number-based recovery from being possible + client(primaryNode).admin().indices().prepareFlush("test").setForce(true).get(); + return super.onNodeStopped(nodeName); + } + }); - logger.info("--> waiting for cluster to return to green after {}shutdown", useSyncIds ? "" : "second "); ensureGreen(); - primaryTerms = assertAndCapturePrimaryTerms(primaryTerms); - if (useSyncIds) { - assertSyncIdsNotNull(); - } - RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); - for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { + final RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); + for (final RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { long recovered = 0; - for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) { - if (file.name().startsWith("segments")) { + int filesRecovered = 0; + for (final RecoveryState.File file : recoveryState.getIndex().fileDetails()) { + if (files.contains(file.name()) == false) { recovered += file.length(); + filesRecovered++; } } - if (!recoveryState.getPrimary() && (useSyncIds == false)) { + if (recoveryState.getPrimary()) { + assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0L)); + assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); + assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0)); + assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); + } else { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", recoveryState.getShardId().getId(), recoveryState.getSourceNode().getName(), recoveryState.getTargetNode().getName(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); @@ -470,19 +468,10 @@ public void testReusePeerRecovery() throws Exception { assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0L)); // we have to recover the segments file since we commit the translog ID on engine startup assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); - assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(1)); - assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - 1)); + assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(filesRecovered)); + assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - filesRecovered)); assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); - } else { - if (useSyncIds && !recoveryState.getPrimary()) { - logger.info("--> replica shard {} recovered from {} to {} using sync id, recovered {}, reuse {}", - recoveryState.getShardId().getId(), recoveryState.getSourceNode().getName(), recoveryState.getTargetNode().getName(), - recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); - } - assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0L)); - assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes())); - assertThat(recoveryState.getIndex().recoveredFileCount(), equalTo(0)); - assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount())); + assertThat("no translog ops should be recovered", recoveryState.getTranslog().recoveredOperations(), equalTo(0)); } } } From eeaa4f9229fa1e10c24cd3241f1c7840416155f6 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 26 Jan 2017 13:09:54 -0500 Subject: [PATCH 39/41] Remove unused imports --- .../org/elasticsearch/gateway/RecoveryFromGatewayIT.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index b55767291aa52..f12b5e383a6ca 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -23,15 +23,11 @@ import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.ShardStats; -import org.elasticsearch.client.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; -import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -48,22 +44,18 @@ import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster.RestartCallback; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.CountDownLatch; import java.util.stream.IntStream; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; From 62aabb01c5df920727a066f067cfb2e5ec9ac888 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 26 Jan 2017 14:26:03 -0500 Subject: [PATCH 40/41] Cleanup test --- .../elasticsearch/gateway/RecoveryFromGatewayIT.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index f12b5e383a6ca..696add629878f 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -43,7 +43,6 @@ import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster.RestartCallback; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSIndexStore; import java.nio.file.DirectoryStream; @@ -384,8 +383,7 @@ public void testLatestVersionLoaded() throws Exception { assertThat(state.metaData().index("test").getAliases().get("test_alias").filter(), notNullValue()); } - @TestLogging("org.elasticsearch.indices.recovery:TRACE") - public void testReusePeerRecovery() throws Exception { + public void testReuseInFileBasedPeerRecovery() throws Exception { internalCluster().startMasterOnlyNode(); final String primaryNode = internalCluster().startDataOnlyNode(nodeSettings(0)); @@ -459,9 +457,9 @@ public Settings onNodeStopped(String nodeName) throws Exception { assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0L)); // we have to recover the segments file since we commit the translog ID on engine startup - assertThat("all bytes should be reused except of the segments file", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); - assertThat("no files should be recovered except of the segments file", recoveryState.getIndex().recoveredFileCount(), equalTo(filesRecovered)); - assertThat("all files should be reused except of the segments file", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - filesRecovered)); + assertThat("all bytes should be reused except the segment from the last round of indexing", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); + assertThat("no files should be recovered except the segment from the last round of indexing", recoveryState.getIndex().recoveredFileCount(), equalTo(filesRecovered)); + assertThat("all files should be reused except the segment from the last round of indexing", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - filesRecovered)); assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); assertThat("no translog ops should be recovered", recoveryState.getTranslog().recoveredOperations(), equalTo(0)); } From 97e0b20c7cc8887aee25d48b7a51c071cc96d3f5 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 26 Jan 2017 14:36:40 -0500 Subject: [PATCH 41/41] More cleanup --- .../gateway/RecoveryFromGatewayIT.java | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java index 696add629878f..c97c0982ebbd6 100644 --- a/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/RecoveryFromGatewayIT.java @@ -438,11 +438,16 @@ public Settings onNodeStopped(String nodeName) throws Exception { final RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); for (final RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) { long recovered = 0; + long reused = 0; int filesRecovered = 0; + int filesReused = 0; for (final RecoveryState.File file : recoveryState.getIndex().fileDetails()) { if (files.contains(file.name()) == false) { recovered += file.length(); filesRecovered++; + } else { + reused += file.length(); + filesReused++; } } if (recoveryState.getPrimary()) { @@ -454,12 +459,14 @@ public Settings onNodeStopped(String nodeName) throws Exception { logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", recoveryState.getShardId().getId(), recoveryState.getSourceNode().getName(), recoveryState.getTargetNode().getName(), recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); - assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); + assertThat("bytes should have been recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0L)); // we have to recover the segments file since we commit the translog ID on engine startup - assertThat("all bytes should be reused except the segment from the last round of indexing", recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); - assertThat("no files should be recovered except the segment from the last round of indexing", recoveryState.getIndex().recoveredFileCount(), equalTo(filesRecovered)); - assertThat("all files should be reused except the segment from the last round of indexing", recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - filesRecovered)); + assertThat("all existing files should be reused, byte count mismatch", recoveryState.getIndex().reusedBytes(), equalTo(reused)); + assertThat(recoveryState.getIndex().reusedBytes(), equalTo(recoveryState.getIndex().totalBytes() - recovered)); + assertThat("the segment from the last round of indexing should be recovered", recoveryState.getIndex().recoveredFileCount(), equalTo(filesRecovered)); + assertThat("all existing files should be reused, file count mismatch", recoveryState.getIndex().reusedFileCount(), equalTo(filesReused)); + assertThat(recoveryState.getIndex().reusedFileCount(), equalTo(recoveryState.getIndex().totalFileCount() - filesRecovered)); assertThat("> 0 files should be reused", recoveryState.getIndex().reusedFileCount(), greaterThan(0)); assertThat("no translog ops should be recovered", recoveryState.getTranslog().recoveredOperations(), equalTo(0)); }