From 2750fd8f5714f756bd64c45771b2960cc10e4dfc Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 14 Dec 2017 18:19:34 +0100 Subject: [PATCH 01/11] Remove SequenceNumbersService --- .../replication/ReplicationOperation.java | 1 - .../elasticsearch/index/engine/Engine.java | 4 +- .../index/engine/EngineConfig.java | 13 +- .../index/engine/InternalEngine.java | 65 ++-- .../index/seqno/GlobalCheckpointTracker.java | 4 +- .../index/seqno/LocalCheckpointTracker.java | 10 +- .../index/seqno/SequenceNumbersService.java | 281 ------------------ .../elasticsearch/index/shard/IndexShard.java | 61 ++-- .../index/shard/LocalShardSnapshot.java | 2 +- 9 files changed, 84 insertions(+), 357 deletions(-) delete mode 100644 core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 6fa06c25457b0..ca3a26b3c301f 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.ReplicationGroup; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; 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 27575a9c354eb..473dc2ba88ed1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -64,8 +64,8 @@ import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.merge.MergeStats; +import org.elasticsearch.index.seqno.LocalCheckpointTracker; 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.index.translog.Translog; @@ -567,7 +567,7 @@ public CommitStats commitStats() { * * @return the sequence number service */ - public abstract SequenceNumbersService seqNoService(); + public abstract LocalCheckpointTracker getLocalCheckpointTracker(); /** * Read the last segments info from the commit pointed to by the searcher manager diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 8c134b140bdb3..1041a7c42b3e1 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -78,6 +79,7 @@ public final class EngineConfig { private final TranslogRecoveryRunner translogRecoveryRunner; @Nullable private final CircuitBreakerService circuitBreakerService; + private final GlobalCheckpointTracker globalCheckpointTracker; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -124,7 +126,8 @@ public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, Thr boolean forceNewHistoryUUID, TranslogConfig translogConfig, TimeValue flushMergesAfter, List externalRefreshListener, List internalRefreshListener, Sort indexSort, - TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService) { + TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, + GlobalCheckpointTracker globalCheckpointTracker) { if (openMode == null) { throw new IllegalArgumentException("openMode must not be null"); } @@ -155,6 +158,7 @@ public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, Thr this.indexSort = indexSort; this.translogRecoveryRunner = translogRecoveryRunner; this.circuitBreakerService = circuitBreakerService; + this.globalCheckpointTracker = globalCheckpointTracker; } /** @@ -227,6 +231,13 @@ public Store getStore() { return store; } + /** + * Returns the global checkpoint tracker + */ + public GlobalCheckpointTracker getGlobalCheckpointTracker() { + return globalCheckpointTracker; + } + /** * Returns the {@link org.apache.lucene.index.MergePolicy} for the engines {@link org.apache.lucene.index.IndexWriter} */ 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 e4f6a6f9b0a21..3cbb1406e520e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -67,9 +67,9 @@ import org.elasticsearch.index.mapper.UidFieldMapper; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; +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.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; @@ -93,6 +93,7 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.LongSupplier; public class InternalEngine extends Engine { @@ -123,7 +124,7 @@ public class InternalEngine extends Engine { private final IndexThrottle throttle; - private final SequenceNumbersService seqNoService; + private final LocalCheckpointTracker localCheckpointTracker; private final String uidField; @@ -150,12 +151,12 @@ public class InternalEngine extends Engine { private final String historyUUID; public InternalEngine(EngineConfig engineConfig) { - this(engineConfig, InternalEngine::sequenceNumberService); + this(engineConfig, InternalEngine::localCheckpointTracker); } InternalEngine( final EngineConfig engineConfig, - final BiFunction seqNoServiceSupplier) { + final Function localCheckpointTrackerSupplier) { super(engineConfig); openMode = engineConfig.getOpenMode(); if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) { @@ -181,9 +182,12 @@ public InternalEngine(EngineConfig engineConfig) { try { final SeqNoStats seqNoStats = loadSeqNoStats(openMode); logger.trace("recovered [{}]", seqNoStats); - this.seqNoService = seqNoServiceSupplier.apply(engineConfig, seqNoStats); + this.localCheckpointTracker = localCheckpointTrackerSupplier.apply(seqNoStats); + engineConfig.getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(seqNoStats.getGlobalCheckpoint(), + "initializing from persisted global checkpoint"); this.snapshotDeletionPolicy = new SnapshotDeletionPolicy( - new CombinedDeletionPolicy(openMode, translogDeletionPolicy, seqNoService::getGlobalCheckpoint) + new CombinedDeletionPolicy(openMode, translogDeletionPolicy, + engineConfig.getGlobalCheckpointTracker()::getGlobalCheckpoint) ); writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG); updateMaxUnsafeAutoIdTimestampFromWriter(writer); @@ -195,7 +199,8 @@ public InternalEngine(EngineConfig engineConfig) { historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer; - translog = openTranslog(engineConfig, writer, translogDeletionPolicy, () -> seqNoService.getGlobalCheckpoint()); + translog = openTranslog(engineConfig, writer, translogDeletionPolicy, + () -> engineConfig.getGlobalCheckpointTracker().getGlobalCheckpoint()); assert translog.getGeneration() != null; this.translog = translog; updateWriterOnOpen(); @@ -310,12 +315,12 @@ protected int getRefCount(IndexSearcher reference) { public void restoreLocalCheckpointFromTranslog() throws IOException { try (ReleasableLock ignored = writeLock.acquire()) { ensureOpen(); - final long localCheckpoint = seqNoService.getLocalCheckpoint(); + final long localCheckpoint = localCheckpointTracker.getCheckpoint(); try (Translog.Snapshot snapshot = getTranslog().newSnapshotFromMinSeqNo(localCheckpoint + 1)) { Translog.Operation operation; while ((operation = snapshot.next()) != null) { if (operation.seqNo() > localCheckpoint) { - seqNoService.markSeqNoAsCompleted(operation.seqNo()); + localCheckpointTracker.markSeqNoAsCompleted(operation.seqNo()); } } } @@ -326,17 +331,17 @@ public void restoreLocalCheckpointFromTranslog() throws IOException { public int fillSeqNoGaps(long primaryTerm) throws IOException { try (ReleasableLock ignored = writeLock.acquire()) { ensureOpen(); - final long localCheckpoint = seqNoService.getLocalCheckpoint(); - final long maxSeqNo = seqNoService.getMaxSeqNo(); + final long localCheckpoint = localCheckpointTracker.getCheckpoint(); + final long maxSeqNo = localCheckpointTracker.getMaxSeqNo(); int numNoOpsAdded = 0; for ( long seqNo = localCheckpoint + 1; seqNo <= maxSeqNo; - seqNo = seqNoService.getLocalCheckpoint() + 1 /* the local checkpoint might have advanced so we leap-frog */) { + seqNo = localCheckpointTracker.getCheckpoint() + 1 /* the local checkpoint might have advanced so we leap-frog */) { innerNoOp(new NoOp(seqNo, primaryTerm, Operation.Origin.PRIMARY, System.nanoTime(), "filling gaps")); numNoOpsAdded++; - assert seqNo <= seqNoService.getLocalCheckpoint() - : "local checkpoint did not advance; was [" + seqNo + "], now [" + seqNoService.getLocalCheckpoint() + "]"; + assert seqNo <= localCheckpointTracker.getCheckpoint() + : "local checkpoint did not advance; was [" + seqNo + "], now [" + localCheckpointTracker.getCheckpoint() + "]"; } return numNoOpsAdded; @@ -354,16 +359,10 @@ private void updateMaxUnsafeAutoIdTimestampFromWriter(IndexWriter writer) { maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), commitMaxUnsafeAutoIdTimestamp)); } - static SequenceNumbersService sequenceNumberService( - final EngineConfig engineConfig, - final SeqNoStats seqNoStats) { - return new SequenceNumbersService( - engineConfig.getShardId(), - engineConfig.getAllocationId(), - engineConfig.getIndexSettings(), + static LocalCheckpointTracker localCheckpointTracker(final SeqNoStats seqNoStats) { + return new LocalCheckpointTracker( seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()); + seqNoStats.getLocalCheckpoint()); } private SeqNoStats loadSeqNoStats(EngineConfig.OpenMode openMode) throws IOException { @@ -718,7 +717,7 @@ private long generateSeqNoForOperation(final Operation operation) { * @return the sequence number */ protected long doGenerateSeqNoForOperation(final Operation operation) { - return seqNoService.generateSeqNo(); + return localCheckpointTracker.generateSeqNo(); } @Override @@ -789,7 +788,7 @@ public IndexResult index(Index index) throws IOException { indexResult.setTranslogLocation(location); } if (indexResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - seqNoService.markSeqNoAsCompleted(indexResult.getSeqNo()); + localCheckpointTracker.markSeqNoAsCompleted(indexResult.getSeqNo()); } indexResult.setTook(System.nanoTime() - index.startTime()); indexResult.freeze(); @@ -819,7 +818,7 @@ private IndexingStrategy planIndexingAsNonPrimary(Index index) throws IOExceptio // this allows to ignore the case where a document was found in the live version maps in // a delete state and return false for the created flag in favor of code simplicity final OpVsLuceneDocStatus opVsLucene; - if (index.seqNo() <= seqNoService.getLocalCheckpoint()){ + if (index.seqNo() <= localCheckpointTracker.getCheckpoint()){ // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already // part of the lucene commit (either from a peer recovery or a local translog) @@ -1080,7 +1079,7 @@ public DeleteResult delete(Delete delete) throws IOException { deleteResult.setTranslogLocation(location); } if (deleteResult.getSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO) { - seqNoService.markSeqNoAsCompleted(deleteResult.getSeqNo()); + localCheckpointTracker.markSeqNoAsCompleted(deleteResult.getSeqNo()); } deleteResult.setTook(System.nanoTime() - delete.startTime()); deleteResult.freeze(); @@ -1106,7 +1105,7 @@ private DeletionStrategy planDeletionAsNonPrimary(Delete delete) throws IOExcept // this allows to ignore the case where a document was found in the live version maps in // a delete state and return true for the found flag in favor of code simplicity final OpVsLuceneDocStatus opVsLucene; - if (delete.seqNo() <= seqNoService.getLocalCheckpoint()) { + if (delete.seqNo() <= localCheckpointTracker.getCheckpoint()) { // the operation seq# is lower then the current local checkpoint and thus was already put into lucene // this can happen during recovery where older operations are sent from the translog that are already // part of the lucene commit (either from a peer recovery or a local translog) @@ -1253,7 +1252,7 @@ private NoOpResult innerNoOp(final NoOp noOp) throws IOException { return noOpResult; } finally { if (seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO) { - seqNoService.markSeqNoAsCompleted(seqNo); + localCheckpointTracker.markSeqNoAsCompleted(seqNo); } } } @@ -1997,7 +1996,7 @@ private void maybeDie(final String maybeMessage, final Throwable maybeFatal) { protected void commitIndexWriter(final IndexWriter writer, final Translog translog, @Nullable final String syncId) throws IOException { ensureCanFlush(); try { - final long localCheckpoint = seqNoService.getLocalCheckpoint(); + final long localCheckpoint = localCheckpointTracker.getCheckpoint(); final Translog.TranslogGeneration translogGeneration = translog.getMinGenerationForSeqNo(localCheckpoint + 1); final String translogFileGeneration = Long.toString(translogGeneration.translogFileGeneration); final String translogUUID = translogGeneration.translogUUID; @@ -2020,7 +2019,7 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl if (syncId != null) { commitData.put(Engine.SYNC_COMMIT_ID, syncId); } - commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(seqNoService.getMaxSeqNo())); + commitData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(localCheckpointTracker.getMaxSeqNo())); commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); logger.trace("committing writer with commit data [{}]", commitData); @@ -2084,8 +2083,8 @@ public MergeStats getMergeStats() { return mergeScheduler.stats(); } - public final SequenceNumbersService seqNoService() { - return seqNoService; + public final LocalCheckpointTracker getLocalCheckpointTracker() { + return localCheckpointTracker; } /** 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 d2b53aac1a045..8226b8f558e51 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -214,7 +214,7 @@ public int hashCode() { * * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID */ - synchronized ObjectLongMap getInSyncGlobalCheckpoints() { + public synchronized ObjectLongMap getInSyncGlobalCheckpoints() { assert primaryMode; assert handoffInProgress == false; final ObjectLongMap globalCheckpoints = new ObjectLongHashMap<>(checkpoints.size()); // upper bound on the size @@ -329,7 +329,7 @@ private static long inSyncCheckpointStates( * @param indexSettings the index settings * @param globalCheckpoint the last known global checkpoint for this shard, or {@link SequenceNumbers#UNASSIGNED_SEQ_NO} */ - GlobalCheckpointTracker( + public GlobalCheckpointTracker( final ShardId shardId, final String allocationId, final IndexSettings indexSettings, 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 2644dbfc32d34..52a8145b9d86b 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java @@ -78,7 +78,7 @@ public LocalCheckpointTracker(final long maxSeqNo, final long localCheckpoint) { * * @return the next assigned sequence number */ - synchronized long generateSeqNo() { + public synchronized long generateSeqNo() { return nextSeqNo++; } @@ -109,7 +109,7 @@ public synchronized void markSeqNoAsCompleted(final long seqNo) { * * @param checkpoint the local checkpoint to reset this tracker to */ - synchronized void resetCheckpoint(final long checkpoint) { + public synchronized void resetCheckpoint(final long checkpoint) { assert checkpoint != SequenceNumbers.UNASSIGNED_SEQ_NO; assert checkpoint <= this.checkpoint; processedSeqNo.clear(); @@ -130,7 +130,7 @@ public long getCheckpoint() { * * @return the maximum sequence number */ - long getMaxSeqNo() { + public long getMaxSeqNo() { return nextSeqNo - 1; } @@ -140,7 +140,7 @@ long getMaxSeqNo() { * * @implNote this is needed to make sure the local checkpoint and max seq no are consistent */ - synchronized SeqNoStats getStats(final long globalCheckpoint) { + public synchronized SeqNoStats getStats(final long globalCheckpoint) { return new SeqNoStats(getMaxSeqNo(), getCheckpoint(), globalCheckpoint); } @@ -151,7 +151,7 @@ synchronized SeqNoStats getStats(final long globalCheckpoint) { * @throws InterruptedException if the thread was interrupted while blocking on the condition */ @SuppressForbidden(reason = "Object#wait") - synchronized void waitForOpsToComplete(final long seqNo) throws InterruptedException { + public synchronized void waitForOpsToComplete(final long seqNo) throws InterruptedException { while (checkpoint < seqNo) { // notified by updateCheckpoint this.wait(); diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java deleted file mode 100644 index 1b46eedacc457..0000000000000 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbersService.java +++ /dev/null @@ -1,281 +0,0 @@ -/* - * 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 com.carrotsearch.hppc.ObjectLongMap; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.index.IndexSettings; -import org.elasticsearch.index.shard.AbstractIndexShardComponent; -import org.elasticsearch.index.shard.ReplicationGroup; -import org.elasticsearch.index.shard.ShardId; - -import java.util.Set; - -/** - * Encapsulates the local and global checkpoints into a single service for use as a shard component. - */ -public class SequenceNumbersService extends AbstractIndexShardComponent { - - 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 - * {@link SequenceNumbers#NO_OPS_PERFORMED}, {@code localCheckpoint} should be set to the last known local checkpoint for this - * shard, or {@link SequenceNumbers#NO_OPS_PERFORMED}, and {@code globalCheckpoint} should be set to the last known global - * checkpoint for this shard, or {@link SequenceNumbers#UNASSIGNED_SEQ_NO}. - * - * @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 SequenceNumbers#NO_OPS_PERFORMED} - * @param localCheckpoint the last known local checkpoint for this shard, or {@link SequenceNumbers#NO_OPS_PERFORMED} - * @param globalCheckpoint the last known global checkpoint for this shard, or {@link SequenceNumbers#UNASSIGNED_SEQ_NO} - */ - public SequenceNumbersService( - final ShardId shardId, - final String allocationId, - final IndexSettings indexSettings, - final long maxSeqNo, - final long localCheckpoint, - final long globalCheckpoint) { - super(shardId, indexSettings); - localCheckpointTracker = new LocalCheckpointTracker(maxSeqNo, localCheckpoint); - globalCheckpointTracker = new GlobalCheckpointTracker(shardId, allocationId, indexSettings, globalCheckpoint); - } - - /** - * Issue the next sequence number. Note that you must call {@link #markSeqNoAsCompleted(long)} after the operation for which the - * issued sequence number completes (whether or not the operation completes successfully). - * - * @return the next assigned sequence number - */ - public final long generateSeqNo() { - return localCheckpointTracker.generateSeqNo(); - } - - /** - * The maximum sequence number issued so far. See {@link LocalCheckpointTracker#getMaxSeqNo()} for additional details. - * - * @return the maximum sequence number - */ - public long 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 LocalCheckpointTracker#markSeqNoAsCompleted(long)} for additional details. - * - * @param seqNo the sequence number to mark as completed - */ - public void markSeqNoAsCompleted(final long seqNo) { - localCheckpointTracker.markSeqNoAsCompleted(seqNo); - } - - /** - * Resets the local checkpoint to the specified value. - * - * @param localCheckpoint the local checkpoint to reset to - */ - public void resetLocalCheckpoint(final long localCheckpoint) { - localCheckpointTracker.resetCheckpoint(localCheckpoint); - } - - /** - * The current sequence number stats. - * - * @return stats encapsulating the maximum sequence number, the local checkpoint and the global checkpoint - */ - public SeqNoStats stats() { - return localCheckpointTracker.getStats(getGlobalCheckpoint()); - } - - /** - * Notifies the service to update the local checkpoint for the shard with the provided allocation ID. See - * {@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) { - globalCheckpointTracker.updateLocalCheckpoint(allocationId, checkpoint); - } - - /** - * Update the local knowledge of the global checkpoint for the specified allocation ID. - * - * @param allocationId the allocation ID to update the global checkpoint for - * @param globalCheckpoint the global checkpoint - */ - public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { - globalCheckpointTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); - } - - /** - * Get the local knowledge of the global checkpoints for all in-sync allocation IDs. - * - * @return a map from allocation ID to the local knowledge of the global checkpoint for that allocation ID - */ - public ObjectLongMap getInSyncGlobalCheckpoints() { - return globalCheckpointTracker.getInSyncGlobalCheckpoints(); - } - - /** - * Called when the recovery process for a shard is ready to open the engine on the target shard. - * See {@link GlobalCheckpointTracker#initiateTracking(String)} for details. - * - * @param allocationId the allocation ID of the shard for which recovery was initiated - */ - public void initiateTracking(final String allocationId) { - globalCheckpointTracker.initiateTracking(allocationId); - } - - /** - * Marks the shard with the provided allocation ID as in-sync with the primary shard. See - * {@link GlobalCheckpointTracker#markAllocationIdAsInSync(String, long)} for additional details. - * - * @param allocationId the allocation ID of the shard to mark as in-sync - * @param localCheckpoint the current local checkpoint on the shard - */ - public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { - globalCheckpointTracker.markAllocationIdAsInSync(allocationId, localCheckpoint); - } - - /** - * Returns the local checkpoint for the shard. - * - * @return the local checkpoint - */ - public long getLocalCheckpoint() { - return localCheckpointTracker.getCheckpoint(); - } - - /** - * Returns the current replication group for the shard. - * - * @return the replication group - */ - public ReplicationGroup getReplicationGroup() { - return globalCheckpointTracker.getReplicationGroup(); - } - - /** - * Returns the global checkpoint for the shard. - * - * @return the global checkpoint - */ - public long getGlobalCheckpoint() { - return globalCheckpointTracker.getGlobalCheckpoint(); - } - - /** - * Updates the global checkpoint on a replica shard after it has been updated by the primary. - * - * @param globalCheckpoint the global checkpoint - * @param reason the reason the global checkpoint was updated - */ - public void updateGlobalCheckpointOnReplica(final long globalCheckpoint, final String reason) { - globalCheckpointTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, reason); - } - - /** - * Returns the local checkpoint information tracked for a specific shard. Used by tests. - */ - public synchronized long getTrackedLocalCheckpointForShard(final String allocationId) { - return globalCheckpointTracker.getTrackedLocalCheckpointForShard(allocationId).getLocalCheckpoint(); - } - - /** - * Activates the global checkpoint tracker in primary mode (see {@link GlobalCheckpointTracker#primaryMode}. - * Called on primary activation or promotion. - */ - public void activatePrimaryMode(final long localCheckpoint) { - globalCheckpointTracker.activatePrimaryMode(localCheckpoint); - } - - /** - * Notifies the service of the current allocation IDs in the cluster state. See - * {@link GlobalCheckpointTracker#updateFromMaster(long, Set, IndexShardRoutingTable, Set)} for details. - * - * @param applyingClusterStateVersion the cluster state version being applied when updating the allocation IDs from the master - * @param inSyncAllocationIds the allocation IDs of the currently in-sync shard copies - * @param routingTable the shard routing table - * @param pre60AllocationIds the allocation IDs of shards that are allocated to pre-6.0 nodes - */ - public void updateAllocationIdsFromMaster( - final long applyingClusterStateVersion, final Set inSyncAllocationIds, final IndexShardRoutingTable routingTable, - final Set pre60AllocationIds) { - globalCheckpointTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable, - pre60AllocationIds); - } - - /** - * Activates the global checkpoint tracker in primary mode (see {@link GlobalCheckpointTracker#primaryMode}. - * Called on primary relocation target during primary relocation handoff. - * - * @param primaryContext the primary context used to initialize the state - */ - public void activateWithPrimaryContext(final GlobalCheckpointTracker.PrimaryContext primaryContext) { - globalCheckpointTracker.activateWithPrimaryContext(primaryContext); - } - - /** - * Check if there are any recoveries pending in-sync. - * - * @return {@code true} if there is at least one shard pending in-sync, otherwise false - */ - public boolean pendingInSync() { - return globalCheckpointTracker.pendingInSync(); - } - - /** - * Get the primary context for the shard. This includes the state of the global checkpoint tracker. - * - * @return the primary context - */ - public GlobalCheckpointTracker.PrimaryContext startRelocationHandoff() { - return globalCheckpointTracker.startRelocationHandoff(); - } - - /** - * Marks a relocation handoff attempt as successful. Moves the tracker into replica mode. - */ - public void completeRelocationHandoff() { - globalCheckpointTracker.completeRelocationHandoff(); - } - - /** - * Fails a relocation handoff attempt. - */ - public void abortRelocationHandoff() { - globalCheckpointTracker.abortRelocationHandoff(); - } - -} 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 444f3f56d9ab5..f08accdc89be6 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -108,7 +108,6 @@ import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; -import org.elasticsearch.index.seqno.SequenceNumbersService; import org.elasticsearch.index.shard.PrimaryReplicaSyncer.ResyncTask; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; @@ -190,6 +189,8 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl private final SearchOperationListener searchOperationListener; + private final GlobalCheckpointTracker globalCheckpointTracker; + protected volatile ShardRouting shardRouting; protected volatile IndexShardState state; protected volatile long primaryTerm; @@ -296,6 +297,8 @@ public IndexShard( this.checkIndexOnStartup = indexSettings.getValue(IndexSettings.INDEX_CHECK_ON_STARTUP); this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, bigArrays); + this.globalCheckpointTracker = new GlobalCheckpointTracker(shardId, shardRouting.allocationId().getId(), indexSettings, + SequenceNumbers.UNASSIGNED_SEQ_NO); // the query cache is a node-level thing, however we want the most popular filters // to be computed on a per-shard basis if (IndexModule.INDEX_QUERY_CACHE_EVERYTHING_SETTING.get(settings)) { @@ -398,10 +401,7 @@ public void updateShardState(final ShardRouting newRouting, } if (newRouting.primary()) { - final Engine engine = getEngineOrNull(); - if (engine != null) { - engine.seqNoService().updateAllocationIdsFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable, pre60AllocationIds); - } + globalCheckpointTracker.updateFromMaster(applyingClusterStateVersion, inSyncAllocationIds, routingTable, pre60AllocationIds); } if (state == IndexShardState.POST_RECOVERY && newRouting.active()) { @@ -414,7 +414,7 @@ public void updateShardState(final ShardRouting newRouting, } if (newRouting.primary() && currentRouting.isRelocationTarget() == false) { - getEngine().seqNoService().activatePrimaryMode(getEngine().seqNoService().getLocalCheckpoint()); + globalCheckpointTracker.activatePrimaryMode(getEngine().getLocalCheckpointTracker().getCheckpoint()); } changeState(IndexShardState.STARTED, "global state is [" + newRouting.state() + "]"); @@ -489,8 +489,8 @@ public void updateShardState(final ShardRouting newRouting, */ engine.rollTranslogGeneration(); engine.fillSeqNoGaps(newPrimaryTerm); - engine.seqNoService().updateLocalCheckpointForShard(currentRouting.allocationId().getId(), - getEngine().seqNoService().getLocalCheckpoint()); + globalCheckpointTracker.updateLocalCheckpoint(currentRouting.allocationId().getId(), + getEngine().getLocalCheckpointTracker().getCheckpoint()); primaryReplicaSyncer.accept(this, new ActionListener() { @Override public void onResponse(ResyncTask resyncTask) { @@ -516,7 +516,7 @@ public void onFailure(Exception e) { } }, e -> failShard("exception during primary term transition", e)); - getEngine().seqNoService().activatePrimaryMode(getEngine().seqNoService().getLocalCheckpoint()); + globalCheckpointTracker.activatePrimaryMode(getEngine().getLocalCheckpointTracker().getCheckpoint()); primaryTerm = newPrimaryTerm; } } @@ -582,17 +582,17 @@ public void relocated( * network operation. Doing this under the mutex can implicitly block the cluster state update thread on network operations. */ verifyRelocatingState(); - final GlobalCheckpointTracker.PrimaryContext primaryContext = getEngine().seqNoService().startRelocationHandoff(); + final GlobalCheckpointTracker.PrimaryContext primaryContext = globalCheckpointTracker.startRelocationHandoff(); try { consumer.accept(primaryContext); synchronized (mutex) { verifyRelocatingState(); changeState(IndexShardState.RELOCATED, reason); } - getEngine().seqNoService().completeRelocationHandoff(); + globalCheckpointTracker.completeRelocationHandoff(); } catch (final Exception e) { try { - getEngine().seqNoService().abortRelocationHandoff(); + globalCheckpointTracker.abortRelocationHandoff(); } catch (final Exception inner) { e.addSuppressed(inner); } @@ -909,7 +909,7 @@ public CommitStats commitStats() { @Nullable public SeqNoStats seqNoStats() { Engine engine = getEngineOrNull(); - return engine == null ? null : engine.seqNoService().stats(); + return engine == null ? null : engine.getLocalCheckpointTracker().getStats(globalCheckpointTracker.getGlobalCheckpoint()); } public IndexingStats indexingStats(String... types) { @@ -1709,7 +1709,7 @@ public void writeIndexingBuffer() { public void updateLocalCheckpointForShard(final String allocationId, final long checkpoint) { verifyPrimary(); verifyNotClosed(); - getEngine().seqNoService().updateLocalCheckpointForShard(allocationId, checkpoint); + globalCheckpointTracker.updateLocalCheckpoint(allocationId, checkpoint); } /** @@ -1721,7 +1721,7 @@ public void updateLocalCheckpointForShard(final String allocationId, final long public void updateGlobalCheckpointForShard(final String allocationId, final long globalCheckpoint) { verifyPrimary(); verifyNotClosed(); - getEngine().seqNoService().updateGlobalCheckpointForShard(allocationId, globalCheckpoint); + globalCheckpointTracker.updateGlobalCheckpointForShard(allocationId, globalCheckpoint); } /** @@ -1731,7 +1731,7 @@ public void updateGlobalCheckpointForShard(final String allocationId, final long * @throws InterruptedException if the thread was interrupted while blocking on the condition */ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { - getEngine().seqNoService().waitForOpsToComplete(seqNo); + getEngine().getLocalCheckpointTracker().waitForOpsToComplete(seqNo); } /** @@ -1742,7 +1742,7 @@ public void waitForOpsToComplete(final long seqNo) throws InterruptedException { */ public void initiateTracking(final String allocationId) { verifyPrimary(); - getEngine().seqNoService().initiateTracking(allocationId); + globalCheckpointTracker.initiateTracking(allocationId); } /** @@ -1755,7 +1755,7 @@ public void initiateTracking(final String allocationId) { */ public void markAllocationIdAsInSync(final String allocationId, final long localCheckpoint) throws InterruptedException { verifyPrimary(); - getEngine().seqNoService().markAllocationIdAsInSync(allocationId, localCheckpoint); + globalCheckpointTracker.markAllocationIdAsInSync(allocationId, localCheckpoint); } /** @@ -1764,7 +1764,7 @@ public void markAllocationIdAsInSync(final String allocationId, final long local * @return the local checkpoint */ public long getLocalCheckpoint() { - return getEngine().seqNoService().getLocalCheckpoint(); + return getEngine().getLocalCheckpointTracker().getCheckpoint(); } /** @@ -1773,7 +1773,7 @@ public long getLocalCheckpoint() { * @return the global checkpoint */ public long getGlobalCheckpoint() { - return getEngine().seqNoService().getGlobalCheckpoint(); + return globalCheckpointTracker.getGlobalCheckpoint(); } /** @@ -1784,7 +1784,7 @@ public long getGlobalCheckpoint() { public ObjectLongMap getInSyncGlobalCheckpoints() { verifyPrimary(); verifyNotClosed(); - return getEngine().seqNoService().getInSyncGlobalCheckpoints(); + return globalCheckpointTracker.getInSyncGlobalCheckpoints(); } /** @@ -1798,7 +1798,7 @@ public void maybeSyncGlobalCheckpoint(final String reason) { return; } // only sync if there are not operations in flight - final SeqNoStats stats = getEngine().seqNoService().stats(); + final SeqNoStats stats = getEngine().getLocalCheckpointTracker().getStats(globalCheckpointTracker.getGlobalCheckpoint()); if (stats.getMaxSeqNo() == stats.getGlobalCheckpoint()) { final ObjectLongMap globalCheckpoints = getInSyncGlobalCheckpoints(); final String allocationId = routingEntry().allocationId().getId(); @@ -1824,7 +1824,7 @@ public void maybeSyncGlobalCheckpoint(final String reason) { public ReplicationGroup getReplicationGroup() { verifyPrimary(); verifyNotClosed(); - return getEngine().seqNoService().getReplicationGroup(); + return globalCheckpointTracker.getReplicationGroup(); } /** @@ -1835,8 +1835,7 @@ public ReplicationGroup getReplicationGroup() { */ public void updateGlobalCheckpointOnReplica(final long globalCheckpoint, final String reason) { verifyReplicationTarget(); - final SequenceNumbersService seqNoService = getEngine().seqNoService(); - final long localCheckpoint = seqNoService.getLocalCheckpoint(); + final long localCheckpoint = getEngine().getLocalCheckpointTracker().getCheckpoint(); if (globalCheckpoint > localCheckpoint) { /* * This can happen during recovery when the shard has started its engine but recovery is not finalized and is receiving global @@ -1853,7 +1852,7 @@ assert state() != IndexShardState.POST_RECOVERY && state() != IndexShardState.ST "that is higher than its local checkpoint [" + localCheckpoint + "]"; return; } - seqNoService.updateGlobalCheckpointOnReplica(globalCheckpoint, reason); + globalCheckpointTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, reason); } /** @@ -1865,9 +1864,9 @@ public void activateWithPrimaryContext(final GlobalCheckpointTracker.PrimaryCont verifyPrimary(); assert shardRouting.isRelocationTarget() : "only relocation target can update allocation IDs from primary context: " + shardRouting; assert primaryContext.getCheckpointStates().containsKey(routingEntry().allocationId().getId()) && - getEngine().seqNoService().getLocalCheckpoint() == + getEngine().getLocalCheckpointTracker().getCheckpoint() == primaryContext.getCheckpointStates().get(routingEntry().allocationId().getId()).getLocalCheckpoint(); - getEngine().seqNoService().activateWithPrimaryContext(primaryContext); + globalCheckpointTracker.activateWithPrimaryContext(primaryContext); } /** @@ -1877,7 +1876,7 @@ public void activateWithPrimaryContext(final GlobalCheckpointTracker.PrimaryCont */ public boolean pendingInSync() { verifyPrimary(); - return getEngine().seqNoService().pendingInSync(); + return globalCheckpointTracker.pendingInSync(); } /** @@ -2189,7 +2188,7 @@ private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode) { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService); + indexSort, this::runTranslogRecovery, circuitBreakerService, globalCheckpointTracker); } /** @@ -2254,7 +2253,7 @@ public void acquireReplicaOperationPermit(final long operationPrimaryTerm, final operationPrimaryTerm, getLocalCheckpoint(), localCheckpoint); - getEngine().seqNoService().resetLocalCheckpoint(localCheckpoint); + getEngine().getLocalCheckpointTracker().resetCheckpoint(localCheckpoint); getEngine().getTranslog().rollGeneration(); }); globalCheckpointUpdated = true; diff --git a/core/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java b/core/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java index f108300b95bc2..e156e988c8700 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java +++ b/core/src/main/java/org/elasticsearch/index/shard/LocalShardSnapshot.java @@ -62,7 +62,7 @@ Index getIndex() { } long maxSeqNo() { - return shard.getEngine().seqNoService().getMaxSeqNo(); + return shard.getEngine().getLocalCheckpointTracker().getMaxSeqNo(); } long maxUnsafeAutoIdTimestamp() { From e0ede9c7e2054caa861dd219ba8785e4da0143a5 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 15 Dec 2017 11:22:18 +0100 Subject: [PATCH 02/11] fix tests --- .../elasticsearch/index/shard/IndexShard.java | 5 + .../index/engine/InternalEngineTests.java | 221 +++++++----------- .../RecoveryDuringReplicationTests.java | 5 +- .../index/shard/IndexShardTests.java | 16 +- .../index/shard/RefreshListenersTests.java | 5 +- .../index/engine/EngineTestCase.java | 37 +-- .../index/shard/IndexShardTestCase.java | 5 + 7 files changed, 128 insertions(+), 166 deletions(-) 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 f08accdc89be6..83661ed310770 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2455,6 +2455,11 @@ EngineFactory getEngineFactory() { return engineFactory; } + // for tests + GlobalCheckpointTracker getGlobalCheckpointTracker() { + return globalCheckpointTracker; + } + /** * Executes a scheduled refresh if necessary. * 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 999b4dca563fb..47aa656cd9f2b 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -106,9 +106,9 @@ import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +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.IndexSearcherWrapper; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; @@ -148,6 +148,7 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.function.ToLongBiFunction; @@ -500,27 +501,18 @@ public void testCommitStats() throws IOException { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); try ( Store store = createStore(); - InternalEngine engine = createEngine(store, createTempDir(), (config, seqNoStats) -> new SequenceNumbersService( - config.getShardId(), - config.getAllocationId(), - config.getIndexSettings(), + InternalEngine engine = createEngine(store, createTempDir(), seqNoStats -> new LocalCheckpointTracker( seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()) { + seqNoStats.getLocalCheckpoint()) { @Override public long getMaxSeqNo() { return maxSeqNo.get(); } @Override - public long getLocalCheckpoint() { + public long getCheckpoint() { return localCheckpoint.get(); } - - @Override - public long getGlobalCheckpoint() { - return globalCheckpoint.get(); - } } )) { CommitStats stats1 = engine.commitStats(); @@ -691,7 +683,7 @@ public void testTranslogRecoveryWithMultipleGenerations() throws IOException { initialEngine = createEngine( store, createTempDir(), - InternalEngine::sequenceNumberService, + InternalEngine::localCheckpointTracker, (engine, operation) -> seqNos.get(counter.getAndIncrement())); for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); @@ -951,40 +943,33 @@ public void testCommitAdvancesMinTranslogForRecovery() throws IOException { IOUtils.close(engine, store); final Path translogPath = createTempDir(); store = createStore(); - final AtomicBoolean inSync = new AtomicBoolean(randomBoolean()); - final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> - new SequenceNumbersService( - config.getShardId(), - config.getAllocationId(), - config.getIndexSettings(), - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()) { - @Override - public long getGlobalCheckpoint() { - return inSync.get() ? getLocalCheckpoint() : SequenceNumbers.UNASSIGNED_SEQ_NO; - } - }; - engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier); + boolean inSync = randomBoolean(); + engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null)); ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); + if (inSync) { + engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( + engine.getLocalCheckpointTracker().getCheckpoint(), "initially in-sync"); + } engine.flush(); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 2L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 2L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(2L)); engine.flush(); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 2L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 2L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(2L)); engine.flush(true, true); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync.get() ? 3L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); - inSync.set(true); + inSync = true; + engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( + engine.getLocalCheckpointTracker().getCheckpoint(), "now in-sync"); engine.flush(true, true); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(4L)); assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(4L)); @@ -1912,10 +1897,10 @@ public void testSeqNoAndCheckpoints() throws IOException { initialEngine = engine; final ShardRouting primary = TestShardRouting.newShardRouting("test", shardId.id(), "node1", null, true, ShardRoutingState.STARTED, allocationId); final ShardRouting replica = TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED); - initialEngine.seqNoService().updateAllocationIdsFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), + initialEngine.config().getGlobalCheckpointTracker().updateFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), replica.allocationId().getId())), new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build(), Collections.emptySet()); - initialEngine.seqNoService().activatePrimaryMode(primarySeqNo); + initialEngine.config().getGlobalCheckpointTracker().activatePrimaryMode(primarySeqNo); for (int op = 0; op < opCount; op++) { final String id; // mostly index, sometimes delete @@ -1928,12 +1913,12 @@ public void testSeqNoAndCheckpoints() throws IOException { final Engine.DeleteResult result = initialEngine.delete(delete); if (!result.hasFailure()) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo(primarySeqNo + 1)); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(primarySeqNo + 1)); indexedIds.remove(id); primarySeqNo++; } else { assertThat(result.getSeqNo(), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO)); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo(primarySeqNo)); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(primarySeqNo)); } } else { // index a document @@ -1946,12 +1931,12 @@ public void testSeqNoAndCheckpoints() throws IOException { final Engine.IndexResult result = initialEngine.index(index); if (!result.hasFailure()) { assertThat(result.getSeqNo(), equalTo(primarySeqNo + 1)); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo(primarySeqNo + 1)); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(primarySeqNo + 1)); indexedIds.add(id); primarySeqNo++; } else { assertThat(result.getSeqNo(), equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO)); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo(primarySeqNo)); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(primarySeqNo)); } } @@ -1959,9 +1944,9 @@ public void testSeqNoAndCheckpoints() throws IOException { // only update rarely as we do it every doc replicaLocalCheckpoint = randomIntBetween(Math.toIntExact(replicaLocalCheckpoint), Math.toIntExact(primarySeqNo)); } - initialEngine.seqNoService().updateLocalCheckpointForShard(primary.allocationId().getId(), - initialEngine.seqNoService().getLocalCheckpoint()); - initialEngine.seqNoService().updateLocalCheckpointForShard(replica.allocationId().getId(), replicaLocalCheckpoint); + initialEngine.config().getGlobalCheckpointTracker().updateLocalCheckpoint(primary.allocationId().getId(), + initialEngine.getLocalCheckpointTracker().getCheckpoint()); + initialEngine.config().getGlobalCheckpointTracker().updateLocalCheckpoint(replica.allocationId().getId(), replicaLocalCheckpoint); if (rarely()) { localCheckpoint = primarySeqNo; @@ -1971,12 +1956,11 @@ public void testSeqNoAndCheckpoints() throws IOException { } logger.info("localcheckpoint {}, global {}", replicaLocalCheckpoint, primarySeqNo); - globalCheckpoint = initialEngine.seqNoService().getGlobalCheckpoint(); + globalCheckpoint = initialEngine.config().getGlobalCheckpointTracker().getGlobalCheckpoint(); - assertEquals(primarySeqNo, initialEngine.seqNoService().getMaxSeqNo()); - assertThat(initialEngine.seqNoService().stats().getMaxSeqNo(), equalTo(primarySeqNo)); - assertThat(initialEngine.seqNoService().stats().getLocalCheckpoint(), equalTo(primarySeqNo)); - assertThat(initialEngine.seqNoService().stats().getGlobalCheckpoint(), equalTo(replicaLocalCheckpoint)); + assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getCheckpoint()); + assertThat(initialEngine.config().getGlobalCheckpointTracker().getGlobalCheckpoint(), equalTo(replicaLocalCheckpoint)); assertThat( Long.parseLong(initialEngine.commitStats().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), @@ -1998,7 +1982,7 @@ public void testSeqNoAndCheckpoints() throws IOException { recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); recoveringEngine.recoverFromTranslog(); - assertEquals(primarySeqNo, recoveringEngine.seqNoService().getMaxSeqNo()); + assertEquals(primarySeqNo, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); assertThat( Long.parseLong(recoveringEngine.commitStats().getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)), equalTo(primarySeqNo)); @@ -2011,9 +1995,9 @@ public void testSeqNoAndCheckpoints() throws IOException { // that the committed max seq no is equivalent to what the current primary seq no is, as all data // we have assigned sequence numbers to should be in the commit equalTo(primarySeqNo)); - assertThat(recoveringEngine.seqNoService().stats().getLocalCheckpoint(), equalTo(primarySeqNo)); - assertThat(recoveringEngine.seqNoService().stats().getMaxSeqNo(), equalTo(primarySeqNo)); - assertThat(recoveringEngine.seqNoService().generateSeqNo(), equalTo(primarySeqNo + 1)); + assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo(primarySeqNo)); + assertThat(recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo(primarySeqNo)); + assertThat(recoveringEngine.getLocalCheckpointTracker().generateSeqNo(), equalTo(primarySeqNo + 1)); } finally { IOUtils.close(recoveringEngine); } @@ -2392,25 +2376,11 @@ public void testTranslogCleanUpPostCommitCrash() throws Exception { ); indexSettings.updateIndexMetaData(builder.build()); - final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> - new SequenceNumbersService( - config.getShardId(), - config.getAllocationId(), - config.getIndexSettings(), - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()) { - @Override - public long getGlobalCheckpoint() { - return getLocalCheckpoint(); - } - }; - try (Store store = createStore()) { AtomicBoolean throwErrorOnCommit = new AtomicBoolean(); final Path translogPath = createTempDir(); try (InternalEngine engine = - new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier) { + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null)) { @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { @@ -2422,12 +2392,14 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s }) { final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc1)); + engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( + engine.getLocalCheckpointTracker().getCheckpoint(), "manual update"); throwErrorOnCommit.set(true); FlushFailedEngineException e = expectThrows(FlushFailedEngineException.class, engine::flush); assertThat(e.getCause().getMessage(), equalTo("power's out")); } try (InternalEngine engine = - new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null), seqNoServiceSupplier)) { + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null))) { engine.recoverFromTranslog(); assertVisibleCount(engine, 1); final long committedGen = Long.valueOf( @@ -2579,7 +2551,7 @@ public void testRecoverFromForeignTranslog() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService()); + new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2633,7 +2605,7 @@ threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService()); + new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); engine = new InternalEngine(newConfig); if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { engine.recoverFromTranslog(); @@ -2664,7 +2636,7 @@ public void testHistoryUUIDCanBeForced() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), true, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService()); + new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); if (newConfig.getOpenMode() == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { Lucene.cleanLuceneIndex(store.directory()); } @@ -3544,7 +3516,7 @@ private ToLongBiFunction getStallingSeqNoGenerator( final AtomicBoolean stall, final AtomicLong expectedLocalCheckpoint) { return (engine, operation) -> { - final long seqNo = engine.seqNoService().generateSeqNo(); + final long seqNo = engine.getLocalCheckpointTracker().generateSeqNo(); final CountDownLatch latch = latchReference.get(); if (stall.get()) { try { @@ -3573,7 +3545,7 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final List threads = new ArrayList<>(); initialEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::sequenceNumberService, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalInitialEngine = initialEngine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); @@ -3596,8 +3568,8 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro } } - assertThat(initialEngine.seqNoService().getLocalCheckpoint(), equalTo(expectedLocalCheckpoint.get())); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo((long) (docs - 1))); + assertThat(initialEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo(expectedLocalCheckpoint.get())); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo((long) (docs - 1))); initialEngine.flush(true, true); latchReference.get().countDown(); @@ -3612,7 +3584,7 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { recoveringEngine.recoverFromTranslog(); recoveringEngine.fillSeqNoGaps(2); - assertThat(recoveringEngine.seqNoService().getLocalCheckpoint(), greaterThanOrEqualTo((long) (docs - 1))); + assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (docs - 1))); } } @@ -3634,8 +3606,8 @@ public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws I } // bake the commit with the local checkpoint stuck at 0 and gaps all along the way up to the max sequence number - assertThat(initialEngine.seqNoService().getLocalCheckpoint(), equalTo((long) 0)); - assertThat(initialEngine.seqNoService().getMaxSeqNo(), equalTo((long) (3 * (docs - 1) + 2))); + assertThat(initialEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo((long) 0)); + assertThat(initialEngine.getLocalCheckpointTracker().getMaxSeqNo(), equalTo((long) (3 * (docs - 1) + 2))); initialEngine.flush(true, true); for (int i = 0; i < docs; i++) { @@ -3652,7 +3624,7 @@ public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws I new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { recoveringEngine.recoverFromTranslog(); recoveringEngine.fillSeqNoGaps(1); - assertThat(recoveringEngine.seqNoService().getLocalCheckpoint(), greaterThanOrEqualTo((long) (3 * (docs - 1) + 2 - 1))); + assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (3 * (docs - 1) + 2 - 1))); } } @@ -3730,7 +3702,7 @@ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOExceptio expectedLocalCheckpoint = numberOfOperations - 1; } - assertThat(engine.seqNoService().getLocalCheckpoint(), equalTo(expectedLocalCheckpoint)); + assertThat(engine.getLocalCheckpointTracker().getCheckpoint(), equalTo(expectedLocalCheckpoint)); try (Engine.GetResult result = engine.get(new Engine.Get(true, "type", "2", uid), searcherFactory)) { assertThat(result.exists(), equalTo(exists)); } @@ -3745,15 +3717,10 @@ public void testNoOps() throws IOException { InternalEngine noOpEngine = null; final int maxSeqNo = randomIntBetween(0, 128); final int localCheckpoint = randomIntBetween(0, maxSeqNo); - final int globalCheckpoint = randomIntBetween(0, localCheckpoint); try { - final BiFunction supplier = (engineConfig, ignored) -> new SequenceNumbersService( - engineConfig.getShardId(), - engineConfig.getAllocationId(), - engineConfig.getIndexSettings(), + final Function supplier = seqNoStats -> new LocalCheckpointTracker( maxSeqNo, - localCheckpoint, - globalCheckpoint); + localCheckpoint); noOpEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { @@ -3771,7 +3738,7 @@ protected long doGenerateSeqNoForOperation(Operation operation) { randomFrom(PRIMARY, REPLICA, PEER_RECOVERY, LOCAL_TRANSLOG_RECOVERY), System.nanoTime(), reason)); - assertThat(noOpEngine.seqNoService().getLocalCheckpoint(), equalTo((long) (maxSeqNo + 1))); + assertThat(noOpEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo((long) (maxSeqNo + 1))); assertThat(noOpEngine.getTranslog().uncommittedOperations(), equalTo(1 + gapsFilled)); // skip to the op that we added to the translog Translog.Operation op; @@ -3803,7 +3770,7 @@ public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierExcepti final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final Map threads = new LinkedHashMap<>(); actualEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::sequenceNumberService, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalActualEngine = actualEngine; final Translog translog = finalActualEngine.getTranslog(); final long generation = finalActualEngine.getTranslog().currentFileGeneration(); @@ -3892,13 +3859,9 @@ public void testRestoreLocalCheckpointFromTranslog() throws IOException { InternalEngine actualEngine = null; try { final Set completedSeqNos = new HashSet<>(); - final BiFunction supplier = (engineConfig, seqNoStats) -> new SequenceNumbersService( - engineConfig.getShardId(), - engineConfig.getAllocationId(), - engineConfig.getIndexSettings(), + final Function supplier = seqNoStats -> new LocalCheckpointTracker( seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()) { + seqNoStats.getLocalCheckpoint()) { @Override public void markSeqNoAsCompleted(long seqNo) { super.markSeqNoAsCompleted(seqNo); @@ -3927,17 +3890,17 @@ public void markSeqNoAsCompleted(long seqNo) { actualEngine.rollTranslogGeneration(); } } - final long currentLocalCheckpoint = actualEngine.seqNoService().getLocalCheckpoint(); + final long currentLocalCheckpoint = actualEngine.getLocalCheckpointTracker().getCheckpoint(); final long resetLocalCheckpoint = randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Math.toIntExact(currentLocalCheckpoint)); - actualEngine.seqNoService().resetLocalCheckpoint(resetLocalCheckpoint); + actualEngine.getLocalCheckpointTracker().resetCheckpoint(resetLocalCheckpoint); completedSeqNos.clear(); actualEngine.restoreLocalCheckpointFromTranslog(); final Set intersection = new HashSet<>(expectedCompletedSeqNos); intersection.retainAll(LongStream.range(resetLocalCheckpoint + 1, operations).boxed().collect(Collectors.toSet())); assertThat(completedSeqNos, equalTo(intersection)); - assertThat(actualEngine.seqNoService().getLocalCheckpoint(), equalTo(currentLocalCheckpoint)); - assertThat(actualEngine.seqNoService().generateSeqNo(), equalTo((long) operations)); + assertThat(actualEngine.getLocalCheckpointTracker().getCheckpoint(), equalTo(currentLocalCheckpoint)); + assertThat(actualEngine.getLocalCheckpointTracker().generateSeqNo(), equalTo((long) operations)); } finally { IOUtils.close(actualEngine); } @@ -3961,7 +3924,7 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { replicaEngine.index(replicaIndexForDoc(doc, 1, indexResult.getSeqNo(), false)); } } - checkpointOnReplica = replicaEngine.seqNoService().getLocalCheckpoint(); + checkpointOnReplica = replicaEngine.getLocalCheckpointTracker().getCheckpoint(); } finally { IOUtils.close(replicaEngine); } @@ -3970,15 +3933,15 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { boolean flushed = false; Engine recoveringEngine = null; try { - assertEquals(docs - 1, engine.seqNoService().getMaxSeqNo()); - assertEquals(docs - 1, engine.seqNoService().getLocalCheckpoint()); - assertEquals(maxSeqIDOnReplica, replicaEngine.seqNoService().getMaxSeqNo()); - assertEquals(checkpointOnReplica, replicaEngine.seqNoService().getLocalCheckpoint()); + assertEquals(docs - 1, engine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(docs - 1, engine.getLocalCheckpointTracker().getCheckpoint()); + assertEquals(maxSeqIDOnReplica, replicaEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(checkpointOnReplica, replicaEngine.getLocalCheckpointTracker().getCheckpoint()); recoveringEngine = new InternalEngine(copy(replicaEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); assertEquals(numDocsOnReplica, recoveringEngine.getTranslog().uncommittedOperations()); recoveringEngine.recoverFromTranslog(); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getMaxSeqNo()); - assertEquals(checkpointOnReplica, recoveringEngine.seqNoService().getLocalCheckpoint()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(checkpointOnReplica, recoveringEngine.getLocalCheckpointTracker().getCheckpoint()); assertEquals((maxSeqIDOnReplica + 1) - numDocsOnReplica, recoveringEngine.fillSeqNoGaps(2)); // now snapshot the tlog and ensure the primary term is updated @@ -3993,8 +3956,8 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { } } - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getMaxSeqNo()); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getLocalCheckpoint()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getCheckpoint()); if ((flushed = randomBoolean())) { recoveringEngine.flush(true, true); } @@ -4010,11 +3973,11 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { assertEquals(0, recoveringEngine.getTranslog().uncommittedOperations()); } recoveringEngine.recoverFromTranslog(); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getMaxSeqNo()); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getLocalCheckpoint()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getCheckpoint()); assertEquals(0, recoveringEngine.fillSeqNoGaps(3)); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getMaxSeqNo()); - assertEquals(maxSeqIDOnReplica, recoveringEngine.seqNoService().getLocalCheckpoint()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); + assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getCheckpoint()); } finally { IOUtils.close(recoveringEngine); } @@ -4106,13 +4069,9 @@ public void testRefreshScopedSearcher() throws IOException { public void testSeqNoGenerator() throws IOException { engine.close(); final long seqNo = randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Integer.MAX_VALUE); - final BiFunction seqNoService = (config, seqNoStats) -> new SequenceNumbersService( - config.getShardId(), - config.getAllocationId(), - config.getIndexSettings(), - SequenceNumbers.NO_OPS_PERFORMED, + final Function seqNoService = seqNoStats -> new LocalCheckpointTracker( SequenceNumbers.NO_OPS_PERFORMED, - SequenceNumbers.UNASSIGNED_SEQ_NO); + SequenceNumbers.NO_OPS_PERFORMED); final AtomicLong seqNoGenerator = new AtomicLong(seqNo); try (Engine e = createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, seqNoService, (engine, operation) -> seqNoGenerator.getAndIncrement())) { final String id = "id"; @@ -4171,21 +4130,6 @@ public void testSeqNoGenerator() throws IOException { public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { IOUtils.close(engine, store); - final AtomicLong globalCheckpoint = new AtomicLong(0); - final BiFunction seqNoServiceSupplier = (config, seqNoStats) -> - new SequenceNumbersService( - config.getShardId(), - config.getAllocationId(), - config.getIndexSettings(), - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint(), - seqNoStats.getGlobalCheckpoint()) { - @Override - public long getGlobalCheckpoint() { - return globalCheckpoint.get(); - } - }; - final IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(), defaultSettings.getScopedSettings()); IndexMetaData.Builder builder = IndexMetaData.builder(indexSettings.getIndexMetaData()) @@ -4196,16 +4140,17 @@ public long getGlobalCheckpoint() { store = createStore(); try (InternalEngine engine - = createEngine(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, seqNoServiceSupplier)) { + = createEngine(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null)) { + engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(0L, "initially set"); int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); engine.index(indexForDoc(testParsedDocument(Integer.toString(docId), null, document, B_1, null))); if (frequently()) { - globalCheckpoint.set(randomIntBetween( - Math.toIntExact(engine.seqNoService().getGlobalCheckpoint()), - Math.toIntExact(engine.seqNoService().getLocalCheckpoint()))); + engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(randomIntBetween( + Math.toIntExact(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint()), + Math.toIntExact(engine.getLocalCheckpointTracker().getCheckpoint())), "updated"); } if (frequently()) { engine.flush(randomBoolean(), true); @@ -4213,10 +4158,10 @@ public long getGlobalCheckpoint() { // Keep only one safe commit as the oldest commit. final IndexCommit safeCommit = commits.get(0); assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), - lessThanOrEqualTo(globalCheckpoint.get())); + lessThanOrEqualTo(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint())); for (int i = 1; i < commits.size(); i++) { assertThat(Long.parseLong(commits.get(i).getUserData().get(SequenceNumbers.MAX_SEQ_NO)), - greaterThan(globalCheckpoint.get())); + greaterThan(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint())); } // Make sure we keep all translog operations after the local checkpoint of the safe commit. long localCheckpointFromSafeCommit = Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); 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 f4e646030f229..8f18ccd8c9ae5 100644 --- a/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -247,8 +247,9 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { // check that local checkpoint of new primary is properly tracked after primary promotion assertThat(newPrimary.getLocalCheckpoint(), equalTo(totalDocs - 1L)); - assertThat(IndexShardTestCase.getEngine(newPrimary).seqNoService() - .getTrackedLocalCheckpointForShard(newPrimary.routingEntry().allocationId().getId()), equalTo(totalDocs - 1L)); + assertThat(IndexShardTestCase.getGlobalCheckpointTracker(newPrimary) + .getTrackedLocalCheckpointForShard(newPrimary.routingEntry().allocationId().getId()).getLocalCheckpoint(), + equalTo(totalDocs - 1L)); // index some more totalDocs += shards.indexDocs(randomIntBetween(0, 5)); diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 1410d4978b14b..b0b597f908573 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -443,7 +443,7 @@ public void testPublishingOrderOnPromotion() throws IOException, BrokenBarrierEx while(stop.get() == false) { if (indexShard.routingEntry().primary()) { assertThat(indexShard.getPrimaryTerm(), equalTo(promotedTerm)); - assertThat(indexShard.getEngine().seqNoService().getReplicationGroup(), notNullValue()); + assertThat(indexShard.getReplicationGroup(), notNullValue()); } } }); @@ -839,7 +839,7 @@ public void testGlobalCheckpointSync() throws IOException { recoverReplica(replicaShard, primaryShard); final int maxSeqNo = randomIntBetween(0, 128); for (int i = 0; i <= maxSeqNo; i++) { - primaryShard.getEngine().seqNoService().generateSeqNo(); + primaryShard.getEngine().getLocalCheckpointTracker().generateSeqNo(); } final long checkpoint = rarely() ? maxSeqNo - scaledRandomIntBetween(0, maxSeqNo) : maxSeqNo; @@ -1607,8 +1607,8 @@ public void testRecoverFromStore() throws IOException { IndexShardTestCase.updateRoutingEntry(newShard, newShard.routingEntry().moveToStarted()); // check that local checkpoint of new primary is properly tracked after recovery assertThat(newShard.getLocalCheckpoint(), equalTo(totalOps - 1L)); - assertThat(IndexShardTestCase.getEngine(newShard).seqNoService() - .getTrackedLocalCheckpointForShard(newShard.routingEntry().allocationId().getId()), equalTo(totalOps - 1L)); + assertThat(newShard.getGlobalCheckpointTracker().getTrackedLocalCheckpointForShard(newShard.routingEntry().allocationId().getId()) + .getLocalCheckpoint(), equalTo(totalOps - 1L)); assertDocCount(newShard, totalOps); closeShards(newShard); } @@ -1626,8 +1626,8 @@ public void testPrimaryHandOffUpdatesLocalCheckpoint() throws IOException { // check that local checkpoint of new primary is properly tracked after primary relocation assertThat(primaryTarget.getLocalCheckpoint(), equalTo(totalOps - 1L)); - assertThat(IndexShardTestCase.getEngine(primaryTarget).seqNoService() - .getTrackedLocalCheckpointForShard(primaryTarget.routingEntry().allocationId().getId()), equalTo(totalOps - 1L)); + assertThat(primaryTarget.getGlobalCheckpointTracker().getTrackedLocalCheckpointForShard( + primaryTarget.routingEntry().allocationId().getId()).getLocalCheckpoint(), equalTo(totalOps - 1L)); assertDocCount(primaryTarget, totalOps); closeShards(primarySource, primaryTarget); } @@ -2246,8 +2246,8 @@ public void testRecoverFromLocalShard() throws IOException { IndexShardTestCase.updateRoutingEntry(targetShard, ShardRoutingHelper.moveToStarted(targetShard.routingEntry())); // check that local checkpoint of new primary is properly tracked after recovery assertThat(targetShard.getLocalCheckpoint(), equalTo(1L)); - assertThat(IndexShardTestCase.getEngine(targetShard).seqNoService() - .getTrackedLocalCheckpointForShard(targetShard.routingEntry().allocationId().getId()), equalTo(1L)); + assertThat(targetShard.getGlobalCheckpointTracker().getTrackedLocalCheckpointForShard( + targetShard.routingEntry().allocationId().getId()).getLocalCheckpoint(), equalTo(1L)); assertDocCount(targetShard, 2); } // now check that it's persistent ie. that the added shards are committed diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 125f45fd00723..b71fcf4b7c24f 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -48,6 +48,8 @@ import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; +import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.TranslogConfig; @@ -123,7 +125,8 @@ public void onFailedEngine(String reason, @Nullable Exception e) { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, null, - new NoneCircuitBreakerService()); + new NoneCircuitBreakerService(), + new GlobalCheckpointTracker(shardId, allocationId, indexSettings, SequenceNumbers.UNASSIGNED_SEQ_NO)); engine = new InternalEngine(config); listeners.setTranslog(engine.getTranslog()); } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index bfc44f71d806c..aeb60df53126b 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -60,9 +60,10 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; +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.ShardId; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; @@ -83,7 +84,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.ToLongBiFunction; import static java.util.Collections.emptyList; @@ -164,7 +165,8 @@ public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, An new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService()); + config.getCircuitBreakerService(), new GlobalCheckpointTracker(config.getShardId(), config.getAllocationId(), + config.getIndexSettings(), SequenceNumbers.UNASSIGNED_SEQ_NO)); } @Override @@ -253,17 +255,17 @@ protected InternalEngine createEngine(Store store, Path translogPath) throws IOE protected InternalEngine createEngine( Store store, Path translogPath, - BiFunction sequenceNumbersServiceSupplier) throws IOException { - return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, sequenceNumbersServiceSupplier); + Function localCheckpointTrackerSupplier) throws IOException { + return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier); } protected InternalEngine createEngine( Store store, Path translogPath, - BiFunction sequenceNumbersServiceSupplier, + Function localCheckpointTrackerSupplier, ToLongBiFunction seqNoForOperation) throws IOException { return createEngine( - defaultSettings, store, translogPath, newMergePolicy(), null, sequenceNumbersServiceSupplier, seqNoForOperation, null); + defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier, seqNoForOperation, null); } protected InternalEngine createEngine( @@ -283,9 +285,9 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable BiFunction sequenceNumbersServiceSupplier) throws IOException { + @Nullable Function localCheckpointTrackerSupplier) throws IOException { return createEngine( - indexSettings, store, translogPath, mergePolicy, indexWriterFactory, sequenceNumbersServiceSupplier, null, null); + indexSettings, store, translogPath, mergePolicy, indexWriterFactory, localCheckpointTrackerSupplier, null, null); } protected InternalEngine createEngine( @@ -294,7 +296,7 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable BiFunction sequenceNumbersServiceSupplier, + @Nullable Function localCheckpointTrackerSupplier, @Nullable ToLongBiFunction seqNoForOperation) throws IOException { return createEngine( indexSettings, @@ -302,7 +304,7 @@ protected InternalEngine createEngine( translogPath, mergePolicy, indexWriterFactory, - sequenceNumbersServiceSupplier, + localCheckpointTrackerSupplier, seqNoForOperation, null); } @@ -313,11 +315,11 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable BiFunction sequenceNumbersServiceSupplier, + @Nullable Function localCheckpointTrackerSupplier, @Nullable ToLongBiFunction seqNoForOperation, @Nullable Sort indexSort) throws IOException { EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null, indexSort); - InternalEngine internalEngine = createInternalEngine(indexWriterFactory, sequenceNumbersServiceSupplier, seqNoForOperation, config); + InternalEngine internalEngine = createInternalEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { internalEngine.recoverFromTranslog(); } @@ -332,10 +334,10 @@ public interface IndexWriterFactory { public static InternalEngine createInternalEngine( @Nullable final IndexWriterFactory indexWriterFactory, - @Nullable final BiFunction sequenceNumbersServiceSupplier, + @Nullable final Function localCheckpointTrackerSupplier, @Nullable final ToLongBiFunction seqNoForOperation, final EngineConfig config) { - if (sequenceNumbersServiceSupplier == null) { + if (localCheckpointTrackerSupplier == null) { return new InternalEngine(config) { @Override IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { @@ -352,7 +354,7 @@ protected long doGenerateSeqNoForOperation(final Operation operation) { } }; } else { - return new InternalEngine(config, sequenceNumbersServiceSupplier) { + return new InternalEngine(config, localCheckpointTrackerSupplier) { @Override IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOException { return (indexWriterFactory != null) ? @@ -404,7 +406,8 @@ public void onFailedEngine(String reason, @Nullable Exception e) { mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), refreshListenerList, Collections.emptyList(), indexSort, handler, - new NoneCircuitBreakerService()); + new NoneCircuitBreakerService(), + new GlobalCheckpointTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.UNASSIGNED_SEQ_NO)); return config; } 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 6e8eefa929780..35f7520852076 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 @@ -60,6 +60,7 @@ import org.elasticsearch.index.mapper.Mapping; import org.elasticsearch.index.mapper.SourceToParse; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.snapshots.IndexShardSnapshotStatus; @@ -633,4 +634,8 @@ protected void snapshotShard(final IndexShard shard, public static Engine getEngine(IndexShard indexShard) { return indexShard.getEngine(); } + + public static GlobalCheckpointTracker getGlobalCheckpointTracker(IndexShard indexShard) { + return indexShard.getGlobalCheckpointTracker(); + } } From c79ea693c941ae6a6d46c3acd6c213693d9530a0 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 15 Dec 2017 11:28:46 +0100 Subject: [PATCH 03/11] fix javadoc --- .../action/support/replication/ReplicationOperation.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index ca3a26b3c301f..660081c5cfb5f 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -399,14 +399,14 @@ void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, Runnable public interface ReplicaResponse { /** - * The local checkpoint for the shard. See {@link SequenceNumbersService#getLocalCheckpoint()}. + * The local checkpoint for the shard. * * @return the local checkpoint **/ long localCheckpoint(); /** - * The global checkpoint for the shard. See {@link SequenceNumbersService#getGlobalCheckpoint()}. + * The global checkpoint for the shard. * * @return the global checkpoint **/ From 48747b2f753937a685a2cd73fc6b598420a25021 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 15 Dec 2017 16:51:30 +0100 Subject: [PATCH 04/11] Remove GlobalCheckpointTracker from EngineConfig --- .../index/engine/EngineConfig.java | 12 +-- .../index/engine/InternalEngine.java | 56 +++++------ .../index/seqno/SequenceNumbers.java | 11 +-- .../elasticsearch/index/shard/IndexShard.java | 11 ++- .../org/elasticsearch/index/store/Store.java | 12 +-- .../index/translog/Translog.java | 16 +-- .../index/translog/TranslogWriter.java | 19 +++- .../recovery/PeerRecoveryTargetService.java | 12 ++- .../index/engine/InternalEngineTests.java | 97 ++++++++----------- .../index/shard/RefreshListenersTests.java | 2 +- .../translog/TranslogDeletionPolicyTests.java | 6 +- .../PeerRecoveryTargetServiceTests.java | 2 +- .../index/engine/EngineTestCase.java | 47 +++++---- 13 files changed, 149 insertions(+), 154 deletions(-) diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 1041a7c42b3e1..30743c18cfe10 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -34,7 +34,6 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.codec.CodecService; -import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; @@ -45,6 +44,7 @@ import java.io.IOException; import java.util.List; +import java.util.function.LongSupplier; /* * Holds all the configuration that is used to create an {@link Engine}. @@ -79,7 +79,7 @@ public final class EngineConfig { private final TranslogRecoveryRunner translogRecoveryRunner; @Nullable private final CircuitBreakerService circuitBreakerService; - private final GlobalCheckpointTracker globalCheckpointTracker; + private final LongSupplier globalCheckpointSupplier; /** * Index setting to change the low level lucene codec used for writing new segments. @@ -127,7 +127,7 @@ public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, Thr List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, - GlobalCheckpointTracker globalCheckpointTracker) { + LongSupplier globalCheckpointSupplier) { if (openMode == null) { throw new IllegalArgumentException("openMode must not be null"); } @@ -158,7 +158,7 @@ public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, Thr this.indexSort = indexSort; this.translogRecoveryRunner = translogRecoveryRunner; this.circuitBreakerService = circuitBreakerService; - this.globalCheckpointTracker = globalCheckpointTracker; + this.globalCheckpointSupplier = globalCheckpointSupplier; } /** @@ -234,8 +234,8 @@ public Store getStore() { /** * Returns the global checkpoint tracker */ - public GlobalCheckpointTracker getGlobalCheckpointTracker() { - return globalCheckpointTracker; + public LongSupplier getGlobalCheckpointSupplier() { + return globalCheckpointSupplier; } /** 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 3cbb1406e520e..274f5bbb3dcbd 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -49,6 +49,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; @@ -68,7 +69,6 @@ import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; @@ -93,7 +93,6 @@ import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; -import java.util.function.Function; import java.util.function.LongSupplier; public class InternalEngine extends Engine { @@ -156,7 +155,7 @@ public InternalEngine(EngineConfig engineConfig) { InternalEngine( final EngineConfig engineConfig, - final Function localCheckpointTrackerSupplier) { + final BiFunction localCheckpointTrackerSupplier) { super(engineConfig); openMode = engineConfig.getOpenMode(); if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) { @@ -180,14 +179,9 @@ public InternalEngine(EngineConfig engineConfig) { mergeScheduler = scheduler = new EngineMergeScheduler(engineConfig.getShardId(), engineConfig.getIndexSettings()); throttle = new IndexThrottle(); try { - final SeqNoStats seqNoStats = loadSeqNoStats(openMode); - logger.trace("recovered [{}]", seqNoStats); - this.localCheckpointTracker = localCheckpointTrackerSupplier.apply(seqNoStats); - engineConfig.getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(seqNoStats.getGlobalCheckpoint(), - "initializing from persisted global checkpoint"); + this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier); this.snapshotDeletionPolicy = new SnapshotDeletionPolicy( - new CombinedDeletionPolicy(openMode, translogDeletionPolicy, - engineConfig.getGlobalCheckpointTracker()::getGlobalCheckpoint) + new CombinedDeletionPolicy(openMode, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier()) ); writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG); updateMaxUnsafeAutoIdTimestampFromWriter(writer); @@ -199,8 +193,7 @@ public InternalEngine(EngineConfig engineConfig) { historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer; - translog = openTranslog(engineConfig, writer, translogDeletionPolicy, - () -> engineConfig.getGlobalCheckpointTracker().getGlobalCheckpoint()); + translog = openTranslog(engineConfig, writer, translogDeletionPolicy, engineConfig.getGlobalCheckpointSupplier()); assert translog.getGeneration() != null; this.translog = translog; updateWriterOnOpen(); @@ -242,6 +235,22 @@ public InternalEngine(EngineConfig engineConfig) { logger.trace("created new InternalEngine"); } + private LocalCheckpointTracker createLocalCheckpointTracker( + BiFunction localCheckpointTrackerSupplier) throws IOException { + final long maxSeqNo; + final long localCheckpoint; + if (openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { + maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; + } else { + final Tuple seqNoStats = store.loadSeqNoStats(); + maxSeqNo = seqNoStats.v1(); + localCheckpoint = seqNoStats.v2(); + logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); + } + return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); + } + /** * This reference manager delegates all it's refresh calls to another (internal) SearcherManager * The main purpose for this is that if we have external refreshes happening we don't issue extra @@ -359,27 +368,10 @@ private void updateMaxUnsafeAutoIdTimestampFromWriter(IndexWriter writer) { maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), commitMaxUnsafeAutoIdTimestamp)); } - static LocalCheckpointTracker localCheckpointTracker(final SeqNoStats seqNoStats) { + static LocalCheckpointTracker localCheckpointTracker(final long maxSeqNo, final long localCheckpoint) { return new LocalCheckpointTracker( - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint()); - } - - private SeqNoStats loadSeqNoStats(EngineConfig.OpenMode openMode) throws IOException { - switch (openMode) { - case OPEN_INDEX_AND_TRANSLOG: - final long globalCheckpoint = Translog.readGlobalCheckpoint(engineConfig.getTranslogConfig().getTranslogPath()); - return store.loadSeqNoStats(globalCheckpoint); - case OPEN_INDEX_CREATE_TRANSLOG: - return store.loadSeqNoStats(SequenceNumbers.UNASSIGNED_SEQ_NO); - case CREATE_INDEX_AND_TRANSLOG: - return new SeqNoStats( - SequenceNumbers.NO_OPS_PERFORMED, - SequenceNumbers.NO_OPS_PERFORMED, - SequenceNumbers.UNASSIGNED_SEQ_NO); - default: - throw new IllegalArgumentException(openMode.toString()); - } + maxSeqNo, + localCheckpoint); } @Override diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java index 21b4134f9837e..23cc2ae9270b1 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java @@ -19,6 +19,8 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.common.collect.Tuple; + import java.util.Map; /** @@ -42,15 +44,12 @@ public class SequenceNumbers { public static final long NO_OPS_PERFORMED = -1L; /** - * Reads the sequence number stats from the commit data (maximum sequence number and local checkpoint) and uses the specified global - * checkpoint. + * Reads the sequence number stats from the commit data (maximum sequence number and local 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, + public static Tuple loadSeqNoStatsFromLuceneCommit( final Iterable> commitData) { long maxSeqNo = NO_OPS_PERFORMED; long localCheckpoint = NO_OPS_PERFORMED; @@ -66,7 +65,7 @@ public static SeqNoStats loadSeqNoStatsFromLuceneCommit( } } - return new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint); + return new Tuple<>(maxSeqNo, localCheckpoint); } /** 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 83661ed310770..bae914b6101ee 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1321,7 +1321,16 @@ private void internalPerformTranslogRecovery(boolean skipTranslogRecovery, boole assert indexExists == false || assertMaxUnsafeAutoIdInCommit(); + final EngineConfig config = newEngineConfig(openMode); + + if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { + globalCheckpointTracker.updateGlobalCheckpointOnReplica( + Translog.readGlobalCheckpoint(config.getTranslogConfig().getTranslogPath()), + "opening index and translog" + ); + } + // we disable deletes since we allow for operations to be executed against the shard while recovering // but we need to make sure we don't loose deletes until we are done recovering config.setEnableGcDeletes(false); @@ -2188,7 +2197,7 @@ private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode) { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, globalCheckpointTracker); + indexSort, this::runTranslogRecovery, circuitBreakerService, globalCheckpointTracker::getGlobalCheckpoint); } /** 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 902cf4b992b11..2a4f118dc08cb 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -52,6 +52,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; @@ -211,17 +212,14 @@ 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. + * Loads the maximum sequence number and local checkpoint from the latest Lucene 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 + * @return a tuple populated with the maximum sequence number and the local checkpoint * @throws IOException if an I/O exception occurred reading the latest Lucene commit point from disk */ - public SeqNoStats loadSeqNoStats(final long globalCheckpoint) throws IOException { + public Tuple loadSeqNoStats() throws IOException { final Map userData = SegmentInfos.readLatestCommit(directory).getUserData(); - return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(globalCheckpoint, userData.entrySet()); + return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(userData.entrySet()); } final void ensureOpen() { 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 9e2e7ddbd0653..e8bf80ab88b9a 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -180,7 +180,7 @@ public Translog( boolean success = false; current = null; try { - current = createWriter(checkpoint.generation + 1); + current = createWriter(checkpoint.generation + 1, getMinFileGeneration(), checkpoint.globalCheckpoint); success = true; } finally { // we have to close all the recovered ones otherwise we leak file handles here @@ -196,11 +196,12 @@ public Translog( final long generation = deletionPolicy.getMinTranslogGenerationForRecovery(); logger.debug("wipe translog location - creating new translog, starting generation [{}]", generation); Files.createDirectories(location); - final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint(0, generation, globalCheckpointSupplier.getAsLong(), generation); + final long initialGlobalCheckpoint = SequenceNumbers.UNASSIGNED_SEQ_NO; + final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint(0, generation, initialGlobalCheckpoint, generation); final Path checkpointFile = location.resolve(CHECKPOINT_FILE_NAME); Checkpoint.write(getChannelFactory(), checkpointFile, checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); IOUtils.fsync(checkpointFile, false); - current = createWriter(generation, generation); + current = createWriter(generation, generation, initialGlobalCheckpoint); readers.clear(); } } catch (Exception e) { @@ -450,7 +451,7 @@ private long sizeOfGensAboveSeqNoInBytes(long minSeqNo) { * @throws IOException if creating the translog failed */ TranslogWriter createWriter(long fileGeneration) throws IOException { - return createWriter(fileGeneration, getMinFileGeneration()); + return createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong()); } /** @@ -461,7 +462,7 @@ TranslogWriter createWriter(long fileGeneration) throws IOException { * needed to solve and initialization problem while constructing an empty translog. * With no readers and no current, a call to {@link #getMinFileGeneration()} would not work. */ - private TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen) throws IOException { + private TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { final TranslogWriter newFile; try { newFile = TranslogWriter.create( @@ -471,9 +472,8 @@ private TranslogWriter createWriter(long fileGeneration, long initialMinTranslog location.resolve(getFilename(fileGeneration)), getChannelFactory(), config.getBufferSize(), - globalCheckpointSupplier, - initialMinTranslogGen, - this::getMinFileGeneration); + initialMinTranslogGen, initialGlobalCheckpoint, + globalCheckpointSupplier, this::getMinFileGeneration); } catch (final IOException e) { throw new TranslogException(shardId, "failed to create new translog file", e); } diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index c12299feaa596..1eed393208c9c 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -119,8 +119,8 @@ static void writeHeader(OutputStreamDataOutput out, BytesRef ref) throws IOExcep } public static TranslogWriter create(ShardId shardId, String translogUUID, long fileGeneration, Path file, ChannelFactory channelFactory, - ByteSizeValue bufferSize, final LongSupplier globalCheckpointSupplier, - final long initialMinTranslogGen, final LongSupplier minTranslogGenerationSupplier) + ByteSizeValue bufferSize, final long initialMinTranslogGen, long initialGlobalCheckpoint, + final LongSupplier globalCheckpointSupplier, final LongSupplier minTranslogGenerationSupplier) throws IOException { final BytesRef ref = new BytesRef(translogUUID); final int firstOperationOffset = getHeaderLength(ref.length); @@ -132,10 +132,21 @@ public static TranslogWriter create(ShardId shardId, String translogUUID, long f writeHeader(out, ref); channel.force(true); final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint(firstOperationOffset, fileGeneration, - globalCheckpointSupplier.getAsLong(), initialMinTranslogGen); + initialGlobalCheckpoint, initialMinTranslogGen); writeCheckpoint(channelFactory, file.getParent(), checkpoint); + final LongSupplier writerGlobalCheckpointSupplier; + if (Assertions.ENABLED) { + writerGlobalCheckpointSupplier = () -> { + long gcp = globalCheckpointSupplier.getAsLong(); + assert gcp >= initialGlobalCheckpoint : + "global checkpoint [" + gcp + "] lower than initial gcp [" + initialGlobalCheckpoint + "]"; + return gcp; + }; + } else { + writerGlobalCheckpointSupplier = globalCheckpointSupplier; + } return new TranslogWriter(channelFactory, shardId, checkpoint, channel, file, bufferSize, - globalCheckpointSupplier, minTranslogGenerationSupplier); + writerGlobalCheckpointSupplier, minTranslogGenerationSupplier); } catch (Exception exception) { // if we fail to bake the file-generation into the checkpoint we stick with the file and once we recover and that // file exists we remove it. We only apply this logic to the checkpoint.generation+1 any other file with a higher generation is an error condition 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 65a8a0d0f6e0b..f3102968abd2a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -32,6 +32,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -41,7 +42,6 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.mapper.MapperException; -import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexEventListener; @@ -354,15 +354,17 @@ private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recove public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { try { final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation()); - final SeqNoStats seqNoStats = recoveryTarget.store().loadSeqNoStats(globalCheckpoint); - if (seqNoStats.getMaxSeqNo() <= seqNoStats.getGlobalCheckpoint()) { - assert seqNoStats.getLocalCheckpoint() <= seqNoStats.getGlobalCheckpoint(); + final Tuple seqNoStats = recoveryTarget.store().loadSeqNoStats(); + long maxSeqNo = seqNoStats.v1(); + long localCheckpoint = seqNoStats.v1(); + if (maxSeqNo <= globalCheckpoint) { + assert localCheckpoint <= globalCheckpoint; /* * Commit point is good for sequence-number based recovery as the maximum sequence number included in it is below the global * checkpoint (i.e., it excludes any operations that may not be on the primary). Recovery will start at the first operation * after the local checkpoint stored in the commit. */ - return seqNoStats.getLocalCheckpoint() + 1; + return localCheckpoint + 1; } else { return SequenceNumbers.UNASSIGNED_SEQ_NO; } 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 47aa656cd9f2b..194a3b4cf4c3a 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -69,10 +69,6 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -107,7 +103,6 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.ShardId; @@ -148,7 +143,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; -import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; import java.util.function.ToLongBiFunction; @@ -445,7 +439,7 @@ public void testSegmentsWithIndexSort() throws Exception { Sort indexSort = new Sort(new SortedSetSortField("_type", false)); try (Store store = createStore(); Engine engine = - createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, null, null, indexSort)) { + createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, null, null, indexSort, null)) { List segments = engine.segments(true); assertThat(segments.isEmpty(), equalTo(true)); @@ -501,9 +495,9 @@ public void testCommitStats() throws IOException { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); try ( Store store = createStore(); - InternalEngine engine = createEngine(store, createTempDir(), seqNoStats -> new LocalCheckpointTracker( - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint()) { + InternalEngine engine = createEngine(store, createTempDir(), (maxSeq, localCP) -> new LocalCheckpointTracker( + maxSeq, + localCP) { @Override public long getMaxSeqNo() { return maxSeqNo.get(); @@ -943,13 +937,14 @@ public void testCommitAdvancesMinTranslogForRecovery() throws IOException { IOUtils.close(engine, store); final Path translogPath = createTempDir(); store = createStore(); - boolean inSync = randomBoolean(); - engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null)); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); + engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier)); ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); + boolean inSync = randomBoolean(); if (inSync) { - engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( - engine.getLocalCheckpointTracker().getCheckpoint(), "initially in-sync"); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); } engine.flush(); @@ -967,9 +962,7 @@ public void testCommitAdvancesMinTranslogForRecovery() throws IOException { assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); - inSync = true; - engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( - engine.getLocalCheckpointTracker().getCheckpoint(), "now in-sync"); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); engine.flush(true, true); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(4L)); assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(4L)); @@ -1889,18 +1882,11 @@ public void testSeqNoAndCheckpoints() throws IOException { final Set indexedIds = new HashSet<>(); long localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; long replicaLocalCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; - final long globalCheckpoint; long maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; InternalEngine initialEngine = null; try { initialEngine = engine; - final ShardRouting primary = TestShardRouting.newShardRouting("test", shardId.id(), "node1", null, true, ShardRoutingState.STARTED, allocationId); - final ShardRouting replica = TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED); - initialEngine.config().getGlobalCheckpointTracker().updateFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), - replica.allocationId().getId())), - new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build(), Collections.emptySet()); - initialEngine.config().getGlobalCheckpointTracker().activatePrimaryMode(primarySeqNo); for (int op = 0; op < opCount; op++) { final String id; // mostly index, sometimes delete @@ -1944,10 +1930,6 @@ public void testSeqNoAndCheckpoints() throws IOException { // only update rarely as we do it every doc replicaLocalCheckpoint = randomIntBetween(Math.toIntExact(replicaLocalCheckpoint), Math.toIntExact(primarySeqNo)); } - initialEngine.config().getGlobalCheckpointTracker().updateLocalCheckpoint(primary.allocationId().getId(), - initialEngine.getLocalCheckpointTracker().getCheckpoint()); - initialEngine.config().getGlobalCheckpointTracker().updateLocalCheckpoint(replica.allocationId().getId(), replicaLocalCheckpoint); - if (rarely()) { localCheckpoint = primarySeqNo; maxSeqNo = primarySeqNo; @@ -1956,19 +1938,14 @@ public void testSeqNoAndCheckpoints() throws IOException { } logger.info("localcheckpoint {}, global {}", replicaLocalCheckpoint, primarySeqNo); - globalCheckpoint = initialEngine.config().getGlobalCheckpointTracker().getGlobalCheckpoint(); assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getMaxSeqNo()); assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getCheckpoint()); - assertThat(initialEngine.config().getGlobalCheckpointTracker().getGlobalCheckpoint(), equalTo(replicaLocalCheckpoint)); assertThat( 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(SequenceNumbers.MAX_SEQ_NO)), equalTo(maxSeqNo)); @@ -1986,9 +1963,6 @@ public void testSeqNoAndCheckpoints() throws IOException { assertThat( 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(SequenceNumbers.MAX_SEQ_NO)), // after recovering from translog, all docs have been flushed to Lucene segments, so here we will assert @@ -2379,8 +2353,11 @@ public void testTranslogCleanUpPostCommitCrash() throws Exception { try (Store store = createStore()) { AtomicBoolean throwErrorOnCommit = new AtomicBoolean(); final Path translogPath = createTempDir(); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); try (InternalEngine engine = - new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null)) { + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null, + globalCheckpointSupplier)) { @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { @@ -2392,14 +2369,14 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s }) { final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc1)); - engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica( - engine.getLocalCheckpointTracker().getCheckpoint(), "manual update"); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); throwErrorOnCommit.set(true); FlushFailedEngineException e = expectThrows(FlushFailedEngineException.class, engine::flush); assertThat(e.getCause().getMessage(), equalTo("power's out")); } try (InternalEngine engine = - new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null))) { + new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null, + globalCheckpointSupplier))) { engine.recoverFromTranslog(); assertVisibleCount(engine, 1); final long committedGen = Long.valueOf( @@ -2551,7 +2528,7 @@ public void testRecoverFromForeignTranslog() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); fail("translog belongs to a different engine"); @@ -2605,7 +2582,7 @@ threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); engine = new InternalEngine(newConfig); if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { engine.recoverFromTranslog(); @@ -2636,7 +2613,7 @@ public void testHistoryUUIDCanBeForced() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), true, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), config.getGlobalCheckpointTracker()); + new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); if (newConfig.getOpenMode() == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { Lucene.cleanLuceneIndex(store.directory()); } @@ -3545,7 +3522,7 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final List threads = new ArrayList<>(); initialEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalInitialEngine = initialEngine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); @@ -3718,7 +3695,7 @@ public void testNoOps() throws IOException { final int maxSeqNo = randomIntBetween(0, 128); final int localCheckpoint = randomIntBetween(0, maxSeqNo); try { - final Function supplier = seqNoStats -> new LocalCheckpointTracker( + final BiFunction supplier = (ms, lcp) -> new LocalCheckpointTracker( maxSeqNo, localCheckpoint); noOpEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier) { @@ -3770,7 +3747,7 @@ public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierExcepti final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final Map threads = new LinkedHashMap<>(); actualEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalActualEngine = actualEngine; final Translog translog = finalActualEngine.getTranslog(); final long generation = finalActualEngine.getTranslog().currentFileGeneration(); @@ -3859,9 +3836,9 @@ public void testRestoreLocalCheckpointFromTranslog() throws IOException { InternalEngine actualEngine = null; try { final Set completedSeqNos = new HashSet<>(); - final Function supplier = seqNoStats -> new LocalCheckpointTracker( - seqNoStats.getMaxSeqNo(), - seqNoStats.getLocalCheckpoint()) { + final BiFunction supplier = (maxSeqNo, localCheckpoint) -> new LocalCheckpointTracker( + maxSeqNo, + localCheckpoint) { @Override public void markSeqNoAsCompleted(long seqNo) { super.markSeqNoAsCompleted(seqNo); @@ -4069,11 +4046,12 @@ public void testRefreshScopedSearcher() throws IOException { public void testSeqNoGenerator() throws IOException { engine.close(); final long seqNo = randomIntBetween(Math.toIntExact(SequenceNumbers.NO_OPS_PERFORMED), Integer.MAX_VALUE); - final Function seqNoService = seqNoStats -> new LocalCheckpointTracker( + final BiFunction localCheckpointTrackerSupplier = (ms, lcp) -> new LocalCheckpointTracker( SequenceNumbers.NO_OPS_PERFORMED, SequenceNumbers.NO_OPS_PERFORMED); final AtomicLong seqNoGenerator = new AtomicLong(seqNo); - try (Engine e = createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, seqNoService, (engine, operation) -> seqNoGenerator.getAndIncrement())) { + try (Engine e = createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, localCheckpointTrackerSupplier, + null, (engine, operation) -> seqNoGenerator.getAndIncrement())) { final String id = "id"; final Field uidField = new Field("_id", id, IdFieldMapper.Defaults.FIELD_TYPE); final String type = "type"; @@ -4139,18 +4117,21 @@ public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { indexSettings.updateIndexMetaData(builder.build()); store = createStore(); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); + try (InternalEngine engine - = createEngine(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null)) { - engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(0L, "initially set"); + = createEngine(indexSettings, store, createTempDir(), NoMergePolicy.INSTANCE, null, null, globalCheckpointSupplier)) { + globalCheckpoint.set(0L); int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, BytesReference.toBytes(B_1), SourceFieldMapper.Defaults.FIELD_TYPE)); engine.index(indexForDoc(testParsedDocument(Integer.toString(docId), null, document, B_1, null))); if (frequently()) { - engine.config().getGlobalCheckpointTracker().updateGlobalCheckpointOnReplica(randomIntBetween( - Math.toIntExact(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint()), - Math.toIntExact(engine.getLocalCheckpointTracker().getCheckpoint())), "updated"); + globalCheckpoint.set(randomIntBetween( + Math.toIntExact(globalCheckpoint.get()), + Math.toIntExact(engine.getLocalCheckpointTracker().getCheckpoint()))); } if (frequently()) { engine.flush(randomBoolean(), true); @@ -4158,10 +4139,10 @@ public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { // Keep only one safe commit as the oldest commit. final IndexCommit safeCommit = commits.get(0); assertThat(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO)), - lessThanOrEqualTo(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint())); + lessThanOrEqualTo(globalCheckpoint.get())); for (int i = 1; i < commits.size(); i++) { assertThat(Long.parseLong(commits.get(i).getUserData().get(SequenceNumbers.MAX_SEQ_NO)), - greaterThan(engine.config().getGlobalCheckpointTracker().getGlobalCheckpoint())); + greaterThan(globalCheckpoint.get())); } // Make sure we keep all translog operations after the local checkpoint of the safe commit. long localCheckpointFromSafeCommit = Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index b71fcf4b7c24f..6024daf45ccd3 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -126,7 +126,7 @@ indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilari eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, null, new NoneCircuitBreakerService(), - new GlobalCheckpointTracker(shardId, allocationId, indexSettings, SequenceNumbers.UNASSIGNED_SEQ_NO)); + () -> SequenceNumbers.UNASSIGNED_SEQ_NO); engine = new InternalEngine(config); listeners.setTranslog(engine.getTranslog()); } diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java index 39fc182623f95..0779ba0f5a7e7 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogDeletionPolicyTests.java @@ -25,8 +25,6 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; import org.mockito.Mockito; @@ -172,8 +170,8 @@ private Tuple, TranslogWriter> createReadersAndWriter(final readers.add(reader); } writer = TranslogWriter.create(new ShardId("index", "uuid", 0), translogUUID, gen, - tempDir.resolve(Translog.getFilename(gen)), FileChannel::open, TranslogConfig.DEFAULT_BUFFER_SIZE, () -> 1L, 1L, () -> 1L - ); + tempDir.resolve(Translog.getFilename(gen)), FileChannel::open, TranslogConfig.DEFAULT_BUFFER_SIZE, 1L, 1L, () -> 1L, + () -> 1L); writer = Mockito.spy(writer); Mockito.doReturn(now - (numberOfReaders - gen + 1) * 1000).when(writer).getLastModifiedTime(); diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 835d16117ad60..16852502e8cc2 100644 --- a/core/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/core/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -114,7 +114,7 @@ private Path writeTranslog( generation, resolve, FileChannel::open, - TranslogConfig.DEFAULT_BUFFER_SIZE, () -> globalCheckpoint, generation, () -> generation)) {} + TranslogConfig.DEFAULT_BUFFER_SIZE, generation, globalCheckpoint, () -> globalCheckpoint, () -> generation)) {} return tempDir; } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index aeb60df53126b..5ec8e8bf32552 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -60,9 +60,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; -import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.store.DirectoryService; @@ -84,7 +82,8 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.function.Function; +import java.util.function.BiFunction; +import java.util.function.LongSupplier; import java.util.function.ToLongBiFunction; import static java.util.Collections.emptyList; @@ -165,8 +164,9 @@ public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, An new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), - config.getCircuitBreakerService(), new GlobalCheckpointTracker(config.getShardId(), config.getAllocationId(), - config.getIndexSettings(), SequenceNumbers.UNASSIGNED_SEQ_NO)); + config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier()); + //new GlobalCheckpointTracker(config.getShardId(), config.getAllocationId(), + //config.getIndexSettings(), SequenceNumbers.UNASSIGNED_SEQ_NO)); } @Override @@ -255,17 +255,17 @@ protected InternalEngine createEngine(Store store, Path translogPath) throws IOE protected InternalEngine createEngine( Store store, Path translogPath, - Function localCheckpointTrackerSupplier) throws IOException { - return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier); + BiFunction localCheckpointTrackerSupplier) throws IOException { + return createEngine(defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier, null); } protected InternalEngine createEngine( Store store, Path translogPath, - Function localCheckpointTrackerSupplier, + BiFunction localCheckpointTrackerSupplier, ToLongBiFunction seqNoForOperation) throws IOException { return createEngine( - defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier, seqNoForOperation, null); + defaultSettings, store, translogPath, newMergePolicy(), null, localCheckpointTrackerSupplier, null, seqNoForOperation); } protected InternalEngine createEngine( @@ -276,7 +276,7 @@ protected InternalEngine createEngine( protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory) throws IOException { - return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, null); + return createEngine(indexSettings, store, translogPath, mergePolicy, indexWriterFactory, null, null); } protected InternalEngine createEngine( @@ -285,9 +285,11 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable Function localCheckpointTrackerSupplier) throws IOException { + @Nullable BiFunction localCheckpointTrackerSupplier, + @Nullable LongSupplier globalCheckpointSupplier) throws IOException { return createEngine( - indexSettings, store, translogPath, mergePolicy, indexWriterFactory, localCheckpointTrackerSupplier, null, null); + indexSettings, store, translogPath, mergePolicy, indexWriterFactory, localCheckpointTrackerSupplier, null, null, + globalCheckpointSupplier); } protected InternalEngine createEngine( @@ -296,7 +298,8 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable Function localCheckpointTrackerSupplier, + @Nullable BiFunction localCheckpointTrackerSupplier, + @Nullable LongSupplier globalCheckpointSupplier, @Nullable ToLongBiFunction seqNoForOperation) throws IOException { return createEngine( indexSettings, @@ -306,7 +309,8 @@ protected InternalEngine createEngine( indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, - null); + null, + globalCheckpointSupplier); } protected InternalEngine createEngine( @@ -315,10 +319,11 @@ protected InternalEngine createEngine( Path translogPath, MergePolicy mergePolicy, @Nullable IndexWriterFactory indexWriterFactory, - @Nullable Function localCheckpointTrackerSupplier, + @Nullable BiFunction localCheckpointTrackerSupplier, @Nullable ToLongBiFunction seqNoForOperation, - @Nullable Sort indexSort) throws IOException { - EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null, indexSort); + @Nullable Sort indexSort, + @Nullable LongSupplier globalCheckpointSupplier) throws IOException { + EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null, indexSort, globalCheckpointSupplier); InternalEngine internalEngine = createInternalEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); if (config.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { internalEngine.recoverFromTranslog(); @@ -334,7 +339,7 @@ public interface IndexWriterFactory { public static InternalEngine createInternalEngine( @Nullable final IndexWriterFactory indexWriterFactory, - @Nullable final Function localCheckpointTrackerSupplier, + @Nullable final BiFunction localCheckpointTrackerSupplier, @Nullable final ToLongBiFunction seqNoForOperation, final EngineConfig config) { if (localCheckpointTrackerSupplier == null) { @@ -375,11 +380,11 @@ protected long doGenerateSeqNoForOperation(final Operation operation) { public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, ReferenceManager.RefreshListener refreshListener) { - return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null); + return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); } public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, - ReferenceManager.RefreshListener refreshListener, Sort indexSort) { + ReferenceManager.RefreshListener refreshListener, Sort indexSort, LongSupplier globalCheckpointSupplier) { IndexWriterConfig iwc = newIndexWriterConfig(); TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); final EngineConfig.OpenMode openMode; @@ -407,7 +412,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), refreshListenerList, Collections.emptyList(), indexSort, handler, new NoneCircuitBreakerService(), - new GlobalCheckpointTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.UNASSIGNED_SEQ_NO)); + globalCheckpointSupplier == null ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : globalCheckpointSupplier); return config; } From b954aac7b93e44e8536a8836631f81b36c8a71ed Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 15 Dec 2017 16:57:40 +0100 Subject: [PATCH 05/11] remove one more method --- .../org/elasticsearch/index/engine/InternalEngine.java | 8 +------- .../elasticsearch/index/engine/InternalEngineTests.java | 6 +++--- 2 files changed, 4 insertions(+), 10 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 78d933d68e4d4..6c6a7fdf07aae 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -150,7 +150,7 @@ public class InternalEngine extends Engine { private final String historyUUID; public InternalEngine(EngineConfig engineConfig) { - this(engineConfig, InternalEngine::localCheckpointTracker); + this(engineConfig, LocalCheckpointTracker::new); } InternalEngine( @@ -368,12 +368,6 @@ private void updateMaxUnsafeAutoIdTimestampFromWriter(IndexWriter writer) { maxUnsafeAutoIdTimestamp.set(Math.max(maxUnsafeAutoIdTimestamp.get(), commitMaxUnsafeAutoIdTimestamp)); } - static LocalCheckpointTracker localCheckpointTracker(final long maxSeqNo, final long localCheckpoint) { - return new LocalCheckpointTracker( - maxSeqNo, - localCheckpoint); - } - @Override public InternalEngine recoverFromTranslog() throws IOException { flushLock.lock(); 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 21c997b67aaf6..94f6d734466ef 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -741,7 +741,7 @@ public void testTranslogRecoveryWithMultipleGenerations() throws IOException { initialEngine = createEngine( store, createTempDir(), - InternalEngine::localCheckpointTracker, + LocalCheckpointTracker::new, (engine, operation) -> seqNos.get(counter.getAndIncrement())); for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); @@ -3591,7 +3591,7 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final List threads = new ArrayList<>(); initialEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, LocalCheckpointTracker::new, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalInitialEngine = initialEngine; for (int i = 0; i < docs; i++) { final String id = Integer.toString(i); @@ -3816,7 +3816,7 @@ public void testMinGenerationForSeqNo() throws IOException, BrokenBarrierExcepti final AtomicLong expectedLocalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final Map threads = new LinkedHashMap<>(); actualEngine = - createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, InternalEngine::localCheckpointTracker, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); + createEngine(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null, LocalCheckpointTracker::new, null, getStallingSeqNoGenerator(latchReference, barrier, stall, expectedLocalCheckpoint)); final InternalEngine finalActualEngine = actualEngine; final Translog translog = finalActualEngine.getTranslog(); final long generation = finalActualEngine.getTranslog().currentFileGeneration(); From ea23a0d32ebab3085ea2ecb8a64d94aaaca670f9 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 10:13:28 +0100 Subject: [PATCH 06/11] address review comments --- .../index/engine/InternalEngine.java | 21 +++++++++------ .../index/seqno/GlobalCheckpointTracker.java | 8 +++++- .../index/seqno/SequenceNumbers.java | 2 +- .../elasticsearch/index/shard/IndexShard.java | 4 ++- .../org/elasticsearch/index/store/Store.java | 5 ++-- .../index/translog/Translog.java | 9 ++++--- .../recovery/PeerRecoveryTargetService.java | 2 +- .../index/engine/InternalEngineTests.java | 26 +++++++++++++++++++ .../index/engine/EvilInternalEngineTests.java | 1 + .../index/engine/EngineTestCase.java | 7 ++--- 10 files changed, 63 insertions(+), 22 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 a765437accb03..6c2e9955780c0 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -239,14 +239,19 @@ private LocalCheckpointTracker createLocalCheckpointTracker( BiFunction localCheckpointTrackerSupplier) throws IOException { final long maxSeqNo; final long localCheckpoint; - if (openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { - maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; - localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; - } else { - final Tuple seqNoStats = store.loadSeqNoStats(); - maxSeqNo = seqNoStats.v1(); - localCheckpoint = seqNoStats.v2(); - logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); + switch (openMode) { + case CREATE_INDEX_AND_TRANSLOG: + maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; + localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; + break; + case OPEN_INDEX_AND_TRANSLOG: + case OPEN_INDEX_CREATE_TRANSLOG: + final Tuple seqNoStats = store.loadSeqNoInfo(); + maxSeqNo = seqNoStats.v1(); + localCheckpoint = seqNoStats.v2(); + logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); + break; + default: throw new IllegalArgumentException("unknown type: " + openMode); } return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } 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 8226b8f558e51..ada7cf7fd0b1b 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/GlobalCheckpointTracker.java @@ -42,6 +42,7 @@ import java.util.Set; import java.util.function.Function; import java.util.function.LongConsumer; +import java.util.function.LongSupplier; import java.util.function.ToLongFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -55,7 +56,7 @@ *

* The global checkpoint is maintained by the primary shard and is replicated to all the replicas (via {@link GlobalCheckpointSyncAction}). */ -public class GlobalCheckpointTracker extends AbstractIndexShardComponent { +public class GlobalCheckpointTracker extends AbstractIndexShardComponent implements LongSupplier { /** * The allocation ID for the shard to which this tracker is a component of. @@ -374,6 +375,11 @@ public synchronized long getGlobalCheckpoint() { return cps.globalCheckpoint; } + @Override + public long getAsLong() { + return getGlobalCheckpoint(); + } + /** * Updates the global checkpoint on a replica shard after it has been updated by the primary. * diff --git a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java index 23cc2ae9270b1..ec1fc4fc43547 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/SequenceNumbers.java @@ -49,7 +49,7 @@ public class SequenceNumbers { * @param commitData the commit data * @return the sequence number stats */ - public static Tuple loadSeqNoStatsFromLuceneCommit( + public static Tuple loadSeqNoInfoFromLuceneCommit( final Iterable> commitData) { long maxSeqNo = NO_OPS_PERFORMED; long localCheckpoint = NO_OPS_PERFORMED; 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 bae914b6101ee..90684267f76fb 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1325,6 +1325,8 @@ private void internalPerformTranslogRecovery(boolean skipTranslogRecovery, boole final EngineConfig config = newEngineConfig(openMode); if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { + // set global checkpoint before opening engine, to ensure that the global checkpoint written to the checkpoint file + // is not reset to the default value, which could prevent future sequence-number based recoveries or rolling back of Lucene. globalCheckpointTracker.updateGlobalCheckpointOnReplica( Translog.readGlobalCheckpoint(config.getTranslogConfig().getTranslogPath()), "opening index and translog" @@ -2197,7 +2199,7 @@ private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode) { IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), - indexSort, this::runTranslogRecovery, circuitBreakerService, globalCheckpointTracker::getGlobalCheckpoint); + indexSort, this::runTranslogRecovery, circuitBreakerService, globalCheckpointTracker); } /** 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 2a4f118dc08cb..693791e66b660 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -75,7 +75,6 @@ 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; @@ -217,9 +216,9 @@ private static SegmentInfos readSegmentsInfo(IndexCommit commit, Directory direc * @return a tuple populated with the maximum sequence number and the local checkpoint * @throws IOException if an I/O exception occurred reading the latest Lucene commit point from disk */ - public Tuple loadSeqNoStats() throws IOException { + public Tuple loadSeqNoInfo() throws IOException { final Map userData = SegmentInfos.readLatestCommit(directory).getUserData(); - return SequenceNumbers.loadSeqNoStatsFromLuceneCommit(userData.entrySet()); + return SequenceNumbers.loadSeqNoInfoFromLuceneCommit(userData.entrySet()); } final void ensureOpen() { 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 e8bf80ab88b9a..36bd61a693f39 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -457,10 +457,11 @@ TranslogWriter createWriter(long fileGeneration) throws IOException { /** * creates a new writer * - * @param fileGeneration the generation of the write to be written - * @param initialMinTranslogGen the minimum translog generation to be written in the first checkpoint. This is - * needed to solve and initialization problem while constructing an empty translog. - * With no readers and no current, a call to {@link #getMinFileGeneration()} would not work. + * @param fileGeneration the generation of the write to be written + * @param initialMinTranslogGen the minimum translog generation to be written in the first checkpoint. This is + * needed to solve and initialization problem while constructing an empty translog. + * With no readers and no current, a call to {@link #getMinFileGeneration()} would not work. + * @param initialGlobalCheckpoint the global checkpoint to be written in the first checkpoint. */ private TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { final TranslogWriter newFile; 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 f3102968abd2a..0c20d7083606a 100644 --- a/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/core/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -354,7 +354,7 @@ private StartRecoveryRequest getStartRecoveryRequest(final RecoveryTarget recove public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { try { final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation()); - final Tuple seqNoStats = recoveryTarget.store().loadSeqNoStats(); + final Tuple seqNoStats = recoveryTarget.store().loadSeqNoInfo(); long maxSeqNo = seqNoStats.v1(); long localCheckpoint = seqNoStats.v1(); if (maxSeqNo <= globalCheckpoint) { 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 d4b0dab35a39a..35068b032293b 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -69,6 +69,10 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; @@ -102,6 +106,7 @@ import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -1945,11 +1950,20 @@ public void testSeqNoAndCheckpoints() throws IOException { final Set indexedIds = new HashSet<>(); long localCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; long replicaLocalCheckpoint = SequenceNumbers.NO_OPS_PERFORMED; + final long globalCheckpoint; long maxSeqNo = SequenceNumbers.NO_OPS_PERFORMED; InternalEngine initialEngine = null; try { initialEngine = engine; + final ShardRouting primary = TestShardRouting.newShardRouting("test", shardId.id(), "node1", null, true, + ShardRoutingState.STARTED, allocationId); + final ShardRouting replica = TestShardRouting.newShardRouting(shardId, "node2", false, ShardRoutingState.STARTED); + GlobalCheckpointTracker gcpTracker = (GlobalCheckpointTracker) initialEngine.config().getGlobalCheckpointSupplier(); + gcpTracker.updateFromMaster(1L, new HashSet<>(Arrays.asList(primary.allocationId().getId(), + replica.allocationId().getId())), + new IndexShardRoutingTable.Builder(shardId).addShard(primary).addShard(replica).build(), Collections.emptySet()); + gcpTracker.activatePrimaryMode(primarySeqNo); for (int op = 0; op < opCount; op++) { final String id; // mostly index, sometimes delete @@ -1993,6 +2007,10 @@ public void testSeqNoAndCheckpoints() throws IOException { // only update rarely as we do it every doc replicaLocalCheckpoint = randomIntBetween(Math.toIntExact(replicaLocalCheckpoint), Math.toIntExact(primarySeqNo)); } + gcpTracker.updateLocalCheckpoint(primary.allocationId().getId(), + initialEngine.getLocalCheckpointTracker().getCheckpoint()); + gcpTracker.updateLocalCheckpoint(replica.allocationId().getId(), replicaLocalCheckpoint); + if (rarely()) { localCheckpoint = primarySeqNo; maxSeqNo = primarySeqNo; @@ -2001,14 +2019,19 @@ public void testSeqNoAndCheckpoints() throws IOException { } logger.info("localcheckpoint {}, global {}", replicaLocalCheckpoint, primarySeqNo); + globalCheckpoint = gcpTracker.getGlobalCheckpoint(); assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getMaxSeqNo()); assertEquals(primarySeqNo, initialEngine.getLocalCheckpointTracker().getCheckpoint()); + assertThat(globalCheckpoint, equalTo(replicaLocalCheckpoint)); assertThat( 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(SequenceNumbers.MAX_SEQ_NO)), equalTo(maxSeqNo)); @@ -2026,6 +2049,9 @@ public void testSeqNoAndCheckpoints() throws IOException { assertThat( 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(SequenceNumbers.MAX_SEQ_NO)), // after recovering from translog, all docs have been flushed to Lucene segments, so here we will assert diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/index/engine/EvilInternalEngineTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/index/engine/EvilInternalEngineTests.java index c32b3ab202080..c473e4e010552 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/index/engine/EvilInternalEngineTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/index/engine/EvilInternalEngineTests.java @@ -79,6 +79,7 @@ public synchronized MergePolicy.OneMerge getNextMerge() { } } }, + null, null)) { // force segments to exist on disk final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 5ec8e8bf32552..a22f1aa7c407e 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -60,6 +60,7 @@ import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ShardId; @@ -165,8 +166,6 @@ public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, An config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier()); - //new GlobalCheckpointTracker(config.getShardId(), config.getAllocationId(), - //config.getIndexSettings(), SequenceNumbers.UNASSIGNED_SEQ_NO)); } @Override @@ -412,7 +411,9 @@ public void onFailedEngine(String reason, @Nullable Exception e) { IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), refreshListenerList, Collections.emptyList(), indexSort, handler, new NoneCircuitBreakerService(), - globalCheckpointSupplier == null ? () -> SequenceNumbers.UNASSIGNED_SEQ_NO : globalCheckpointSupplier); + globalCheckpointSupplier == null ? + new GlobalCheckpointTracker(shardId, allocationId.getId(), indexSettings, + SequenceNumbers.UNASSIGNED_SEQ_NO) : globalCheckpointSupplier); return config; } From 9b5fff77edfe566911660bbb6a2cfe2425a5f012 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 10:39:44 +0100 Subject: [PATCH 07/11] fix test --- .../elasticsearch/index/engine/InternalEngineTests.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) 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 35068b032293b..ec23b741827cc 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -106,9 +106,9 @@ import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.SourceFieldMapper; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.LocalCheckpointTracker; -import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.ShardId; @@ -4213,10 +4213,9 @@ public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { final Path translogPath = createTempDir(); store = createStore(); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); final EngineConfig engineConfig = config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null, - globalCheckpointSupplier); + () -> globalCheckpoint.get()); try (Engine engine = new InternalEngine(engineConfig) { @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { @@ -4228,7 +4227,6 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s super.commitIndexWriter(writer, translog, syncId); } }) { - globalCheckpoint.set(0L); int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); @@ -4236,6 +4234,7 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s engine.index(indexForDoc(testParsedDocument(Integer.toString(docId), null, document, B_1, null))); if (frequently()) { globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); + engine.getTranslog().sync(); } if (frequently()) { final long lastSyncedGlobalCheckpoint = Translog.readGlobalCheckpoint(translogPath); From 6795c8b7099cb281ec4e8da4583b29e5838d0e40 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 10:43:12 +0100 Subject: [PATCH 08/11] fix another test --- .../main/java/org/elasticsearch/index/translog/Translog.java | 4 ++-- .../java/org/elasticsearch/index/translog/TranslogTests.java | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) 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 36bd61a693f39..58396a134dfdd 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -450,7 +450,7 @@ private long sizeOfGensAboveSeqNoInBytes(long minSeqNo) { * @return a writer for the new translog * @throws IOException if creating the translog failed */ - TranslogWriter createWriter(long fileGeneration) throws IOException { + private TranslogWriter createWriter(long fileGeneration) throws IOException { return createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong()); } @@ -463,7 +463,7 @@ TranslogWriter createWriter(long fileGeneration) throws IOException { * With no readers and no current, a call to {@link #getMinFileGeneration()} would not work. * @param initialGlobalCheckpoint the global checkpoint to be written in the first checkpoint. */ - private TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { + TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { final TranslogWriter newFile; try { newFile = TranslogWriter.create( diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index e8229569ac481..890db4a795918 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -2084,7 +2084,8 @@ public void testFailWhileCreateWriteWithRecoveredTLogs() throws IOException { try { new Translog(config, translog.getTranslogUUID(), createTranslogDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO) { @Override - protected TranslogWriter createWriter(long fileGeneration) throws IOException { + protected TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) + throws IOException { throw new MockDirectoryWrapper.FakeIOException(); } }; From 2a85aac0f1852c4e559d2e2bd3b6317f65a2c231 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 10:54:11 +0100 Subject: [PATCH 09/11] fix visibility --- .../main/java/org/elasticsearch/index/translog/Translog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 58396a134dfdd..f2a53d82ad059 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -450,7 +450,7 @@ private long sizeOfGensAboveSeqNoInBytes(long minSeqNo) { * @return a writer for the new translog * @throws IOException if creating the translog failed */ - private TranslogWriter createWriter(long fileGeneration) throws IOException { + TranslogWriter createWriter(long fileGeneration) throws IOException { return createWriter(fileGeneration, getMinFileGeneration(), globalCheckpointSupplier.getAsLong()); } From 369722afa45acc17d271bb90a9a1a6a8d9879fca Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 13:01:21 +0100 Subject: [PATCH 10/11] fix javadoc --- .../org/elasticsearch/index/seqno/LocalCheckpointTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 52a8145b9d86b..34926a36f4573 100644 --- a/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java +++ b/core/src/main/java/org/elasticsearch/index/seqno/LocalCheckpointTracker.java @@ -138,7 +138,7 @@ public long getMaxSeqNo() { /** * constructs a {@link SeqNoStats} object, using local state and the supplied global checkpoint * - * @implNote this is needed to make sure the local checkpoint and max seq no are consistent + * This is needed to make sure the local checkpoint and max seq no are consistent */ public synchronized SeqNoStats getStats(final long globalCheckpoint) { return new SeqNoStats(getMaxSeqNo(), getCheckpoint(), globalCheckpoint); From e1d45abdda1da71acf8d7a4a3f453ccf216b0fb3 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Mon, 18 Dec 2017 13:42:32 +0100 Subject: [PATCH 11/11] thanks @bleskes --- .../elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 7d6bf0ff91202..4c9ff0cbb0720 100644 --- a/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/core/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -3107,7 +3107,7 @@ public void testSnapshottingWithMissingSequenceNumbers() { final Index index = resolveIndex(indexName); final IndexShard primary = internalCluster().getInstance(IndicesService.class, dataNode).getShardOrNull(new ShardId(index, 0)); // create a gap in the sequence numbers - getEngineFromShard(primary).seqNoService().generateSeqNo(); + getEngineFromShard(primary).getLocalCheckpointTracker().generateSeqNo(); for (int i = 5; i < 10; i++) { index(indexName, "_doc", Integer.toString(i), "foo", "bar" + i);