From 2826b414b807b3f91ead497a8765c1d79bc894db Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 13 Jan 2018 22:31:06 +0100 Subject: [PATCH 01/25] basic utils in place --- .../index/engine/EngineDiskUtils.java | 115 ++++++++++++++++++ .../index/translog/Translog.java | 19 +++ 2 files changed, 134 insertions(+) create mode 100644 server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java new file mode 100644 index 0000000000000..932cdf5d2d598 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java @@ -0,0 +1,115 @@ +/* + * 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.engine; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexCommit; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.store.Directory; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +public final class EngineDiskUtils { + + private EngineDiskUtils() { + } + + public static void createEmpty(final Directory dir, final Path translogPath, final ShardId shardId) throws IOException { + try (IndexWriter writer = newIndexWriter(true, dir)) { + final String translogUuid = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId); + final Map map = new HashMap<>(); + map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); + map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); + map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); + map.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(SequenceNumbers.NO_OPS_PERFORMED)); + map.put(InternalEngine.MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, "-1"); + updateCommitData(writer, map); + } + } + + + public static void bootstrapNewHistoryFromLuceneIndex(final Directory dir, final Path translogPath, final ShardId shardId) + throws IOException { + try (IndexWriter writer = newIndexWriter(false, dir)) { + final Map userData = getUserData(writer); + final long maxSeqNo = Long.parseLong(userData.get(SequenceNumbers.MAX_SEQ_NO)); + final String translogUuid = Translog.createEmptyTranslog(translogPath, maxSeqNo, shardId); + final Map map = new HashMap<>(); + map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); + map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); + map.put(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID()); + map.put(SequenceNumbers.LOCAL_CHECKPOINT_KEY, Long.toString(maxSeqNo)); + updateCommitData(writer, map); + } + } + + public static void createNewTranslog(final Directory dir, final Path translogPath, long initialGlobalCheckpoint, final ShardId shardId) + throws IOException { + try (IndexWriter writer = newIndexWriter(false, dir)) { + final String translogUuid = Translog.createEmptyTranslog(translogPath, initialGlobalCheckpoint, shardId); + final Map map = new HashMap<>(); + map.put(Translog.TRANSLOG_GENERATION_KEY, "1"); + map.put(Translog.TRANSLOG_UUID_KEY, translogUuid); + updateCommitData(writer, map); + } + } + + private static void updateCommitData(IndexWriter writer, Map keysToUpdate) throws IOException { + List commits = DirectoryReader.listCommits(writer.getDirectory()); + if (commits.size() != 1) { + throw new UnsupportedOperationException("can only update commit data if there's a single commit, found [" + + commits.size() + "]"); + } + + final Map userData = getUserData(writer); + userData.putAll(keysToUpdate); + writer.setLiveCommitData(userData.entrySet()); + writer.commit(); + } + + private static Map getUserData(IndexWriter writer) { + final Map userData = new HashMap<>(); + writer.getLiveCommitData().forEach(e -> userData.put(e.getKey(), e.getValue())); + return userData; + } + + private static IndexWriter newIndexWriter(final boolean existing, final Directory dir) throws IOException { + IndexWriterConfig iwc = new IndexWriterConfig(null) + .setCommitOnClose(false) + // we don't want merges to happen here - we call maybe merge on the engine + // later once we stared it up otherwise we would need to wait for it here + // we also don't specify a codec here and merges should use the engines for this index + .setMergePolicy(NoMergePolicy.INSTANCE) + .setOpenMode(existing ? IndexWriterConfig.OpenMode.APPEND : IndexWriterConfig.OpenMode.CREATE); + return new IndexWriter(dir, iwc); + } +} diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index b4bf6173f74cf..9f89752e51fa1 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -36,6 +36,7 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.IndexSettings; @@ -45,6 +46,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.AbstractIndexShardComponent; import org.elasticsearch.index.shard.IndexShardComponent; +import org.elasticsearch.index.shard.ShardId; import java.io.Closeable; import java.io.EOFException; @@ -1719,4 +1721,21 @@ TranslogWriter getCurrent() { List getReaders() { return readers; } + + public static String createEmptyTranslog(final Path location, final long initialGlobalCheckpoint, final ShardId shardId) + throws IOException { + IOUtils.rm(location); + Files.createDirectories(location); + final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint(0, 1, initialGlobalCheckpoint, 1); + final Path checkpointFile = location.resolve(CHECKPOINT_FILE_NAME); + Checkpoint.write(FileChannel::open, checkpointFile, checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + IOUtils.fsync(checkpointFile, false); + final String translogUUID = UUIDs.randomBase64UUID(); + TranslogWriter writer = TranslogWriter.create(shardId, translogUUID, 1, location.resolve(getFilename(1)), FileChannel::open, + new ByteSizeValue(10), 1, initialGlobalCheckpoint, + () -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); } + ); + writer.close(); + return translogUUID; + } } From d9c236b7dbf9a33ec065ba2835ba7c3515733c31 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Jan 2018 17:17:53 -0500 Subject: [PATCH 02/25] integrated --- .../index/engine/CombinedDeletionPolicy.java | 21 +- .../index/engine/EngineConfig.java | 41 +--- .../index/engine/EngineDiskUtils.java | 22 +- .../index/engine/InternalEngine.java | 160 ++++-------- .../elasticsearch/index/shard/IndexShard.java | 65 ++--- .../index/shard/StoreRecovery.java | 27 +- .../index/translog/Translog.java | 87 +++---- .../recovery/PeerRecoveryTargetService.java | 2 +- ...ryPrepareForTranslogOperationsRequest.java | 18 +- .../indices/recovery/RecoveryTarget.java | 17 +- .../recovery/RecoveryTargetHandler.java | 4 +- .../recovery/RemoteRecoveryTargetHandler.java | 4 +- .../engine/CombinedDeletionPolicyTests.java | 10 +- .../index/engine/InternalEngineTests.java | 231 ++++++++---------- .../index/shard/IndexShardTests.java | 11 +- .../index/shard/RefreshListenersTests.java | 7 +- .../index/engine/EngineTestCase.java | 59 +++-- 17 files changed, 283 insertions(+), 503 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index e5d8cacf73657..9fbff4b1b6750 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -43,15 +43,13 @@ */ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { private final TranslogDeletionPolicy translogDeletionPolicy; - private final EngineConfig.OpenMode openMode; private final LongSupplier globalCheckpointSupplier; private final ObjectIntHashMap snapshottedCommits; // Number of snapshots held against each commit point. private IndexCommit safeCommit; // the most recent safe commit point - its max_seqno at most the persisted global checkpoint. private IndexCommit lastCommit; // the most recent commit point - CombinedDeletionPolicy(EngineConfig.OpenMode openMode, TranslogDeletionPolicy translogDeletionPolicy, + CombinedDeletionPolicy(TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) { - this.openMode = openMode; this.translogDeletionPolicy = translogDeletionPolicy; this.globalCheckpointSupplier = globalCheckpointSupplier; this.snapshottedCommits = new ObjectIntHashMap<>(); @@ -59,21 +57,8 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { @Override public void onInit(List commits) throws IOException { - switch (openMode) { - case CREATE_INDEX_AND_TRANSLOG: - break; - case OPEN_INDEX_CREATE_TRANSLOG: - assert commits.isEmpty() == false : "index is opened, but we have no commits"; - // When an engine starts with OPEN_INDEX_CREATE_TRANSLOG, a new fresh index commit will be created immediately. - // We therefore can simply skip processing here as `onCommit` will be called right after with a new commit. - break; - case OPEN_INDEX_AND_TRANSLOG: - assert commits.isEmpty() == false : "index is opened, but we have no commits"; - onCommit(commits); - break; - default: - throw new IllegalArgumentException("unknown openMode [" + openMode + "]"); - } + assert commits.isEmpty() == false : "index is opened, but we have no commits"; + onCommit(commits); } @Override diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 30743c18cfe10..352c3ba3e6280 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -75,7 +75,6 @@ public final class EngineConfig { private final List internalRefreshListener; @Nullable private final Sort indexSort; - private final boolean forceNewHistoryUUID; private final TranslogRecoveryRunner translogRecoveryRunner; @Nullable private final CircuitBreakerService circuitBreakerService; @@ -113,24 +112,20 @@ public final class EngineConfig { Property.IndexScope, Property.Dynamic); private final TranslogConfig translogConfig; - private final OpenMode openMode; /** * Creates a new {@link org.elasticsearch.index.engine.EngineConfig} */ - public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, ThreadPool threadPool, + public EngineConfig(ShardId shardId, String allocationId, ThreadPool threadPool, IndexSettings indexSettings, Engine.Warmer warmer, Store store, MergePolicy mergePolicy, Analyzer analyzer, Similarity similarity, CodecService codecService, Engine.EventListener eventListener, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, - boolean forceNewHistoryUUID, TranslogConfig translogConfig, TimeValue flushMergesAfter, + TranslogConfig translogConfig, TimeValue flushMergesAfter, List externalRefreshListener, List internalRefreshListener, Sort indexSort, TranslogRecoveryRunner translogRecoveryRunner, CircuitBreakerService circuitBreakerService, LongSupplier globalCheckpointSupplier) { - if (openMode == null) { - throw new IllegalArgumentException("openMode must not be null"); - } this.shardId = shardId; this.allocationId = allocationId; this.indexSettings = indexSettings; @@ -151,8 +146,6 @@ public EngineConfig(OpenMode openMode, ShardId shardId, String allocationId, Thr this.queryCachingPolicy = queryCachingPolicy; this.translogConfig = translogConfig; this.flushMergesAfter = flushMergesAfter; - this.openMode = openMode; - this.forceNewHistoryUUID = forceNewHistoryUUID; this.externalRefreshListener = externalRefreshListener; this.internalRefreshListener = internalRefreshListener; this.indexSort = indexSort; @@ -315,22 +308,6 @@ public TranslogConfig getTranslogConfig() { */ public TimeValue getFlushMergesAfter() { return flushMergesAfter; } - /** - * Returns the {@link OpenMode} for this engine config. - */ - public OpenMode getOpenMode() { - return openMode; - } - - - /** - * Returns true if a new history uuid must be generated. If false, a new uuid will only be generated if no existing - * one is found. - */ - public boolean getForceNewHistoryUUID() { - return forceNewHistoryUUID; - } - @FunctionalInterface public interface TranslogRecoveryRunner { int run(Engine engine, Translog.Snapshot snapshot) throws IOException; @@ -343,20 +320,6 @@ public TranslogRecoveryRunner getTranslogRecoveryRunner() { return translogRecoveryRunner; } - /** - * Engine open mode defines how the engine should be opened or in other words what the engine should expect - * to recover from. We either create a brand new engine with a new index and translog or we recover from an existing index. - * If the index exists we also have the ability open only the index and create a new transaction log which happens - * during remote recovery since we have already transferred the index files but the translog is replayed from remote. The last - * and safest option opens the lucene index as well as it's referenced transaction log for a translog recovery. - * See also {@link Engine#recoverFromTranslog()} - */ - public enum OpenMode { - CREATE_INDEX_AND_TRANSLOG, - OPEN_INDEX_CREATE_TRANSLOG, - OPEN_INDEX_AND_TRANSLOG; - } - /** * The refresh listeners to add to Lucene for externally visible refreshes */ diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java index 932cdf5d2d598..935c4cd19a218 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java @@ -19,8 +19,6 @@ package org.elasticsearch.index.engine; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; @@ -32,8 +30,8 @@ import java.io.IOException; import java.nio.file.Path; +import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; @@ -83,13 +81,17 @@ public static void createNewTranslog(final Directory dir, final Path translogPat } } - private static void updateCommitData(IndexWriter writer, Map keysToUpdate) throws IOException { - List commits = DirectoryReader.listCommits(writer.getDirectory()); - if (commits.size() != 1) { - throw new UnsupportedOperationException("can only update commit data if there's a single commit, found [" + - commits.size() + "]"); + + public static void verifyHasHistoryUUID(final Directory dir) throws IOException { + try (IndexWriter writer = newIndexWriter(false, dir)) { + final Map userData = getUserData(writer); + if (userData.containsKey(Engine.HISTORY_UUID_KEY) == false) { + updateCommitData(writer, Collections.singletonMap(Engine.HISTORY_UUID_KEY, UUIDs.randomBase64UUID())); + } } + } + private static void updateCommitData(IndexWriter writer, Map keysToUpdate) throws IOException { final Map userData = getUserData(writer); userData.putAll(keysToUpdate); writer.setLiveCommitData(userData.entrySet()); @@ -102,14 +104,14 @@ private static Map getUserData(IndexWriter writer) { return userData; } - private static IndexWriter newIndexWriter(final boolean existing, final Directory dir) throws IOException { + private static IndexWriter newIndexWriter(final boolean create, final Directory dir) throws IOException { IndexWriterConfig iwc = new IndexWriterConfig(null) .setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here // we also don't specify a codec here and merges should use the engines for this index .setMergePolicy(NoMergePolicy.INSTANCE) - .setOpenMode(existing ? IndexWriterConfig.OpenMode.APPEND : IndexWriterConfig.OpenMode.CREATE); + .setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND); return new IndexWriter(dir, iwc); } } diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 1b7b891efd6ff..ffb622e294a29 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -31,7 +31,6 @@ import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.ReferenceManager; @@ -49,7 +48,6 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; -import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; @@ -131,7 +129,6 @@ public class InternalEngine extends Engine { // are falling behind and when writing indexing buffer to disk is too slow. When this is 0, there is no throttling, else we throttling // incoming indexing ops to a single thread: private final AtomicInteger throttleRequestCount = new AtomicInteger(); - private final EngineConfig.OpenMode openMode; private final AtomicBoolean pendingTranslogRecovery = new AtomicBoolean(false); public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; private final AtomicLong maxUnsafeAutoIdTimestamp = new AtomicLong(-1); @@ -155,7 +152,6 @@ public InternalEngine(EngineConfig engineConfig) { final EngineConfig engineConfig, final BiFunction localCheckpointTrackerSupplier) { super(engineConfig); - openMode = engineConfig.getOpenMode(); if (engineConfig.isAutoGeneratedIDsOptimizationEnabled() == false) { maxUnsafeAutoIdTimestamp.set(Long.MAX_VALUE); } @@ -181,22 +177,14 @@ public InternalEngine(EngineConfig engineConfig) { assert translog.getGeneration() != null; this.translog = translog; final IndexCommit startingCommit = getStartingCommitPoint(); - assert openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG || startingCommit != null : - "Starting commit should be non-null; mode [" + openMode + "]; startingCommit [" + startingCommit + "]"; + assert startingCommit != null : "Starting commit should be non-null"; this.localCheckpointTracker = createLocalCheckpointTracker(localCheckpointTrackerSupplier, startingCommit); - this.combinedDeletionPolicy = new CombinedDeletionPolicy(openMode, translogDeletionPolicy, - translog::getLastSyncedGlobalCheckpoint); - writer = createWriter(openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, startingCommit); + this.combinedDeletionPolicy = new CombinedDeletionPolicy(translogDeletionPolicy, translog::getLastSyncedGlobalCheckpoint); + writer = createWriter(startingCommit); updateMaxUnsafeAutoIdTimestampFromWriter(writer); - assert engineConfig.getForceNewHistoryUUID() == false - || openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG - || openMode == EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG - : "OpenMode must be either CREATE_INDEX_AND_TRANSLOG or OPEN_INDEX_CREATE_TRANSLOG if forceNewHistoryUUID; " + - "openMode [" + openMode + "], forceNewHistoryUUID [" + engineConfig.getForceNewHistoryUUID() + "]"; - historyUUID = loadOrGenerateHistoryUUID(writer, engineConfig.getForceNewHistoryUUID()); + historyUUID = loadOrGenerateHistoryUUID(writer); Objects.requireNonNull(historyUUID, "history uuid should not be null"); indexWriter = writer; - updateWriterOnOpen(); } catch (IOException | TranslogCorruptedException e) { throw new EngineCreationFailureException(shardId, "failed to create engine", e); } catch (AssertionError e) { @@ -215,7 +203,7 @@ public InternalEngine(EngineConfig engineConfig) { internalSearcherManager.addListener(versionMap); assert pendingTranslogRecovery.get() == false : "translog recovery can't be pending before we set it"; // don't allow commits until we are done with recovering - pendingTranslogRecovery.set(openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); + pendingTranslogRecovery.set(true); for (ReferenceManager.RefreshListener listener: engineConfig.getExternalRefreshListener()) { this.externalSearcherManager.addListener(listener); } @@ -239,20 +227,10 @@ private LocalCheckpointTracker createLocalCheckpointTracker( BiFunction localCheckpointTrackerSupplier, IndexCommit startingCommit) throws IOException { final long maxSeqNo; final long 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 SequenceNumbers.CommitInfo seqNoStats = store.loadSeqNoInfo(startingCommit); - maxSeqNo = seqNoStats.maxSeqNo; - localCheckpoint = seqNoStats.localCheckpoint; - logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); - break; - default: throw new IllegalArgumentException("unknown type: " + openMode); - } + final SequenceNumbers.CommitInfo seqNoStats = store.loadSeqNoInfo(startingCommit); + maxSeqNo = seqNoStats.maxSeqNo; + localCheckpoint = seqNoStats.localCheckpoint; + logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); return localCheckpointTrackerSupplier.apply(maxSeqNo, localCheckpoint); } @@ -378,9 +356,6 @@ public InternalEngine recoverFromTranslog() throws IOException { flushLock.lock(); try (ReleasableLock lock = readLock.acquire()) { ensureOpen(); - if (openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - throw new IllegalStateException("Can't recover from translog with open mode: " + openMode); - } if (pendingTranslogRecovery.get() == false) { throw new IllegalStateException("Engine has already been recovered"); } @@ -403,36 +378,30 @@ public InternalEngine recoverFromTranslog() throws IOException { @Override public void skipTranslogRecovery() { - if (openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - throw new IllegalStateException("Can't skip translog recovery with open mode: " + openMode); - } assert pendingTranslogRecovery.get() : "translogRecovery is not pending but should be"; pendingTranslogRecovery.set(false); // we are good - now we can commit } private IndexCommit getStartingCommitPoint() throws IOException { - if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - final long lastSyncedGlobalCheckpoint = translog.getLastSyncedGlobalCheckpoint(); - final long minRetainedTranslogGen = translog.getMinFileGeneration(); - final List existingCommits = DirectoryReader.listCommits(store.directory()); - // We may not have a safe commit if an index was create before v6.2; and if there is a snapshotted commit whose full translog - // files are not retained but max_seqno is at most the global checkpoint, we may mistakenly select it as a starting commit. - // To avoid this issue, we only select index commits whose translog files are fully retained. - if (engineConfig.getIndexSettings().getIndexVersionCreated().before(Version.V_6_2_0)) { - final List recoverableCommits = new ArrayList<>(); - for (IndexCommit commit : existingCommits) { - if (minRetainedTranslogGen <= Long.parseLong(commit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY))) { - recoverableCommits.add(commit); - } + final long lastSyncedGlobalCheckpoint = translog.getLastSyncedGlobalCheckpoint(); + final long minRetainedTranslogGen = translog.getMinFileGeneration(); + final List existingCommits = DirectoryReader.listCommits(store.directory()); + // We may not have a safe commit if an index was create before v6.2; and if there is a snapshotted commit whose full translog + // files are not retained but max_seqno is at most the global checkpoint, we may mistakenly select it as a starting commit. + // To avoid this issue, we only select index commits whose translog files are fully retained. + if (engineConfig.getIndexSettings().getIndexVersionCreated().before(Version.V_6_2_0)) { + final List recoverableCommits = new ArrayList<>(); + for (IndexCommit commit : existingCommits) { + if (minRetainedTranslogGen <= Long.parseLong(commit.getUserData().get(Translog.TRANSLOG_GENERATION_KEY))) { + recoverableCommits.add(commit); } - assert recoverableCommits.isEmpty() == false : "No commit point with full translog found; " + - "commits [" + existingCommits + "], minRetainedTranslogGen [" + minRetainedTranslogGen + "]"; - return CombinedDeletionPolicy.findSafeCommitPoint(recoverableCommits, lastSyncedGlobalCheckpoint); - } else { - return CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, lastSyncedGlobalCheckpoint); } + assert recoverableCommits.isEmpty() == false : "No commit point with full translog found; " + + "commits [" + existingCommits + "], minRetainedTranslogGen [" + minRetainedTranslogGen + "]"; + return CombinedDeletionPolicy.findSafeCommitPoint(recoverableCommits, lastSyncedGlobalCheckpoint); + } else { + return CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, lastSyncedGlobalCheckpoint); } - return null; } private void recoverFromTranslogInternal() throws IOException { @@ -451,53 +420,23 @@ private void recoverFromTranslogInternal() throws IOException { if (opsRecovered > 0) { logger.trace("flushing post recovery from translog. ops recovered [{}]. committed translog id [{}]. current id [{}]", opsRecovered, translogGeneration == null ? null : translogGeneration.translogFileGeneration, translog.currentFileGeneration()); - flush(true, true); - refresh("translog_recovery"); - } else if (translog.isCurrent(translogGeneration) == false) { - commitIndexWriter(indexWriter, translog, lastCommittedSegmentInfos.getUserData().get(Engine.SYNC_COMMIT_ID)); + commitIndexWriter(indexWriter, translog, null); refreshLastCommittedSegmentInfos(); + refresh("translog_recovery"); + translog.trimUnreferencedReaders(); } - // clean up what's not needed - translog.trimUnreferencedReaders(); } private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) throws IOException { - assert openMode != null; final TranslogConfig translogConfig = engineConfig.getTranslogConfig(); - String translogUUID = null; - if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - translogUUID = loadTranslogUUIDFromLastCommit(); - // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! - if (translogUUID == null) { - throw new IndexFormatTooOldException("translog", "translog has no generation nor a UUID - this might be an index from a previous version consider upgrading to N-1 first"); - } + final String translogUUID = loadTranslogUUIDFromLastCommit(); + // We expect that this shard already exists, so it must already have an existing translog else something is badly wrong! + if (translogUUID == null) { + throw new IndexFormatTooOldException("translog", "translog has no generation nor a UUID - this might be an index from a previous version consider upgrading to N-1 first"); } return new Translog(translogConfig, translogUUID, translogDeletionPolicy, globalCheckpointSupplier); } - /** If needed, updates the metadata in the index writer to match the potentially new translog and history uuid */ - private void updateWriterOnOpen() throws IOException { - Objects.requireNonNull(historyUUID); - final Map commitUserData = commitDataAsMap(indexWriter); - boolean needsCommit = false; - if (historyUUID.equals(commitUserData.get(HISTORY_UUID_KEY)) == false) { - needsCommit = true; - } else { - assert config().getForceNewHistoryUUID() == false : "config forced a new history uuid but it didn't change"; - assert openMode != EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG : "new index but it already has an existing history uuid"; - } - if (translog.getTranslogUUID().equals(commitUserData.get(Translog.TRANSLOG_UUID_KEY)) == false) { - needsCommit = true; - } else { - assert openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG : "translog uuid didn't change but open mode is " + openMode; - } - if (needsCommit) { - commitIndexWriter(indexWriter, translog, openMode == EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG - ? commitUserData.get(SYNC_COMMIT_ID) : null); - } - } - - @Override public Translog getTranslog() { ensureOpen(); @@ -520,31 +459,20 @@ public long getWritingBytes() { */ @Nullable private String loadTranslogUUIDFromLastCommit() throws IOException { - assert openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG : - "Only reuse existing translogUUID with OPEN_INDEX_AND_TRANSLOG; openMode = [" + openMode + "]"; final Map commitUserData = store.readLastCommittedSegmentsInfo().getUserData(); - if (commitUserData.containsKey(Translog.TRANSLOG_UUID_KEY)) { - if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) { - throw new IllegalStateException("commit doesn't contain translog generation id"); - } - return commitUserData.get(Translog.TRANSLOG_UUID_KEY); - } else { - return null; + if (commitUserData.containsKey(Translog.TRANSLOG_GENERATION_KEY) == false) { + throw new IllegalStateException("commit doesn't contain translog generation id"); } + return commitUserData.get(Translog.TRANSLOG_UUID_KEY); } /** - * Reads the current stored history ID from the IW commit data. Generates a new UUID if not found or if generation is forced. + * Reads the current stored history ID from the IW commit data. */ - private String loadOrGenerateHistoryUUID(final IndexWriter writer, boolean forceNew) throws IOException { - String uuid = commitDataAsMap(writer).get(HISTORY_UUID_KEY); - if (uuid == null || forceNew) { - assert - forceNew || // recovery from a local store creates an index that doesn't have yet a history_uuid - openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG || - config().getIndexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1) : - "existing index was created after 6_0_0_rc1 but has no history uuid"; - uuid = UUIDs.randomBase64UUID(); + private String loadOrGenerateHistoryUUID(final IndexWriter writer) throws IOException { + final String uuid = commitDataAsMap(writer).get(HISTORY_UUID_KEY); + if (uuid == null) { + throw new IllegalStateException("commit doesn't contain history uuid"); } return uuid; } @@ -1812,9 +1740,9 @@ private long loadCurrentVersionFromIndex(Term uid) throws IOException { } } - private IndexWriter createWriter(boolean create, IndexCommit startingCommit) throws IOException { + private IndexWriter createWriter(IndexCommit startingCommit) throws IOException { try { - final IndexWriterConfig iwc = getIndexWriterConfig(create, startingCommit); + final IndexWriterConfig iwc = getIndexWriterConfig(startingCommit); return createWriter(store.directory(), iwc); } catch (LockObtainFailedException ex) { logger.warn("could not lock IndexWriter", ex); @@ -1827,10 +1755,10 @@ IndexWriter createWriter(Directory directory, IndexWriterConfig iwc) throws IOEx return new IndexWriter(directory, iwc); } - private IndexWriterConfig getIndexWriterConfig(boolean create, IndexCommit startingCommit) { + private IndexWriterConfig getIndexWriterConfig(IndexCommit startingCommit) { final IndexWriterConfig iwc = new IndexWriterConfig(engineConfig.getAnalyzer()); iwc.setCommitOnClose(false); // we by default don't commit on close - iwc.setOpenMode(create ? IndexWriterConfig.OpenMode.CREATE : IndexWriterConfig.OpenMode.APPEND); + iwc.setOpenMode(IndexWriterConfig.OpenMode.APPEND); iwc.setIndexCommit(startingCommit); iwc.setIndexDeletionPolicy(combinedDeletionPolicy); // with tests.verbose, lucene sets this up: plumb to align with filesystem stream diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 3832cd0ae2055..ab1724e2c511b 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1276,37 +1276,13 @@ int runTranslogRecovery(Engine engine, Translog.Snapshot snapshot) throws IOExce return opsRecovered; } - /** creates an empty index and translog and opens the engine **/ - public void createIndexAndTranslog() throws IOException { - assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.EMPTY_STORE; - assert shardRouting.primary() && shardRouting.isRelocationTarget() == false; - // note: these are set when recovering from the translog - final RecoveryState.Translog translogStats = recoveryState().getTranslog(); - translogStats.totalOperations(0); - translogStats.totalOperationsOnStart(0); - globalCheckpointTracker.updateGlobalCheckpointOnReplica(SequenceNumbers.NO_OPS_PERFORMED, "index created"); - innerOpenEngineAndTranslog(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, false); - } - - /** opens the engine on top of the existing lucene engine but creates an empty translog **/ - public void openIndexAndCreateTranslog(boolean forceNewHistoryUUID, long globalCheckpoint) throws IOException { - assert recoveryState.getRecoverySource().getType() != RecoverySource.Type.EMPTY_STORE && - recoveryState.getRecoverySource().getType() != RecoverySource.Type.EXISTING_STORE; - SequenceNumbers.CommitInfo commitInfo = store.loadSeqNoInfo(null); - assert commitInfo.localCheckpoint >= globalCheckpoint : - "trying to create a shard whose local checkpoint [" + commitInfo.localCheckpoint + "] is < global checkpoint [" - + globalCheckpoint + "]"; - globalCheckpointTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "opening index with a new translog"); - innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG, forceNewHistoryUUID); - } - /** * opens the engine on top of the existing lucene engine and translog. * Operations from the translog will be replayed to bring lucene up to date. **/ - public void openIndexAndRecoveryFromTranslog() throws IOException { + public void openEngineAndRecoverFromTranslog() throws IOException { assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE; - innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, false); + innerOpenEngineAndTranslog(); getEngine().recoverFromTranslog(); } @@ -1314,13 +1290,13 @@ public void openIndexAndRecoveryFromTranslog() throws IOException { * Opens the engine on top of the existing lucene engine and translog. * The translog is kept but its operations won't be replayed. */ - public void openIndexAndSkipTranslogRecovery() throws IOException { + public void openEngineAndSkipTranslogRecovery() throws IOException { assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.PEER; - innerOpenEngineAndTranslog(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, false); + innerOpenEngineAndTranslog(); getEngine().skipTranslogRecovery(); } - private void innerOpenEngineAndTranslog(final EngineConfig.OpenMode openMode, final boolean forceNewHistoryUUID) throws IOException { + private void innerOpenEngineAndTranslog() throws IOException { if (state != IndexShardState.RECOVERING) { throw new IndexShardNotRecoveringException(shardId, state); } @@ -1335,28 +1311,21 @@ private void innerOpenEngineAndTranslog(final EngineConfig.OpenMode openMode, fi } recoveryState.setStage(RecoveryState.Stage.TRANSLOG); - assert openMode == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG || assertMaxUnsafeAutoIdInCommit(); - - - final EngineConfig config = newEngineConfig(openMode, forceNewHistoryUUID); + final EngineConfig config = newEngineConfig(); // 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); - if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - // we have to set it before we open an engine and recover from the translog because - // acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in, - // and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in. - globalCheckpointTracker.updateGlobalCheckpointOnReplica(Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()), - "read from translog checkpoint"); - } + // we have to set it before we open an engine and recover from the translog because + // acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in, + // and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in. + globalCheckpointTracker.updateGlobalCheckpointOnReplica(Translog.readGlobalCheckpoint(translogConfig.getTranslogPath()), + "read from translog checkpoint"); createNewEngine(config); verifyNotClosed(); - if (openMode == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - // We set active because we are now writing operations to the engine; this way, if we go idle after some time and become inactive, - // we still give sync'd flush a chance to run: - active.set(true); - } + // We set active because we are now writing operations to the engine; this way, if we go idle after some time and become inactive, + // we still give sync'd flush a chance to run: + active.set(true); assertSequenceNumbersInCommit(); assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); } @@ -2175,12 +2144,12 @@ private DocumentMapperForType docMapper(String type) { return mapperService.documentMapperWithAutoCreate(type); } - private EngineConfig newEngineConfig(EngineConfig.OpenMode openMode, final boolean forceNewHistoryUUID) { + private EngineConfig newEngineConfig() { Sort indexSort = indexSortSupplier.get(); - return new EngineConfig(openMode, shardId, shardRouting.allocationId().getId(), + return new EngineConfig(shardId, shardRouting.allocationId().getId(), threadPool, indexSettings, warmer, store, indexSettings.getMergePolicy(), mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, - indexCache.query(), cachingPolicy, forceNewHistoryUUID, translogConfig, + indexCache.query(), cachingPolicy, translogConfig, IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), Collections.singletonList(refreshListeners), Collections.singletonList(new RefreshMetricUpdater(refreshMetric)), diff --git a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java index 81ffbea642c58..990cbe3aa3b34 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/elasticsearch/index/shard/StoreRecovery.java @@ -40,6 +40,7 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.Index; +import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.mapper.MapperService; @@ -389,8 +390,8 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe recoveryState.getIndex().updateVersion(version); if (recoveryState.getRecoverySource().getType() == RecoverySource.Type.LOCAL_SHARDS) { assert indexShouldExists; - indexShard.openIndexAndCreateTranslog(true, store.loadSeqNoInfo(null).localCheckpoint); - } else { + EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), indexShard.shardPath().resolveTranslog(), shardId); + } else if (indexShouldExists) { // since we recover from local, just fill the files and size try { final RecoveryState.Index index = recoveryState.getIndex(); @@ -400,13 +401,11 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe } catch (IOException e) { logger.debug("failed to list file details", e); } - if (indexShouldExists) { - indexShard.openIndexAndRecoveryFromTranslog(); - indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); - } else { - indexShard.createIndexAndTranslog(); - } + } else { + EngineDiskUtils.createEmpty(store.directory(), indexShard.shardPath().resolveTranslog(), shardId); } + indexShard.openEngineAndRecoverFromTranslog(); + indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("post recovery from shard_store"); } catch (EngineException | IOException e) { @@ -446,16 +445,10 @@ private void restore(final IndexShard indexShard, final Repository repository, f } final IndexId indexId = repository.getRepositoryData().resolveIndexId(indexName); repository.restoreShard(indexShard, restoreSource.snapshot().getSnapshotId(), restoreSource.version(), indexId, snapshotShardId, indexShard.recoveryState()); - final Store store = indexShard.store(); - final long localCheckpoint; - store.incRef(); - try { - localCheckpoint = store.loadSeqNoInfo(null).localCheckpoint; - } finally { - store.decRef(); - } - indexShard.openIndexAndCreateTranslog(true, localCheckpoint); + EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(indexShard.store().directory(), indexShard.shardPath().resolveTranslog(), + shardId); assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; + indexShard.openEngineAndRecoverFromTranslog(); indexShard.getEngine().fillSeqNoGaps(indexShard.getPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index 9f89752e51fa1..e7b7ee220db4e 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -133,23 +133,19 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC * translog file referenced by this generation. The translog creation will fail if this generation can't be opened. * * @param config the configuration of this translog - * @param expectedTranslogUUID the translog uuid to open, null for a new translog + * @param translogUUID the translog uuid to open, null for a new translog * @param deletionPolicy an instance of {@link TranslogDeletionPolicy} that controls when a translog file can be safely * deleted * @param globalCheckpointSupplier a supplier for the global checkpoint */ public Translog( - final TranslogConfig config, final String expectedTranslogUUID, TranslogDeletionPolicy deletionPolicy, + final TranslogConfig config, final String translogUUID, TranslogDeletionPolicy deletionPolicy, final LongSupplier globalCheckpointSupplier) throws IOException { super(config.getShardId(), config.getIndexSettings()); this.config = config; this.globalCheckpointSupplier = globalCheckpointSupplier; this.deletionPolicy = deletionPolicy; - if (expectedTranslogUUID == null) { - translogUUID = UUIDs.randomBase64UUID(); - } else { - translogUUID = expectedTranslogUUID; - } + this.translogUUID = translogUUID; bigArrays = config.getBigArrays(); ReadWriteLock rwl = new ReentrantReadWriteLock(); readLock = new ReleasableLock(rwl.readLock()); @@ -158,53 +154,38 @@ public Translog( Files.createDirectories(this.location); try { - if (expectedTranslogUUID != null) { - final Checkpoint checkpoint = readCheckpoint(location); - final Path nextTranslogFile = location.resolve(getFilename(checkpoint.generation + 1)); - final Path currentCheckpointFile = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); - // this is special handling for error condition when we create a new writer but we fail to bake - // the newly written file (generation+1) into the checkpoint. This is still a valid state - // we just need to cleanup before we continue - // we hit this before and then blindly deleted the new generation even though we managed to bake it in and then hit this: - // https://discuss.elastic.co/t/cannot-recover-index-because-of-missing-tanslog-files/38336 as an example - // - // For this to happen we must have already copied the translog.ckp file into translog-gen.ckp so we first check if that file exists - // if not we don't even try to clean it up and wait until we fail creating it - assert Files.exists(nextTranslogFile) == false || Files.size(nextTranslogFile) <= TranslogWriter.getHeaderLength(expectedTranslogUUID) : "unexpected translog file: [" + nextTranslogFile + "]"; - if (Files.exists(currentCheckpointFile) // current checkpoint is already copied - && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning - logger.warn("deleted previously created, but not yet committed, next generation [{}]. This can happen due to a tragic exception when creating a new generation", nextTranslogFile.getFileName()); - } - this.readers.addAll(recoverFromFiles(checkpoint)); - if (readers.isEmpty()) { - throw new IllegalStateException("at least one reader must be recovered"); - } - boolean success = false; - current = null; - try { - 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 - // for instance if we have a lot of tlog and we can't create the writer we keep on holding - // on to all the uncommitted tlog files if we don't close - if (success == false) { - IOUtils.closeWhileHandlingException(readers); - } + final Checkpoint checkpoint = readCheckpoint(location); + final Path nextTranslogFile = location.resolve(getFilename(checkpoint.generation + 1)); + final Path currentCheckpointFile = location.resolve(getCommitCheckpointFileName(checkpoint.generation)); + // this is special handling for error condition when we create a new writer but we fail to bake + // the newly written file (generation+1) into the checkpoint. This is still a valid state + // we just need to cleanup before we continue + // we hit this before and then blindly deleted the new generation even though we managed to bake it in and then hit this: + // https://discuss.elastic.co/t/cannot-recover-index-because-of-missing-tanslog-files/38336 as an example + // + // For this to happen we must have already copied the translog.ckp file into translog-gen.ckp so we first check if that file exists + // if not we don't even try to clean it up and wait until we fail creating it + assert Files.exists(nextTranslogFile) == false || Files.size(nextTranslogFile) <= TranslogWriter.getHeaderLength(translogUUID) : "unexpected translog file: [" + nextTranslogFile + "]"; + if (Files.exists(currentCheckpointFile) // current checkpoint is already copied + && Files.deleteIfExists(nextTranslogFile)) { // delete it and log a warning + logger.warn("deleted previously created, but not yet committed, next generation [{}]. This can happen due to a tragic exception when creating a new generation", nextTranslogFile.getFileName()); + } + this.readers.addAll(recoverFromFiles(checkpoint)); + if (readers.isEmpty()) { + throw new IllegalStateException("at least one reader must be recovered"); + } + boolean success = false; + current = null; + try { + 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 + // for instance if we have a lot of tlog and we can't create the writer we keep on holding + // on to all the uncommitted tlog files if we don't close + if (success == false) { + IOUtils.closeWhileHandlingException(readers); } - } else { - IOUtils.rm(location); - // start from whatever generation lucene points to - final long generation = deletionPolicy.getMinTranslogGenerationForRecovery(); - logger.debug("wipe translog location - creating new translog, starting generation [{}]", generation); - Files.createDirectories(location); - final long initialGlobalCheckpoint = globalCheckpointSupplier.getAsLong(); - 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, initialGlobalCheckpoint); - readers.clear(); } } catch (Exception e) { // close the opened translog files if we fail to create a new translog... diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 5920e286aa194..553a7e7842f00 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -393,7 +393,7 @@ class PrepareForTranslogOperationsRequestHandler implements TransportRequestHand public void messageReceived(RecoveryPrepareForTranslogOperationsRequest request, TransportChannel channel) throws Exception { try (RecoveryRef recoveryRef = onGoingRecoveries.getRecoverySafe(request.recoveryId(), request.shardId() )) { - recoveryRef.target().prepareForTranslogOperations(request.createNewTranslog(), request.totalTranslogOps()); + recoveryRef.target().prepareForTranslogOperations(request.isFileBasedRecovery(), request.totalTranslogOps()); } channel.sendResponse(TransportResponse.Empty.INSTANCE); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java index 28df2897d9778..65ccb078c94a0 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryPrepareForTranslogOperationsRequest.java @@ -33,13 +33,13 @@ class RecoveryPrepareForTranslogOperationsRequest extends TransportRequest { private final long recoveryId; private final ShardId shardId; private final int totalTranslogOps; - private final boolean createNewTranslog; + private final boolean fileBasedRecovery; - RecoveryPrepareForTranslogOperationsRequest(long recoveryId, ShardId shardId, int totalTranslogOps, boolean createNewTranslog) { + RecoveryPrepareForTranslogOperationsRequest(long recoveryId, ShardId shardId, int totalTranslogOps, boolean fileBasedRecovery) { this.recoveryId = recoveryId; this.shardId = shardId; this.totalTranslogOps = totalTranslogOps; - this.createNewTranslog = createNewTranslog; + this.fileBasedRecovery = fileBasedRecovery; } RecoveryPrepareForTranslogOperationsRequest(StreamInput in) throws IOException { @@ -51,9 +51,9 @@ class RecoveryPrepareForTranslogOperationsRequest extends TransportRequest { in.readLong(); // maxUnsafeAutoIdTimestamp } if (in.getVersion().onOrAfter(Version.V_6_2_0)) { - createNewTranslog = in.readBoolean(); + fileBasedRecovery = in.readBoolean(); } else { - createNewTranslog = true; + fileBasedRecovery = true; } } @@ -70,10 +70,10 @@ public int totalTranslogOps() { } /** - * Whether or not the recover target should create a new local translog + * Whether or not the recovery is file based */ - boolean createNewTranslog() { - return createNewTranslog; + public boolean isFileBasedRecovery() { + return fileBasedRecovery; } @Override @@ -86,7 +86,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP); // maxUnsafeAutoIdTimestamp } if (out.getVersion().onOrAfter(Version.V_6_2_0)) { - out.writeBoolean(createNewTranslog); + out.writeBoolean(fileBasedRecovery); } } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 1bbcb9efa9644..2bc953a88a9fc 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -31,6 +31,7 @@ import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; @@ -40,6 +41,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.mapper.MapperException; import org.elasticsearch.index.seqno.GlobalCheckpointTracker; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -362,14 +364,9 @@ private void ensureRefCount() { /*** Implementation of {@link RecoveryTargetHandler } */ @Override - public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException { + public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException { state().getTranslog().totalOperations(totalTranslogOps); - if (createNewTranslog) { - // TODO: Assigns the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 - indexShard().openIndexAndCreateTranslog(false, SequenceNumbers.UNASSIGNED_SEQ_NO); - } else { - indexShard().openIndexAndSkipTranslogRecovery(); - } + indexShard().openEngineAndSkipTranslogRecovery(); } @Override @@ -442,6 +439,12 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa final Store store = store(); try { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); + if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) { + EngineDiskUtils.verifyHasHistoryUUID(store.directory()); + } + // TODO: Assigns the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 + EngineDiskUtils.createNewTranslog(store.directory(), indexShard.shardPath().resolveTranslog(), + SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. // this means we transferred files from the remote that have not be checksummed and they are diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index 736d602044656..f37938e947ea8 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -32,10 +32,10 @@ public interface RecoveryTargetHandler { /** * Prepares the target to receive translog operations, after all file have been copied - * @param createNewTranslog whether or not to delete the local translog on the target + * @param fileBasedRecovery whether or not this call is part of an file based recovery * @param totalTranslogOps total translog operations expected to be sent */ - void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException; + void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException; /** * The finalize request refreshes the engine now that new segments are available, enables garbage collection of tombstone files, and diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 4ea2be0e72659..9d89f23b0d303 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -76,9 +76,9 @@ public RemoteRecoveryTargetHandler(long recoveryId, ShardId shardId, TransportSe } @Override - public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException { + public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException { transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.PREPARE_TRANSLOG, - new RecoveryPrepareForTranslogOperationsRequest(recoveryId, shardId, totalTranslogOps, createNewTranslog), + new RecoveryPrepareForTranslogOperationsRequest(recoveryId, shardId, totalTranslogOps, fileBasedRecovery), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionTimeout()).build(), EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java index e74cde52aa418..7d8d2c3b70a7a 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/CombinedDeletionPolicyTests.java @@ -38,8 +38,6 @@ import java.util.concurrent.atomic.AtomicLong; import static java.util.Collections.singletonList; -import static org.elasticsearch.index.engine.EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG; -import static org.elasticsearch.index.engine.EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.equalTo; import static org.mockito.Mockito.doAnswer; @@ -54,7 +52,7 @@ public class CombinedDeletionPolicyTests extends ESTestCase { public void testKeepCommitsAfterGlobalCheckpoint() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(translogPolicy, globalCheckpoint::get); final LongArrayList maxSeqNoList = new LongArrayList(); final LongArrayList translogGenList = new LongArrayList(); @@ -93,7 +91,7 @@ public void testAcquireIndexCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(); final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(translogPolicy, globalCheckpoint::get); long lastMaxSeqNo = between(1, 1000); long lastTranslogGen = between(1, 20); int safeIndex = 0; @@ -156,7 +154,7 @@ public void testLegacyIndex() throws Exception { final UUID translogUUID = UUID.randomUUID(); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_AND_TRANSLOG, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(translogPolicy, globalCheckpoint::get); long legacyTranslogGen = randomNonNegativeLong(); IndexCommit legacyCommit = mockLegacyIndexCommit(translogUUID, legacyTranslogGen); @@ -188,7 +186,7 @@ public void testLegacyIndex() throws Exception { public void testDeleteInvalidCommits() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(randomNonNegativeLong()); TranslogDeletionPolicy translogPolicy = createTranslogDeletionPolicy(); - CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(OPEN_INDEX_CREATE_TRANSLOG, translogPolicy, globalCheckpoint::get); + CombinedDeletionPolicy indexPolicy = new CombinedDeletionPolicy(translogPolicy, globalCheckpoint::get); final int invalidCommits = between(1, 10); final List commitList = new ArrayList<>(); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index a508d691ed3a6..30d5d24a95ab5 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -79,7 +79,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -637,7 +636,7 @@ public IndexSearcher wrap(IndexSearcher searcher) throws EngineException { InternalEngine engine = createEngine(store, translog); engine.close(); - engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + engine = new InternalEngine(engine.config()); assertTrue(engine.isRecovering()); engine.recoverFromTranslog(); Engine.Searcher searcher = wrapper.wrap(engine.acquireSearcher("test")); @@ -652,7 +651,7 @@ public void testFlushIsDisabledDuringTranslogRecovery() throws IOException { engine.index(indexForDoc(doc)); engine.close(); - engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + engine = new InternalEngine(engine.config()); expectThrows(IllegalStateException.class, () -> engine.flush(true, true)); assertTrue(engine.isRecovering()); engine.recoverFromTranslog(); @@ -686,7 +685,7 @@ public void testTranslogMultipleOperationsSameDocument() throws IOException { Engine recoveringEngine = null; try { - recoveringEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + recoveringEngine = new InternalEngine(engine.config()); recoveringEngine.recoverFromTranslog(); try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) { final TotalHitCountCollector collector = new TotalHitCountCollector(); @@ -714,20 +713,19 @@ public void testTranslogRecoveryDoesNotReplayIntoTranslog() throws IOException { Engine recoveringEngine = null; try { - final AtomicBoolean flushed = new AtomicBoolean(); - recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)) { + final AtomicBoolean committed = new AtomicBoolean(); + recoveringEngine = new InternalEngine(initialEngine.config()) { + @Override - public CommitId flush(boolean force, boolean waitIfOngoing) throws EngineException { - assertThat(getTranslog().uncommittedOperations(), equalTo(docs)); - final CommitId commitId = super.flush(force, waitIfOngoing); - flushed.set(true); - return commitId; + protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { + committed.set(true); + super.commitIndexWriter(writer, translog, syncId); } }; assertThat(recoveringEngine.getTranslog().uncommittedOperations(), equalTo(docs)); recoveringEngine.recoverFromTranslog(); - assertTrue(flushed.get()); + assertTrue(committed.get()); } finally { IOUtils.close(recoveringEngine); } @@ -758,7 +756,7 @@ public void testTranslogRecoveryWithMultipleGenerations() throws IOException { } } initialEngine.close(); - recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + recoveringEngine = new InternalEngine(initialEngine.config()); recoveringEngine.recoverFromTranslog(); try (Engine.Searcher searcher = recoveringEngine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), docs); @@ -1005,9 +1003,9 @@ public void testCommitAdvancesMinTranslogForRecovery() throws IOException { IOUtils.close(engine, store); final Path translogPath = createTempDir(); store = createStore(); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); - engine = new InternalEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier)); + engine = createEngine(config(defaultSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier)); ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); boolean inSync = randomBoolean(); @@ -1016,30 +1014,30 @@ public void testCommitAdvancesMinTranslogForRecovery() throws IOException { } engine.flush(); - assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 2L : 1L)); - assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(2L)); + assertThat(engine.getTranslog().currentFileGeneration(), equalTo(3L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 3L : 1L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); engine.flush(); - assertThat(engine.getTranslog().currentFileGeneration(), equalTo(2L)); - 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 ? 3L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(3L)); - globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); engine.flush(true, true); assertThat(engine.getTranslog().currentFileGeneration(), equalTo(4L)); - assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(4L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(inSync ? 4L : 1L)); assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(4L)); + + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); + engine.flush(true, true); + assertThat(engine.getTranslog().currentFileGeneration(), equalTo(5L)); + assertThat(engine.getTranslog().getDeletionPolicy().getMinTranslogGenerationForRecovery(), equalTo(5L)); + assertThat(engine.getTranslog().getDeletionPolicy().getTranslogGenerationOfLastCommit(), equalTo(5L)); } public void testSyncedFlush() throws IOException { try (Store store = createStore(); - Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null))) { + Engine engine = createEngine(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null)) { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), B_1, null); engine.index(indexForDoc(doc)); @@ -1065,8 +1063,8 @@ public void testRenewSyncFlush() throws Exception { final int iters = randomIntBetween(2, 5); // run this a couple of times to get some coverage for (int i = 0; i < iters; i++) { try (Store store = createStore(); - InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), - new LogDocMergePolicy(), null))) { + InternalEngine engine = + createEngine(config(defaultSettings, store, createTempDir(), new LogDocMergePolicy(), null))) { final String syncId = randomUnicodeOfCodepointLengthBetween(10, 20); Engine.Index doc1 = indexForDoc(testParsedDocument("1", null, testDocumentWithTextField(), B_1, null)); engine.index(doc1); @@ -1121,7 +1119,7 @@ public void testRenewSyncFlush() throws Exception { } public void testSyncedFlushSurvivesEngineRestart() throws IOException { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); IOUtils.close(store, engine); store = createStore(); engine = createEngine(store, primaryTranslogDir, globalCheckpoint::get); @@ -1140,12 +1138,13 @@ public void testSyncedFlushSurvivesEngineRestart() throws IOException { } else { engine.flushAndClose(); } - engine = new InternalEngine(copy(config, randomFrom(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))); - - if (engine.config().getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG && randomBoolean()) { - engine.recoverFromTranslog(); + if (randomBoolean()) { + EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), + SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); } - assertEquals(engine.config().getOpenMode().toString(), engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); + engine = new InternalEngine(config); + engine.recoverFromTranslog(); + assertEquals(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID), syncId); } public void testSyncedFlushVanishesOnReplay() throws IOException { @@ -1161,7 +1160,7 @@ public void testSyncedFlushVanishesOnReplay() throws IOException { engine.index(indexForDoc(doc)); EngineConfig config = engine.config(); engine.close(); - engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + engine = new InternalEngine(config); engine.recoverFromTranslog(); assertNull("Sync ID must be gone since we have a document to replay", engine.getLastCommittedSegmentInfos().getUserData().get(Engine.SYNC_COMMIT_ID)); } @@ -1266,7 +1265,7 @@ public void testVersioningNewIndex() throws IOException { public void testForceMerge() throws IOException { try (Store store = createStore(); - Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), + Engine engine = createEngine(config(defaultSettings, store, createTempDir(), new LogByteSizeMergePolicy(), null))) { // use log MP here we test some behavior in ESMP int numDocs = randomIntBetween(10, 100); for (int i = 0; i < numDocs; i++) { @@ -2047,7 +2046,7 @@ public void testSeqNoAndCheckpoints() throws IOException { InternalEngine recoveringEngine = null; try { - recoveringEngine = new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + recoveringEngine = new InternalEngine(initialEngine.config()); recoveringEngine.recoverFromTranslog(); assertEquals(primarySeqNo, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); @@ -2076,8 +2075,7 @@ public void testSeqNoAndCheckpoints() throws IOException { public void testConcurrentWritesAndCommits() throws Exception { List commits = new ArrayList<>(); try (Store store = createStore(); - InternalEngine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) { - + InternalEngine engine = createEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) { final int numIndexingThreads = scaledRandomIntBetween(2, 4); final int numDocsPerThread = randomIntBetween(500, 1000); final CyclicBarrier barrier = new CyclicBarrier(numIndexingThreads + 1); @@ -2241,7 +2239,7 @@ public void testIndexWriterIFDInfoStream() throws IllegalAccessException, IOExce public void testEnableGcDeletes() throws Exception { try (Store store = createStore(); - Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) { + Engine engine = createEngine(config(defaultSettings, store, createTempDir(), newMergePolicy(), null))) { engine.config().setEnableGcDeletes(false); final BiFunction searcherFactory = engine::acquireSearcher; @@ -2324,7 +2322,7 @@ public void testFailStart() throws IOException { InternalEngine holder; try { holder = createEngine(store, translogPath); - } catch (EngineCreationFailureException ex) { + } catch (EngineCreationFailureException | IOException ex) { assertEquals(store.refCount(), refCount); continue; } @@ -2370,9 +2368,9 @@ public void testMissingTranslog() throws IOException { } catch (EngineCreationFailureException ex) { // expected } - // now it should be OK. - EngineConfig config = copy(config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null), - EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG); + // when a new translog is created it should be ok + EngineDiskUtils.createNewTranslog(store.directory(), primaryTranslogDir, SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); + EngineConfig config = config(defaultSettings, store, primaryTranslogDir, newMergePolicy(), null); engine = new InternalEngine(config); } @@ -2447,8 +2445,9 @@ 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 AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final LongSupplier globalCheckpointSupplier = () -> globalCheckpoint.get(); + EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId); try (InternalEngine engine = new InternalEngine(config(indexSettings, store, translogPath, newMergePolicy(), null, null, globalCheckpointSupplier)) { @@ -2461,6 +2460,7 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s } } }) { + engine.recoverFromTranslog(); final ParsedDocument doc1 = testParsedDocument("1", null, testDocumentWithTextField(), SOURCE, null); engine.index(indexForDoc(doc1)); globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); @@ -2493,7 +2493,8 @@ public void testSkipTranslogReplay() throws IOException { } assertVisibleCount(engine, numDocs); engine.close(); - engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG)); + engine = new InternalEngine(engine.config()); + engine.skipTranslogRecovery(); try (Engine.Searcher searcher = engine.acquireSearcher("test")) { TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), randomIntBetween(numDocs, numDocs + 10)); assertThat(topDocs.totalHits, equalTo(0L)); @@ -2533,7 +2534,7 @@ public void testTranslogReplay() throws IOException { parser.mappingUpdate = dynamicUpdate(); engine.close(); - engine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, inSyncGlobalCheckpointSupplier)); // we need to reuse the engine config unless the parser.mappingModified won't work + engine = new InternalEngine(copy(engine.config(), inSyncGlobalCheckpointSupplier)); // we need to reuse the engine config unless the parser.mappingModified won't work engine.recoverFromTranslog(); assertVisibleCount(engine, numDocs, false); @@ -2606,9 +2607,11 @@ public void testRecoverFromForeignTranslog() throws IOException { Translog.TranslogGeneration generation = engine.getTranslog().getGeneration(); engine.close(); + final Path badTranslogLog = createTempDir(); + final String badUUID = Translog.createEmptyTranslog(badTranslogLog, SequenceNumbers.NO_OPS_PERFORMED, shardId); Translog translog = new Translog( - new TranslogConfig(shardId, createTempDir(), INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE), - null, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + new TranslogConfig(shardId, badTranslogLog, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE), + badUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED); translog.add(new Translog.Index("test", "SomeBogusId", 0, "{}".getBytes(Charset.forName("UTF-8")))); assertEquals(generation.translogFileGeneration, translog.currentFileGeneration()); translog.close(); @@ -2618,10 +2621,10 @@ public void testRecoverFromForeignTranslog() throws IOException { TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE); - EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, allocationId.getId(), + EngineConfig brokenConfig = new EngineConfig(shardId, allocationId.getId(), threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, TimeValue.timeValueMinutes(5), + IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); try { @@ -2663,28 +2666,24 @@ public void testHistoryUUIDIsSetIfMissing() throws IOException { writer.commit(); } + EngineDiskUtils.verifyHasHistoryUUID(store.directory()); + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() .put(defaultSettings.getSettings()) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0_beta1) .build()); EngineConfig config = engine.config(); - EngineConfig newConfig = new EngineConfig( - randomBoolean() ? EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG : EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG, shardId, allocationId.getId(), threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), false, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), + IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); engine = new InternalEngine(newConfig); - if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - engine.recoverFromTranslog(); - assertVisibleCount(engine, numDocs, false); - } else { - assertVisibleCount(engine, 0, false); - } + engine.recoverFromTranslog(); + assertVisibleCount(engine, numDocs, false); assertThat(engine.getHistoryUUID(), notNullValue()); } @@ -2700,25 +2699,18 @@ public void testHistoryUUIDCanBeForced() throws IOException { final String oldHistoryUUID = engine.getHistoryUUID(); engine.close(); EngineConfig config = engine.config(); + EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); EngineConfig newConfig = new EngineConfig( - randomBoolean() ? EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG : EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG, shardId, allocationId.getId(), threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), true, config.getTranslogConfig(), TimeValue.timeValueMinutes(5), + IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); - if (newConfig.getOpenMode() == EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG) { - Lucene.cleanLuceneIndex(store.directory()); - } + new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); engine = new InternalEngine(newConfig); - if (newConfig.getOpenMode() == EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG) { - engine.recoverFromTranslog(); - assertVisibleCount(engine, numDocs, false); - } else { - assertVisibleCount(engine, 0, false); - } + engine.recoverFromTranslog(); + assertVisibleCount(engine, 0, false); assertThat(engine.getHistoryUUID(), not(equalTo(oldHistoryUUID))); } @@ -2815,17 +2807,17 @@ protected void doRun() throws Exception { } public void testCurrentTranslogIDisCommitted() throws IOException { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); try (Store store = createStore()) { EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); // create { + EngineDiskUtils.createEmpty(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG))){ - assertFalse(engine.isRecovering()); + try (InternalEngine engine = createEngine(config)){ engine.index(firstIndexRequest); globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); @@ -2837,12 +2829,15 @@ public void testCurrentTranslogIDisCommitted() throws IOException { // open and recover tlog { for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { + try (InternalEngine engine = new InternalEngine(config)) { assertTrue(engine.isRecovering()); Map userData = engine.getLastCommittedSegmentInfos().getUserData(); if (i == 0) { assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); } else { + // creating an empty index will create the first translog gen and commit it + // opening the empty index will make the second translog file but not commit it + // opening the engine again (i=0) will make the third translog file, which then be committed assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); } assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); @@ -2855,12 +2850,14 @@ public void testCurrentTranslogIDisCommitted() throws IOException { } // open index with new tlog { - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) { + EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), + SequenceNumbers.NO_OPS_PERFORMED, shardId); + try (InternalEngine engine = new InternalEngine(config)) { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); - assertEquals(1, engine.getTranslog().currentFileGeneration()); + engine.recoverFromTranslog(); + assertEquals(2, engine.getTranslog().currentFileGeneration()); assertEquals(0L, engine.getTranslog().uncommittedOperations()); } } @@ -2868,7 +2865,7 @@ public void testCurrentTranslogIDisCommitted() throws IOException { // open and recover tlog with empty tlog { for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { + try (InternalEngine engine = new InternalEngine(config)) { Map userData = engine.getLastCommittedSegmentInfos().getUserData(); assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); @@ -3369,7 +3366,7 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { final long timestamp2 = randomNonNegativeLong(); final long maxTimestamp12 = Math.max(timestamp1, timestamp2); try (Store store = createStore(newFSDirectory(storeDir)); - Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { + Engine engine = createEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); @@ -3387,11 +3384,13 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); engine.flush(); } - try (Store store = createStore(newFSDirectory(storeDir)); - Engine engine = new InternalEngine( - copy(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null), - randomFrom(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG)))) { - assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); + try (Store store = createStore(newFSDirectory(storeDir))) { + if (randomBoolean()) { + EngineDiskUtils.createNewTranslog(store.directory(), translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId); + } + try (Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { + assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); + } } } @@ -3487,7 +3486,7 @@ public void afterRefresh(boolean didRefresh) throws IOException { } } }); - InternalEngine internalEngine = new InternalEngine(config); + InternalEngine internalEngine = createEngine(config); int docId = 0; final ParsedDocument doc = testParsedDocument(Integer.toString(docId), null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); @@ -3658,53 +3657,13 @@ public void testSequenceNumberAdvancesToMaxSeqOnEngineOpenOnPrimary() throws Bro IOUtils.close(initialEngine); } - try (Engine recoveringEngine = - new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { + try (Engine recoveringEngine = new InternalEngine(initialEngine.config())) { recoveringEngine.recoverFromTranslog(); recoveringEngine.fillSeqNoGaps(2); assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (docs - 1))); } } - public void testSequenceNumberAdvancesToMaxSeqNoOnEngineOpenOnReplica() throws IOException { - final long v = 1; - final VersionType t = VersionType.EXTERNAL; - final long ts = IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP; - final int docs = randomIntBetween(1, 32); - InternalEngine initialEngine = null; - try { - initialEngine = engine; - for (int i = 0; i < docs; i++) { - final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); - final Term uid = newUid(doc); - // create a gap at sequence number 3 * i + 1 - initialEngine.index(new Engine.Index(uid, doc, 3 * i, 1, v, t, REPLICA, System.nanoTime(), ts, false)); - initialEngine.delete(new Engine.Delete("type", id, uid, 3 * i + 2, 1, v, t, REPLICA, System.nanoTime())); - } - - // bake the commit with the local checkpoint stuck at 0 and gaps all along the way up to the max sequence number - 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++) { - final String id = Integer.toString(i); - final ParsedDocument doc = testParsedDocument(id, null, testDocumentWithTextField(), SOURCE, null); - final Term uid = newUid(doc); - initialEngine.index(new Engine.Index(uid, doc, 3 * i + 1, 1, v, t, REPLICA, System.nanoTime(), ts, false)); - } - } finally { - IOUtils.close(initialEngine); - } - - try (Engine recoveringEngine = - new InternalEngine(copy(initialEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { - recoveringEngine.recoverFromTranslog(); - recoveringEngine.fillSeqNoGaps(1); - assertThat(recoveringEngine.getLocalCheckpointTracker().getCheckpoint(), greaterThanOrEqualTo((long) (3 * (docs - 1) + 2 - 1))); - } - } /** java docs */ public void testOutOfOrderSequenceNumbersWithVersionConflict() throws IOException { @@ -3799,7 +3758,7 @@ public void testNoOps() throws IOException { final BiFunction supplier = (ms, lcp) -> new LocalCheckpointTracker( maxSeqNo, localCheckpoint); - noOpEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier) { + noOpEngine = new InternalEngine(engine.config(), supplier) { @Override protected long doGenerateSeqNoForOperation(Operation operation) { throw new UnsupportedOperationException(); @@ -3946,7 +3905,7 @@ public void markSeqNoAsCompleted(long seqNo) { completedSeqNos.add(seqNo); } }; - actualEngine = new InternalEngine(copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG), supplier); + actualEngine = new InternalEngine(engine.config(), supplier); final int operations = randomIntBetween(0, 1024); final Set expectedCompletedSeqNos = new HashSet<>(); for (int i = 0; i < operations; i++) { @@ -4015,7 +3974,7 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { 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)); + recoveringEngine = new InternalEngine(replicaEngine.config()); assertEquals(numDocsOnReplica, recoveringEngine.getTranslog().uncommittedOperations()); recoveringEngine.recoverFromTranslog(); assertEquals(maxSeqIDOnReplica, recoveringEngine.getLocalCheckpointTracker().getMaxSeqNo()); @@ -4046,7 +4005,7 @@ public void testFillUpSequenceIdGapsOnRecovery() throws IOException { // now do it again to make sure we preserve values etc. try { - recoveringEngine = new InternalEngine(copy(replicaEngine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG)); + recoveringEngine = new InternalEngine(replicaEngine.config()); if (flushed) { assertEquals(0, recoveringEngine.getTranslog().uncommittedOperations()); } @@ -4219,10 +4178,11 @@ public void testKeepTranslogAfterGlobalCheckpoint() throws Exception { final Path translogPath = createTempDir(); store = createStore(); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final EngineConfig engineConfig = config(indexSettings, store, translogPath, NoMergePolicy.INSTANCE, null, null, () -> globalCheckpoint.get()); + EngineDiskUtils.createEmpty(store.directory(), translogPath, shardId); try (Engine engine = new InternalEngine(engineConfig) { @Override protected void commitIndexWriter(IndexWriter writer, Translog translog, String syncId) throws IOException { @@ -4234,6 +4194,7 @@ protected void commitIndexWriter(IndexWriter writer, Translog translog, String s super.commitIndexWriter(writer, translog, syncId); } }) { + engine.recoverFromTranslog(); int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { ParseContext.Document document = testDocumentWithTextField(); @@ -4324,7 +4285,7 @@ public void testConcurrentAppendUpdateAndRefresh() throws InterruptedException, public void testAcquireIndexCommit() throws Exception { IOUtils.close(engine, store); store = createStore(); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) { int numDocs = between(1, 20); for (int i = 0; i < numDocs; i++) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index cd75c7a08fbc3..f2355c388db8e 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -150,7 +150,6 @@ import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasKey; -import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.lessThan; @@ -2108,7 +2107,7 @@ public void testShardActiveDuringInternalRecovery() throws IOException { shard.prepareForIndexRecovery(); // Shard is still inactive since we haven't started recovering yet assertFalse(shard.isActive()); - shard.openIndexAndRecoveryFromTranslog(); + shard.openEngineAndRecoverFromTranslog(); // Shard should now be active since we did recover: assertTrue(shard.isActive()); closeShards(shard); @@ -2136,8 +2135,8 @@ public void testShardActiveDuringPeerRecovery() throws IOException { new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> { }) { @Override - public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException { - super.prepareForTranslogOperations(createNewTranslog, totalTranslogOps); + public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException { + super.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps); // Shard is still inactive since we haven't started recovering yet assertFalse(replica.isActive()); @@ -2185,8 +2184,8 @@ public void testRefreshListenersDuringPeerRecovery() throws IOException { }) { // we're only checking that listeners are called when the engine is open, before there is no point @Override - public void prepareForTranslogOperations(boolean createNewTranslog, int totalTranslogOps) throws IOException { - super.prepareForTranslogOperations(createNewTranslog, totalTranslogOps); + public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException { + super.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps); assertListenerCalled.accept(replica); } diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index 6024daf45ccd3..d42479a57389d 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -48,7 +48,6 @@ 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; @@ -57,9 +56,9 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.threadpool.Scheduler.Cancellable; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.threadpool.Scheduler.Cancellable; import org.elasticsearch.threadpool.ThreadPool.Names; import org.junit.After; import org.junit.Before; @@ -121,9 +120,9 @@ public void onFailedEngine(String reason, @Nullable Exception e) { // we don't need to notify anybody in this test } }; - EngineConfig config = new EngineConfig(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, shardId, allocationId, threadPool, + EngineConfig config = new EngineConfig(shardId, allocationId, threadPool, indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), - eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, + eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, null, new NoneCircuitBreakerService(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); 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 2b25ef160b44a..2c3e73d589631 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 @@ -36,7 +36,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -155,24 +154,20 @@ public void setUp() throws Exception { } } - public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode) { - return copy(config, openMode, config.getAnalyzer()); - } - - public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, LongSupplier globalCheckpointSupplier) { - return new EngineConfig(openMode, config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), + public EngineConfig copy(EngineConfig config, LongSupplier globalCheckpointSupplier) { + return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(), config.getStore(), config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), - config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), + config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(), globalCheckpointSupplier); } - public EngineConfig copy(EngineConfig config, EngineConfig.OpenMode openMode, Analyzer analyzer) { - return new EngineConfig(openMode, config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), + public EngineConfig copy(EngineConfig config, Analyzer analyzer) { + return new EngineConfig(config.getShardId(), config.getAllocationId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(), config.getStore(), config.getMergePolicy(), analyzer, config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getQueryCache(), config.getQueryCachingPolicy(), - config.getForceNewHistoryUUID(), config.getTranslogConfig(), config.getFlushMergesAfter(), + config.getTranslogConfig(), config.getFlushMergesAfter(), config.getExternalRefreshListener(), Collections.emptyList(), config.getIndexSort(), config.getTranslogRecoveryRunner(), config.getCircuitBreakerService(), config.getGlobalCheckpointSupplier()); } @@ -253,7 +248,8 @@ protected Translog createTranslog() throws IOException { protected Translog createTranslog(Path translogPath) throws IOException { TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE); - return new Translog(translogConfig, null, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + String translogUUID = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId); + return new Translog(translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED); } protected InternalEngine createEngine(Store store, Path translogPath) throws IOException { @@ -336,10 +332,23 @@ protected InternalEngine createEngine( @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(); + return createEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); + } + + protected InternalEngine createEngine(EngineConfig config) throws IOException { + return createEngine(null, null, null, config); + } + + private InternalEngine createEngine(@Nullable IndexWriterFactory indexWriterFactory, + @Nullable BiFunction localCheckpointTrackerSupplier, + @Nullable ToLongBiFunction seqNoForOperation, + EngineConfig config) throws IOException { + final Directory directory = config.getStore().directory(); + if (Lucene.indexExists(directory) == false) { + EngineDiskUtils.createEmpty(directory, config.getTranslogConfig().getTranslogPath(), config.getShardId()); } + InternalEngine internalEngine = createInternalEngine(indexWriterFactory, localCheckpointTrackerSupplier, seqNoForOperation, config); + internalEngine.recoverFromTranslog(); return internalEngine; } @@ -392,23 +401,13 @@ 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, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null, () -> SequenceNumbers.NO_OPS_PERFORMED); } public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, 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; - try { - if (Lucene.indexExists(store.directory()) == false) { - openMode = EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG; - } else { - openMode = EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG; - } - } catch (IOException e) { - throw new ElasticsearchException("can't find index?", e); - } Engine.EventListener listener = new Engine.EventListener() { @Override public void onFailedEngine(String reason, @Nullable Exception e) { @@ -419,14 +418,14 @@ public void onFailedEngine(String reason, @Nullable Exception e) { indexSettings.getSettings())); final List refreshListenerList = refreshListener == null ? emptyList() : Collections.singletonList(refreshListener); - EngineConfig config = new EngineConfig(openMode, shardId, allocationId.getId(), threadPool, indexSettings, null, store, + EngineConfig config = new EngineConfig(shardId, allocationId.getId(), threadPool, indexSettings, null, store, mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, - IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), false, translogConfig, + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), refreshListenerList, Collections.emptyList(), indexSort, handler, new NoneCircuitBreakerService(), globalCheckpointSupplier == null ? - new GlobalCheckpointTracker(shardId, allocationId.getId(), indexSettings, - SequenceNumbers.UNASSIGNED_SEQ_NO) : globalCheckpointSupplier); + new GlobalCheckpointTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED) : + globalCheckpointSupplier); return config; } From 5be23dc2a0db9ad29eaf8b765a32b9f145e5ef84 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Jan 2018 17:35:16 -0500 Subject: [PATCH 03/25] lint --- .../java/org/elasticsearch/index/engine/EngineTestCase.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 2c3e73d589631..3446167d11438 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 @@ -249,7 +249,8 @@ protected Translog createTranslog() throws IOException { protected Translog createTranslog(Path translogPath) throws IOException { TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, INDEX_SETTINGS, BigArrays.NON_RECYCLING_INSTANCE); String translogUUID = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId); - return new Translog(translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED); + return new Translog( + translogConfig, translogUUID, createTranslogDeletionPolicy(INDEX_SETTINGS), () -> SequenceNumbers.NO_OPS_PERFORMED); } protected InternalEngine createEngine(Store store, Path translogPath) throws IOException { From 88bd7380b1b786693fbf8a373cac9c8ba0e68a40 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 14 Jan 2018 17:42:57 -0500 Subject: [PATCH 04/25] fix index shard assertion --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ab1724e2c511b..dea7c2cb9e634 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1281,7 +1281,9 @@ int runTranslogRecovery(Engine engine, Translog.Snapshot snapshot) throws IOExce * Operations from the translog will be replayed to bring lucene up to date. **/ public void openEngineAndRecoverFromTranslog() throws IOException { - assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE; + assert + recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE || + recoveryState.getRecoverySource().getType() == RecoverySource.Type.EMPTY_STORE; innerOpenEngineAndTranslog(); getEngine().recoverFromTranslog(); } From 781621000868f0b4b44fff3078d6d109c5c7cb62 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 08:33:04 -0500 Subject: [PATCH 05/25] fix translog tests --- .../index/translog/Translog.java | 9 +- .../indices/recovery/RecoveryTarget.java | 2 +- .../index/translog/TranslogTests.java | 149 ++++++++++-------- 3 files changed, 93 insertions(+), 67 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/translog/Translog.java b/server/src/main/java/org/elasticsearch/index/translog/Translog.java index e7b7ee220db4e..1fa7b4a982c1f 100644 --- a/server/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/server/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -1705,14 +1705,19 @@ List getReaders() { public static String createEmptyTranslog(final Path location, final long initialGlobalCheckpoint, final ShardId shardId) throws IOException { + final ChannelFactory channelFactory = FileChannel::open; + return createEmptyTranslog(location, initialGlobalCheckpoint, shardId, channelFactory); + } + + static String createEmptyTranslog(Path location, long initialGlobalCheckpoint, ShardId shardId, ChannelFactory channelFactory) throws IOException { IOUtils.rm(location); Files.createDirectories(location); final Checkpoint checkpoint = Checkpoint.emptyTranslogCheckpoint(0, 1, initialGlobalCheckpoint, 1); final Path checkpointFile = location.resolve(CHECKPOINT_FILE_NAME); - Checkpoint.write(FileChannel::open, checkpointFile, checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); + Checkpoint.write(channelFactory, checkpointFile, checkpoint, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); IOUtils.fsync(checkpointFile, false); final String translogUUID = UUIDs.randomBase64UUID(); - TranslogWriter writer = TranslogWriter.create(shardId, translogUUID, 1, location.resolve(getFilename(1)), FileChannel::open, + TranslogWriter writer = TranslogWriter.create(shardId, translogUUID, 1, location.resolve(getFilename(1)), channelFactory, new ByteSizeValue(10), 1, initialGlobalCheckpoint, () -> { throw new UnsupportedOperationException(); }, () -> { throw new UnsupportedOperationException(); } ); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 2bc953a88a9fc..b64c420165d7e 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -442,7 +442,7 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) { EngineDiskUtils.verifyHasHistoryUUID(store.directory()); } - // TODO: Assigns the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 + // TODO: Assign the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 EngineDiskUtils.createNewTranslog(store.directory(), indexShard.shardPath().resolveTranslog(), SequenceNumbers.UNASSIGNED_SEQ_NO, shardId); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { diff --git a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index 890db4a795918..78e4d58c732da 100644 --- a/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/server/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -146,11 +146,19 @@ protected void afterIfSuccessful() throws Exception { } - protected Translog createTranslog(TranslogConfig config, String translogUUID) throws IOException { + protected Translog createTranslog(TranslogConfig config) throws IOException { + String translogUUID = + Translog.createEmptyTranslog(config.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); return new Translog(config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()), - () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + () -> SequenceNumbers.NO_OPS_PERFORMED); } + protected Translog openTranslog(TranslogConfig config, String translogUUID) throws IOException { + return new Translog(config, translogUUID, createTranslogDeletionPolicy(config.getIndexSettings()), + () -> SequenceNumbers.NO_OPS_PERFORMED); + } + + private void markCurrentGenAsCommitted(Translog translog) throws IOException { long genToCommit = translog.currentFileGeneration(); long genToRetain = randomLongBetween(translog.getDeletionPolicy().getMinTranslogGenerationForRecovery(), genToCommit); @@ -193,10 +201,11 @@ public void tearDown() throws Exception { } private Translog create(Path path) throws IOException { - globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final TranslogConfig translogConfig = getTranslogConfig(path); final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(translogConfig.getIndexSettings()); - return new Translog(translogConfig, null, deletionPolicy, () -> globalCheckpoint.get()); + final String translogUUID = Translog.createEmptyTranslog(path, SequenceNumbers.NO_OPS_PERFORMED, shardId); + return new Translog(translogConfig, translogUUID, deletionPolicy, () -> globalCheckpoint.get()); } private TranslogConfig getTranslogConfig(final Path path) { @@ -371,39 +380,39 @@ public void testStats() throws IOException { { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(1)); - assertThat(stats.getTranslogSizeInBytes(), equalTo(97L)); + assertThat(stats.getTranslogSizeInBytes(), equalTo(140L)); assertThat(stats.getUncommittedOperations(), equalTo(1)); - assertThat(stats.getUncommittedSizeInBytes(), equalTo(97L)); + assertThat(stats.getUncommittedSizeInBytes(), equalTo(140L)); } translog.add(new Translog.Delete("test", "2", 1, newUid("2"))); { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(2)); - assertThat(stats.getTranslogSizeInBytes(), equalTo(146L)); + assertThat(stats.getTranslogSizeInBytes(), equalTo(189L)); assertThat(stats.getUncommittedOperations(), equalTo(2)); - assertThat(stats.getUncommittedSizeInBytes(), equalTo(146L)); + assertThat(stats.getUncommittedSizeInBytes(), equalTo(189L)); } translog.add(new Translog.Delete("test", "3", 2, newUid("3"))); { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(3)); - assertThat(stats.getTranslogSizeInBytes(), equalTo(195L)); + assertThat(stats.getTranslogSizeInBytes(), equalTo(238L)); assertThat(stats.getUncommittedOperations(), equalTo(3)); - assertThat(stats.getUncommittedSizeInBytes(), equalTo(195L)); + assertThat(stats.getUncommittedSizeInBytes(), equalTo(238L)); } translog.add(new Translog.NoOp(3, 1, randomAlphaOfLength(16))); { final TranslogStats stats = stats(); assertThat(stats.estimatedNumberOfOperations(), equalTo(4)); - assertThat(stats.getTranslogSizeInBytes(), equalTo(237L)); + assertThat(stats.getTranslogSizeInBytes(), equalTo(280L)); assertThat(stats.getUncommittedOperations(), equalTo(4)); - assertThat(stats.getUncommittedSizeInBytes(), equalTo(237L)); + assertThat(stats.getUncommittedSizeInBytes(), equalTo(280L)); } - final long expectedSizeInBytes = 280L; + final long expectedSizeInBytes = 323L; translog.rollGeneration(); { final TranslogStats stats = stats(); @@ -1234,15 +1243,15 @@ public void testBasicRecovery() throws IOException { translog.close(); if (translogGeneration == null) { - translog = createTranslog(config, null); + translog = createTranslog(config); assertEquals(0, translog.stats().estimatedNumberOfOperations()); - assertEquals(1, translog.currentFileGeneration()); + assertEquals(2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); try(Translog.Snapshot snapshot = translog.newSnapshot()) { assertNull(snapshot.next()); } } else { - translog = new Translog(config, translogGeneration.translogUUID, translog.getDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + translog = new Translog(config, translogGeneration.translogUUID, translog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED); assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); try (Translog.Snapshot snapshot = translog.newSnapshotFromGen(translogGeneration.translogFileGeneration)) { @@ -1268,7 +1277,8 @@ public void testRecoveryUncommitted() throws IOException { if (op == prepareOp) { translogGeneration = translog.getGeneration(); translog.rollGeneration(); - assertEquals("expected this to be the first commit", 1L, translogGeneration.translogFileGeneration); + assertEquals("expected this to be the first roll (1 gen is on creation, 2 when opened)", + 2L, translogGeneration.translogFileGeneration); assertNotNull(translogGeneration.translogUUID); } } @@ -1280,7 +1290,7 @@ public void testRecoveryUncommitted() throws IOException { TranslogConfig config = translog.getConfig(); final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); @@ -1294,9 +1304,10 @@ public void testRecoveryUncommitted() throws IOException { } } if (randomBoolean()) { // recover twice - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertNotNull(translogGeneration); - assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); + assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", + translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; @@ -1322,7 +1333,8 @@ public void testRecoveryUncommittedFileExists() throws IOException { if (op == prepareOp) { translogGeneration = translog.getGeneration(); translog.rollGeneration(); - assertEquals("expected this to be the first commit", 1L, translogGeneration.translogFileGeneration); + assertEquals("expected this to be the first roll (1 gen is on creation, 2 when opened)", + 2L, translogGeneration.translogFileGeneration); assertNotNull(translogGeneration.translogUUID); } } @@ -1338,7 +1350,7 @@ public void testRecoveryUncommittedFileExists() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); @@ -1353,9 +1365,10 @@ public void testRecoveryUncommittedFileExists() throws IOException { } if (randomBoolean()) { // recover twice - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertNotNull(translogGeneration); - assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); + assertEquals("lastCommitted must be 3 less than current - we never finished the commit and run recovery twice", + translogGeneration.translogFileGeneration + 3, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); try (Translog.Snapshot snapshot = new SortedSnapshot(translog.newSnapshot())) { int upTo = sync ? translogOperations : prepareOp; @@ -1380,7 +1393,8 @@ public void testRecoveryUncommittedCorruptedCheckpoint() throws IOException { if (op == prepareOp) { translogGeneration = translog.getGeneration(); translog.rollGeneration(); - assertEquals("expected this to be the first commit", 1L, translogGeneration.translogFileGeneration); + assertEquals("expected this to be the first roll (1 gen is on creation, 2 when opened)", + 2L, translogGeneration.translogFileGeneration); assertNotNull(translogGeneration.translogUUID); } } @@ -1390,19 +1404,19 @@ public void testRecoveryUncommittedCorruptedCheckpoint() throws IOException { TranslogConfig config = translog.getConfig(); Path ckp = config.getTranslogPath().resolve(Translog.CHECKPOINT_FILE_NAME); Checkpoint read = Checkpoint.read(ckp); - Checkpoint corrupted = Checkpoint.emptyTranslogCheckpoint(0, 0, SequenceNumbers.UNASSIGNED_SEQ_NO, 0); + Checkpoint corrupted = Checkpoint.emptyTranslogCheckpoint(0, 0, SequenceNumbers.NO_OPS_PERFORMED, 0); Checkpoint.write(FileChannel::open, config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)), corrupted, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW); final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); - try (Translog ignored = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog ignored = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { fail("corrupted"); } catch (IllegalStateException ex) { - assertEquals("Checkpoint file translog-2.ckp already exists but has corrupted content expected: Checkpoint{offset=3123, " + - "numOps=55, generation=2, minSeqNo=45, maxSeqNo=99, globalCheckpoint=-2, minTranslogGeneration=1} but got: Checkpoint{offset=0, numOps=0, " + - "generation=0, minSeqNo=-1, maxSeqNo=-1, globalCheckpoint=-2, minTranslogGeneration=0}", ex.getMessage()); + assertEquals("Checkpoint file translog-3.ckp already exists but has corrupted content expected: Checkpoint{offset=3123, " + + "numOps=55, generation=3, minSeqNo=45, maxSeqNo=99, globalCheckpoint=-1, minTranslogGeneration=1} but got: Checkpoint{offset=0, numOps=0, " + + "generation=0, minSeqNo=-1, maxSeqNo=-1, globalCheckpoint=-1, minTranslogGeneration=0}", ex.getMessage()); } Checkpoint.write(FileChannel::open, config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation)), read, StandardOpenOption.WRITE, StandardOpenOption.TRUNCATE_EXISTING); - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertNotNull(translogGeneration); assertEquals("lastCommitted must be 2 less than current - we never finished the commit", translogGeneration.translogFileGeneration + 2, translog.currentFileGeneration()); assertFalse(translog.syncNeeded()); @@ -1479,12 +1493,12 @@ public void testOpenForeignTranslog() throws IOException { final String foreignTranslog = randomRealisticUnicodeOfCodepointLengthBetween(1, translogGeneration.translogUUID.length()); try { - new Translog(config, foreignTranslog, createTranslogDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + new Translog(config, foreignTranslog, createTranslogDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED); fail("translog doesn't belong to this UUID"); } catch (TranslogCorruptedException ex) { } - this.translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + this.translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); try (Translog.Snapshot snapshot = this.translog.newSnapshotFromGen(translogGeneration.translogFileGeneration)) { for (int i = firstUncommitted; i < translogOperations; i++) { Translog.Operation next = snapshot.next(); @@ -1670,7 +1684,7 @@ public void testFailFlush() throws IOException { translog.close(); // we are closed final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = translog.getDeletionPolicy(); - try (Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertEquals("lastCommitted must be 1 less than current", translogGeneration.translogFileGeneration + 1, tlog.currentFileGeneration()); assertFalse(tlog.syncNeeded()); @@ -1806,7 +1820,7 @@ protected void afterAdd() throws IOException { } } try (Translog tlog = - new Translog(config, translogUUID, createTranslogDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + new Translog(config, translogUUID, createTranslogDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED); Translog.Snapshot snapshot = tlog.newSnapshot()) { if (writtenOperations.size() != snapshot.totalOperations()) { for (int i = 0; i < threadCount; i++) { @@ -1852,7 +1866,7 @@ public void testRecoveryFromAFutureGenerationCleansUp() throws IOException { final TranslogDeletionPolicy deletionPolicy = new TranslogDeletionPolicy(-1, -1); deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration); - translog = new Translog(config, translog.getTranslogUUID(), deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + translog = new Translog(config, translog.getTranslogUUID(), deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); assertThat(translog.getMinFileGeneration(), equalTo(1L)); // no trimming done yet, just recovered for (long gen = 1; gen < translog.currentFileGeneration(); gen++) { @@ -1908,7 +1922,7 @@ public void testRecoveryFromFailureOnTrimming() throws IOException { final TranslogDeletionPolicy deletionPolicy = new TranslogDeletionPolicy(-1, -1); deletionPolicy.setTranslogGenerationOfLastCommit(randomLongBetween(comittedGeneration, Long.MAX_VALUE)); deletionPolicy.setMinTranslogGenerationForRecovery(comittedGeneration); - try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { // we don't know when things broke exactly assertThat(translog.getMinFileGeneration(), greaterThanOrEqualTo(1L)); assertThat(translog.getMinFileGeneration(), lessThanOrEqualTo(comittedGeneration)); @@ -1956,25 +1970,28 @@ public void onceFailedFailAlways() { private Translog getFailableTranslog(final FailSwitch fail, final TranslogConfig config, final boolean partialWrites, final boolean throwUnknownException, String translogUUID, final TranslogDeletionPolicy deletionPolicy) throws IOException { - return new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO) { + final ChannelFactory channelFactory = (file, openOption) -> { + FileChannel channel = FileChannel.open(file, openOption); + boolean success = false; + try { + final boolean isCkpFile = file.getFileName().toString().endsWith(".ckp"); // don't do partial writes for checkpoints we rely on the fact that the bytes are written as an atomic operation + ThrowingFileChannel throwingFileChannel = new ThrowingFileChannel(fail, isCkpFile ? false : partialWrites, throwUnknownException, channel); + success = true; + return throwingFileChannel; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(channel); + } + } + }; + if (translogUUID == null) { + translogUUID = Translog.createEmptyTranslog( + config.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId, channelFactory); + } + return new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED) { @Override ChannelFactory getChannelFactory() { - final ChannelFactory factory = super.getChannelFactory(); - - return (file, openOption) -> { - FileChannel channel = factory.open(file, openOption); - boolean success = false; - try { - final boolean isCkpFile = file.getFileName().toString().endsWith(".ckp"); // don't do partial writes for checkpoints we rely on the fact that the bytes are written as an atomic operation - ThrowingFileChannel throwingFileChannel = new ThrowingFileChannel(fail, isCkpFile ? false : partialWrites, throwUnknownException, channel); - success = true; - return throwingFileChannel; - } finally { - if (success == false) { - IOUtils.closeWhileHandlingException(channel); - } - } - }; + return channelFactory; } @Override @@ -2078,11 +2095,11 @@ private static final class UnknownException extends RuntimeException { public void testFailWhileCreateWriteWithRecoveredTLogs() throws IOException { Path tempDir = createTempDir(); TranslogConfig config = getTranslogConfig(tempDir); - Translog translog = createTranslog(config, null); + Translog translog = createTranslog(config); translog.add(new Translog.Index("test", "boom", 0, "boom".getBytes(Charset.forName("UTF-8")))); translog.close(); try { - new Translog(config, translog.getTranslogUUID(), createTranslogDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO) { + new Translog(config, translog.getTranslogUUID(), createTranslogDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED) { @Override protected TranslogWriter createWriter(long fileGeneration, long initialMinTranslogGen, long initialGlobalCheckpoint) throws IOException { @@ -2105,7 +2122,7 @@ public void testRecoverWithUnbackedNextGen() throws IOException { Checkpoint read = Checkpoint.read(ckp); Files.copy(ckp, config.getTranslogPath().resolve(Translog.getCommitCheckpointFileName(read.generation))); Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 1) + ".tlog")); - try (Translog tlog = createTranslog(config, translog.getTranslogUUID()); + try (Translog tlog = openTranslog(config, translog.getTranslogUUID()); Translog.Snapshot snapshot = tlog.newSnapshot()) { assertFalse(tlog.syncNeeded()); @@ -2116,7 +2133,7 @@ public void testRecoverWithUnbackedNextGen() throws IOException { tlog.add(new Translog.Index("test", "" + 1, 1, Integer.toString(2).getBytes(Charset.forName("UTF-8")))); } - try (Translog tlog = createTranslog(config, translog.getTranslogUUID()); + try (Translog tlog = openTranslog(config, translog.getTranslogUUID()); Translog.Snapshot snapshot = tlog.newSnapshot()) { assertFalse(tlog.syncNeeded()); @@ -2140,7 +2157,7 @@ public void testRecoverWithUnbackedNextGenInIllegalState() throws IOException { Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 1) + ".tlog")); try { - Translog tlog = new Translog(config, translog.getTranslogUUID(), translog.getDeletionPolicy(), () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + Translog tlog = new Translog(config, translog.getTranslogUUID(), translog.getDeletionPolicy(), () -> SequenceNumbers.NO_OPS_PERFORMED); fail("file already exists?"); } catch (TranslogException ex) { // all is well @@ -2162,7 +2179,7 @@ public void testRecoverWithUnbackedNextGenAndFutureFile() throws IOException { Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 1) + ".tlog")); // we add N+1 and N+2 to ensure we only delete the N+1 file and never jump ahead and wipe without the right condition Files.createFile(config.getTranslogPath().resolve("translog-" + (read.generation + 2) + ".tlog")); - try (Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO)) { + try (Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED)) { assertFalse(tlog.syncNeeded()); try (Translog.Snapshot snapshot = tlog.newSnapshot()) { for (int i = 0; i < 1; i++) { @@ -2175,7 +2192,7 @@ public void testRecoverWithUnbackedNextGenAndFutureFile() throws IOException { } try { - Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + Translog tlog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); fail("file already exists?"); } catch (TranslogException ex) { // all is well @@ -2281,7 +2298,11 @@ public void testWithRandomException() throws IOException { TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(); deletionPolicy.setTranslogGenerationOfLastCommit(minGenForRecovery); deletionPolicy.setMinTranslogGenerationForRecovery(minGenForRecovery); - try (Translog translog = new Translog(config, generationUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + if (generationUUID == null) { + // we never managed to successfully create a translog, make it + generationUUID = Translog.createEmptyTranslog(config.getTranslogPath(), SequenceNumbers.NO_OPS_PERFORMED, shardId); + } + try (Translog translog = new Translog(config, generationUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); Translog.Snapshot snapshot = translog.newSnapshotFromGen(minGenForRecovery)) { assertEquals(syncedDocs.size(), snapshot.totalOperations()); for (int i = 0; i < syncedDocs.size(); i++) { @@ -2346,14 +2367,14 @@ public void testPendingDelete() throws IOException { final String translogUUID = translog.getTranslogUUID(); final TranslogDeletionPolicy deletionPolicy = createTranslogDeletionPolicy(config.getIndexSettings()); translog.close(); - translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); translog.add(new Translog.Index("test", "2", 1, new byte[]{2})); translog.rollGeneration(); Closeable lock = translog.acquireRetentionLock(); translog.add(new Translog.Index("test", "3", 2, new byte[]{3})); translog.close(); IOUtils.close(lock); - translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + translog = new Translog(config, translogUUID, deletionPolicy, () -> SequenceNumbers.NO_OPS_PERFORMED); } public static Translog.Location randomTranslogLocation() { From e13f0f278e6308c49dd3bf3e160eccc36f272ec3 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 08:51:20 -0500 Subject: [PATCH 06/25] fix recovery tests --- .../indices/recovery/RecoveryTests.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java index 2089c36d06bc0..d0c26f10f0de9 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoveryTests.java @@ -31,7 +31,6 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.VersionType; @@ -43,7 +42,6 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.SnapshotMatchers; import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.index.translog.TranslogConfig; import java.util.HashMap; import java.util.List; @@ -51,7 +49,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; -import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -203,13 +200,8 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { final String historyUUIDtoUse = UUIDs.randomBase64UUID(random()); if (randomBoolean()) { // create a new translog - final TranslogConfig translogConfig = - new TranslogConfig(replica.shardId(), replica.shardPath().resolveTranslog(), replica.indexSettings(), - BigArrays.NON_RECYCLING_INSTANCE); - try (Translog translog = new Translog(translogConfig, null, createTranslogDeletionPolicy(), () -> flushedDocs)) { - translogUUIDtoUse = translog.getTranslogUUID(); - translogGenToUse = translog.currentFileGeneration(); - } + translogUUIDtoUse = Translog.createEmptyTranslog(replica.shardPath().resolveTranslog(), flushedDocs, replica.shardId()); + translogGenToUse = 1; } else { translogUUIDtoUse = translogGeneration.translogUUID; translogGenToUse = translogGeneration.translogFileGeneration; From 963e86e265dea8d5efa75cc8689dff2e6acf6f07 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 09:06:37 -0500 Subject: [PATCH 07/25] remove assertion openEngineAndRecoverFromTranslog is now used by all recovery types. --- .../main/java/org/elasticsearch/index/shard/IndexShard.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index dea7c2cb9e634..571884454e6af 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1281,9 +1281,6 @@ int runTranslogRecovery(Engine engine, Translog.Snapshot snapshot) throws IOExce * Operations from the translog will be replayed to bring lucene up to date. **/ public void openEngineAndRecoverFromTranslog() throws IOException { - assert - recoveryState.getRecoverySource().getType() == RecoverySource.Type.EXISTING_STORE || - recoveryState.getRecoverySource().getType() == RecoverySource.Type.EMPTY_STORE; innerOpenEngineAndTranslog(); getEngine().recoverFromTranslog(); } From aba67a411adf22b6fecd1cd6a7a8027cb0fdc216 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 10:30:20 -0500 Subject: [PATCH 08/25] fix RefreshListenersTests --- .../index/shard/RefreshListenersTests.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index d42479a57389d..8de046ab85a03 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.EngineDiskUtils; import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; import org.elasticsearch.index.mapper.IdFieldMapper; @@ -120,13 +121,14 @@ public void onFailedEngine(String reason, @Nullable Exception e) { // we don't need to notify anybody in this test } }; + EngineDiskUtils.createEmpty(store.directory(), translogConfig.getTranslogPath(), shardId); EngineConfig config = new EngineConfig(shardId, allocationId, threadPool, - indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), - eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, - TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, null, - new NoneCircuitBreakerService(), - () -> SequenceNumbers.UNASSIGNED_SEQ_NO); + indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), + eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, + TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, + (e, s) -> 0, new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); engine = new InternalEngine(config); + engine.recoverFromTranslog(); listeners.setTranslog(engine.getTranslog()); } From 8d1fe7970c14c7e4603cac740a68aa30d10abfc1 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 10:32:42 -0500 Subject: [PATCH 09/25] fix testShardActiveDuringPeerRecovery --- .../org/elasticsearch/index/shard/IndexShardTests.java | 8 -------- 1 file changed, 8 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index f2355c388db8e..e043ee35c1c35 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2134,14 +2134,6 @@ public void testShardActiveDuringPeerRecovery() throws IOException { recoverReplica(replica, primary, (shard, discoveryNode) -> new RecoveryTarget(shard, discoveryNode, recoveryListener, aLong -> { }) { - @Override - public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) throws IOException { - super.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps); - // Shard is still inactive since we haven't started recovering yet - assertFalse(replica.isActive()); - - } - @Override public long indexTranslogOperations(List operations, int totalTranslogOps) throws IOException { final long localCheckpoint = super.indexTranslogOperations(operations, totalTranslogOps); From 4bc7f9bfa1a687f1ce55ea4a41c7ab5ecc2084e8 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 10:53:22 -0500 Subject: [PATCH 10/25] fix testStressMaybeFlushOrRollTranslogGeneration --- .../elasticsearch/index/shard/IndexShardIT.java | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index e02b6c04a89d3..ff92e4cb9c7ed 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -27,7 +27,6 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.index.IndexRequest; -import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -73,7 +72,6 @@ import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.InternalSettingsPlugin; -import org.elasticsearch.test.junit.annotations.TestLogging; import java.io.IOException; import java.io.UncheckedIOException; @@ -103,8 +101,8 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; -import static org.hamcrest.Matchers.containsString; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -409,15 +407,15 @@ public void testStressMaybeFlushOrRollTranslogGeneration() throws Exception { IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); assertFalse(shard.shouldFlush()); - final String key; final boolean flush = randomBoolean(); + final Settings settings; if (flush) { - key = "index.translog.flush_threshold_size"; + // size of the operation plus two generations of overhead. + settings = Settings.builder().put("index.translog.flush_threshold_size", "180b").build(); } else { - key = "index.translog.generation_threshold_size"; + // size of the operation plus header and footer + settings = Settings.builder().put("index.translog.generation_threshold_size", "117b").build(); } - // size of the operation plus header and footer - final Settings settings = Settings.builder().put(key, "117b").build(); client().admin().indices().prepareUpdateSettings("test").setSettings(settings).get(); client().prepareIndex("test", "test", "0") .setSource("{}", XContentType.JSON) From d6d3f6c74a3312697725a6c0614958481665908a Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 10:54:55 -0500 Subject: [PATCH 11/25] fix testMaybeFlush --- .../test/java/org/elasticsearch/index/shard/IndexShardIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index ff92e4cb9c7ed..873fb14ef0a62 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -333,7 +333,7 @@ public void testMaybeFlush() throws Exception { assertFalse(shard.shouldFlush()); client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() .put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), - new ByteSizeValue(117 /* size of the operation + header&footer*/, ByteSizeUnit.BYTES)).build()).get(); + new ByteSizeValue(160 /* size of the operation + two generations header&footer*/, ByteSizeUnit.BYTES)).build()).get(); client().prepareIndex("test", "test", "0") .setSource("{}", XContentType.JSON).setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); assertFalse(shard.shouldFlush()); From 0bbd50bcf96a09ade139dac8b0e1c59d2ec0578b Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 11:47:39 -0500 Subject: [PATCH 12/25] properly acquire store reference in RecoveryTarget#cleanFiles --- .../org/elasticsearch/indices/recovery/RecoveryTarget.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index b64c420165d7e..52ffe75dd561c 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -437,6 +437,7 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa // to recover from in case of a full cluster shutdown just when this code executes... renameAllTempFiles(); final Store store = store(); + store.incRef(); try { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) { @@ -468,6 +469,8 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa RecoveryFailedException rfe = new RecoveryFailedException(state(), "failed to clean after recovery", ex); fail(rfe, true); throw rfe; + } finally { + store.decRef(); } } From 152aca35de461f929a4c116fb6e7c79d15b0b27c Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 12:16:27 -0500 Subject: [PATCH 13/25] fix translog retention rest tests --- .../test/indices.stats/20_translog.yml | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml index f5a9469f357fb..364b398565e51 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml @@ -15,7 +15,7 @@ setup: - do: indices.stats: metric: [ translog ] - - set: { indices.test.primaries.translog.size_in_bytes: empty_size } + - set: { indices.test.primaries.translog.size_in_bytes: creation_size } - do: index: @@ -27,9 +27,9 @@ setup: - do: indices.stats: metric: [ translog ] - - gt: { indices.test.primaries.translog.size_in_bytes: $empty_size } + - gt: { indices.test.primaries.translog.size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.operations: 1 } - - gt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $empty_size } + - gt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.uncommitted_operations: 1 } - do: @@ -39,9 +39,10 @@ setup: - do: indices.stats: metric: [ translog ] - - gt: { indices.test.primaries.translog.size_in_bytes: $empty_size } + - gt: { indices.test.primaries.translog.size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.operations: 1 } - - match: { indices.test.primaries.translog.uncommitted_size_in_bytes: $empty_size } + ## creation translog size has some overhead due to an initial empty generation that will be trimmed later + - lt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.uncommitted_operations: 0 } - do: @@ -59,7 +60,8 @@ setup: - do: indices.stats: metric: [ translog ] - - match: { indices.test.primaries.translog.size_in_bytes: $empty_size } + ## creation translog size has some overhead due to an initial empty generation that will be trimmed later + - lte: { indices.test.primaries.translog.size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.operations: 0 } - - match: { indices.test.primaries.translog.uncommitted_size_in_bytes: $empty_size } + - lte: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.uncommitted_operations: 0 } From 2c36b889564dcd01165925482cf45857b88cbab2 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Mon, 15 Jan 2018 12:18:09 -0800 Subject: [PATCH 14/25] fix flush docs --- docs/reference/indices/flush.asciidoc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index 0c75fd011b418..91fac0908ef7f 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -93,12 +93,12 @@ which returns something similar to: { "commit" : { "id" : "3M3zkw2GHMo2Y4h4/KFKCg==", - "generation" : 2, + "generation" : 3, "user_data" : { "translog_uuid" : "hnOG3xFcTDeoI_kvvvOdNA", "history_uuid" : "XP7KDJGiS1a2fHYiFL5TXQ", "local_checkpoint" : "-1", - "translog_generation" : "1", + "translog_generation" : "3", "max_seq_no" : "-1", "sync_id" : "AVvFY-071siAOuFGEO9P", <1> "max_unsafe_auto_id_timestamp" : "-1" From 5b9b8cb3f6b050490852299bbe9a182c795e2fc0 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sat, 20 Jan 2018 20:26:20 +0100 Subject: [PATCH 15/25] fix static method and compilation --- .../index/engine/InternalEngine.java | 3 +- .../recovery/PeerRecoveryTargetService.java | 2 +- .../index/engine/InternalEngineTests.java | 36 +++---------------- 3 files changed, 7 insertions(+), 34 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 0ba2584d27422..bf8219bb7f83c 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -69,6 +69,7 @@ import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogCorruptedException; @@ -229,7 +230,7 @@ private LocalCheckpointTracker createLocalCheckpointTracker( BiFunction localCheckpointTrackerSupplier, IndexCommit startingCommit) throws IOException { final long maxSeqNo; final long localCheckpoint; - final SequenceNumbers.CommitInfo seqNoStats = store.loadSeqNoInfo(startingCommit); + final SequenceNumbers.CommitInfo seqNoStats = Store.loadSeqNoInfo(startingCommit); maxSeqNo = seqNoStats.maxSeqNo; localCheckpoint = seqNoStats.localCheckpoint; logger.trace("recovered maximum sequence number [{}] and local checkpoint [{}]", maxSeqNo, localCheckpoint); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 553a7e7842f00..f2f1a60d8eff1 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -359,7 +359,7 @@ public static long getStartingSeqNo(final RecoveryTarget recoveryTarget) { final long globalCheckpoint = Translog.readGlobalCheckpoint(recoveryTarget.translogLocation()); final List existingCommits = DirectoryReader.listCommits(recoveryTarget.store().directory()); final IndexCommit safeCommit = CombinedDeletionPolicy.findSafeCommitPoint(existingCommits, globalCheckpoint); - final SequenceNumbers.CommitInfo seqNoStats = recoveryTarget.store().loadSeqNoInfo(safeCommit); + final SequenceNumbers.CommitInfo seqNoStats = Store.loadSeqNoInfo(safeCommit); if (seqNoStats.maxSeqNo <= globalCheckpoint) { assert seqNoStats.localCheckpoint <= globalCheckpoint; /* diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index b4fb743b9644a..d423aaa0e6ac7 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -78,8 +78,8 @@ import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; -import org.elasticsearch.common.logging.ServerLoggers; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.logging.ServerLoggers; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -4324,9 +4324,9 @@ public void testAcquireIndexCommit() throws Exception { public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception { IOUtils.close(engine); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); - final EngineConfig config = copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, globalCheckpoint::get); + final EngineConfig config = copy(engine.config(), globalCheckpoint::get); final IndexCommit safeCommit; - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) { + try (InternalEngine engine = new InternalEngine(config)) { final int numDocs = between(5, 50); for (int i = 0; i < numDocs; i++) { index(engine, i); @@ -4340,40 +4340,12 @@ public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception { globalCheckpoint.set(Long.parseLong(safeCommit.getUserData().get(SequenceNumbers.MAX_SEQ_NO))); engine.getTranslog().sync(); } - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { + try (InternalEngine engine = new InternalEngine(config)) { final List existingCommits = DirectoryReader.listCommits(engine.store.directory()); assertThat("OPEN_INDEX_AND_TRANSLOG should keep only safe commit", existingCommits, contains(safeCommit)); } } - public void testOpenIndexCreateTranslogKeepOnlyLastCommit() throws Exception { - IOUtils.close(engine); - final EngineConfig config = copy(engine.config(), EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); - final Map lastCommit; - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG))) { - engine.skipTranslogRecovery(); - final int numDocs = between(5, 50); - for (int i = 0; i < numDocs; i++) { - index(engine, i); - if (randomBoolean()) { - engine.flush(); - } - } - final List commits = DirectoryReader.listCommits(engine.store.directory()); - lastCommit = commits.get(commits.size() - 1).getUserData(); - } - try (InternalEngine engine = new InternalEngine(copy(config, EngineConfig.OpenMode.OPEN_INDEX_CREATE_TRANSLOG))) { - final List existingCommits = DirectoryReader.listCommits(engine.store.directory()); - assertThat("OPEN_INDEX_CREATE_TRANSLOG should keep only last commit", existingCommits, hasSize(1)); - final Map userData = existingCommits.get(0).getUserData(); - assertThat(userData.get(SequenceNumbers.MAX_SEQ_NO), equalTo(lastCommit.get(SequenceNumbers.MAX_SEQ_NO))); - assertThat(userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY), equalTo(lastCommit.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY))); - // Translog tags should be fresh. - assertThat(userData.get(Translog.TRANSLOG_UUID_KEY), not(equalTo(lastCommit.get(Translog.TRANSLOG_UUID_KEY)))); - assertThat(userData.get(Translog.TRANSLOG_GENERATION_KEY), equalTo("1")); - } - } - public void testCleanUpCommitsWhenGlobalCheckpointAdvanced() throws Exception { IOUtils.close(engine, store); store = createStore(); From ac96edebd9d7265a61a18d1642fa370410612cd2 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 21 Jan 2018 13:30:32 +0100 Subject: [PATCH 16/25] missing sync --- .../org/elasticsearch/index/engine/CombinedDeletionPolicy.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java index 8fcc7d588ebc0..dc05225b58be4 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/CombinedDeletionPolicy.java @@ -58,7 +58,7 @@ public final class CombinedDeletionPolicy extends IndexDeletionPolicy { } @Override - public void onInit(List commits) throws IOException { + public synchronized void onInit(List commits) throws IOException { assert commits.isEmpty() == false : "index is opened, but we have no commits"; assert startingCommit != null && commits.contains(startingCommit) : "Starting commit not in the existing commit list; " + "startingCommit [" + startingCommit + "], commit list [" + commits + "]"; From 3b11801b19bd2dafafcce71e752478da65235199 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 21 Jan 2018 17:21:32 +0100 Subject: [PATCH 17/25] fix InternalEngineTests --- .../index/engine/EngineDiskUtils.java | 2 +- .../index/engine/InternalEngineTests.java | 17 +++++++++-------- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java index 5aa8fef15bdeb..7ba8de093d544 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java @@ -79,7 +79,7 @@ public static void createNewTranslog(final Directory dir, final Path translogPat throws IOException { if (Assertions.ENABLED) { final List existingCommits = DirectoryReader.listCommits(dir); - assert existingCommits.size() == 1 : "Open index create translog should have one commit, commits[" + existingCommits + "]"; + assert existingCommits.size() == 1 : "creating a translog translog should have one commit, commits[" + existingCommits + "]"; SequenceNumbers.CommitInfo commitInfo = Store.loadSeqNoInfo(existingCommits.get(0)); assert commitInfo.localCheckpoint >= initialGlobalCheckpoint : "trying to create a shard whose local checkpoint [" + commitInfo.localCheckpoint + "] is < global checkpoint [" diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index d423aaa0e6ac7..698e03f05e043 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -3362,21 +3362,21 @@ public void testRetryConcurrently() throws InterruptedException, IOException { public void testEngineMaxTimestampIsInitialized() throws IOException { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final long timestamp1 = Math.abs(randomNonNegativeLong()); final Path storeDir = createTempDir(); final Path translogDir = createTempDir(); final long timestamp2 = randomNonNegativeLong(); final long maxTimestamp12 = Math.max(timestamp1, timestamp2); - try (Store store = createStore(newFSDirectory(storeDir)); - Engine engine = createEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { + final EngineConfig config = config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); + try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = createEngine(config)) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); engine.index(appendOnlyPrimary(doc, true, timestamp1)); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); } - try (Store store = createStore(newFSDirectory(storeDir)); - Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { + try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = new InternalEngine(config)) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); engine.recoverFromTranslog(); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); @@ -3384,10 +3384,11 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { new BytesArray("{}".getBytes(Charset.defaultCharset())), null); engine.index(appendOnlyPrimary(doc, true, timestamp2)); assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); + globalCheckpoint.set(1); // make sure flush cleans up commits for later. engine.flush(); } try (Store store = createStore(newFSDirectory(storeDir))) { - if (randomBoolean()) { + if (randomBoolean() || true) { EngineDiskUtils.createNewTranslog(store.directory(), translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId); } try (Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { @@ -4323,7 +4324,7 @@ public void testAcquireIndexCommit() throws Exception { public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception { IOUtils.close(engine); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final EngineConfig config = copy(engine.config(), globalCheckpoint::get); final IndexCommit safeCommit; try (InternalEngine engine = new InternalEngine(config)) { @@ -4342,14 +4343,14 @@ public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception { } try (InternalEngine engine = new InternalEngine(config)) { final List existingCommits = DirectoryReader.listCommits(engine.store.directory()); - assertThat("OPEN_INDEX_AND_TRANSLOG should keep only safe commit", existingCommits, contains(safeCommit)); + assertThat("safe commit should be kept", existingCommits, contains(safeCommit)); } } public void testCleanUpCommitsWhenGlobalCheckpointAdvanced() throws Exception { IOUtils.close(engine, store); store = createStore(); - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.UNASSIGNED_SEQ_NO); + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); try (InternalEngine engine = createEngine(store, createTempDir(), globalCheckpoint::get)) { final int numDocs = scaledRandomIntBetween(10, 100); for (int docId = 0; docId < numDocs; docId++) { From aad5774b3fd060e3d0d02cb3406595299c0db7f9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 7 Feb 2018 10:33:45 +0100 Subject: [PATCH 18/25] fix tests --- .../index/engine/InternalEngineTests.java | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index bd2f1cf7a486e..e1009468c3b85 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -150,6 +150,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; @@ -3372,15 +3373,16 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { final Path translogDir = createTempDir(); final long timestamp2 = randomNonNegativeLong(); final long maxTimestamp12 = Math.max(timestamp1, timestamp2); - final EngineConfig config = config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); - try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = createEngine(config)) { + final Function configSupplier = + store -> config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null, null, globalCheckpoint::get); + try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = createEngine(configSupplier.apply(store))) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); final ParsedDocument doc = testParsedDocument("1", null, testDocumentWithTextField(), new BytesArray("{}".getBytes(Charset.defaultCharset())), null); engine.index(appendOnlyPrimary(doc, true, timestamp1)); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); } - try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = new InternalEngine(config)) { + try (Store store = createStore(newFSDirectory(storeDir)); Engine engine = new InternalEngine(configSupplier.apply(store))) { assertEquals(IndexRequest.UNSET_AUTO_GENERATED_TIMESTAMP, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); engine.recoverFromTranslog(); assertEquals(timestamp1, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); @@ -3395,7 +3397,7 @@ public void testEngineMaxTimestampIsInitialized() throws IOException { if (randomBoolean() || true) { EngineDiskUtils.createNewTranslog(store.directory(), translogDir, SequenceNumbers.NO_OPS_PERFORMED, shardId); } - try (Engine engine = new InternalEngine(config(defaultSettings, store, translogDir, NoMergePolicy.INSTANCE, null))) { + try (Engine engine = new InternalEngine(configSupplier.apply(store))) { assertEquals(maxTimestamp12, engine.segmentsStats(false).getMaxUnsafeAutoIdTimestamp()); } } @@ -4337,7 +4339,7 @@ public void testOpenIndexAndTranslogKeepOnlySafeCommit() throws Exception { final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); final EngineConfig config = copy(engine.config(), globalCheckpoint::get); final IndexCommit safeCommit; - try (InternalEngine engine = new InternalEngine(config)) { + try (InternalEngine engine = createEngine(config)) { final int numDocs = between(5, 50); for (int i = 0; i < numDocs; i++) { index(engine, i); From 7086f922e59a9e0470b67af69ff26f2c121df1b5 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 7 Feb 2018 10:36:06 +0100 Subject: [PATCH 19/25] remove invalid assertion that was merged --- .../java/org/elasticsearch/index/shard/IndexShard.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 6047e9f01f67f..8bd10c6cd5c4a 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -1285,13 +1285,6 @@ public void openEngineAndRecoverFromTranslog() throws IOException { * The translog is kept but its operations won't be replayed. */ public void openEngineAndSkipTranslogRecovery() throws IOException { - if (Assertions.ENABLED) { - // This assertion is only guaranteed if all nodes are on 6.2+. - if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_6_2_0)) { - final List existingCommits = DirectoryReader.listCommits(store.directory()); - assert existingCommits.size() == 1 : "Open index create translog should have one commit, commits[" + existingCommits + "]"; - } - } innerOpenEngineAndTranslog(); getEngine().skipTranslogRecovery(); } From d5f5a0ca756c09f2c8cd5ef00868d4fc8cfc8d4d Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 4 Mar 2018 07:51:52 -0800 Subject: [PATCH 20/25] tweak translog recovery --- .../java/org/elasticsearch/index/engine/InternalEngine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 75848e063d15f..cc4390f450cee 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -431,8 +431,8 @@ private void recoverFromTranslogInternal() throws IOException { commitIndexWriter(indexWriter, translog, null); refreshLastCommittedSegmentInfos(); refresh("translog_recovery"); - translog.trimUnreferencedReaders(); } + translog.trimUnreferencedReaders(); } private Translog openTranslog(EngineConfig engineConfig, TranslogDeletionPolicy translogDeletionPolicy, LongSupplier globalCheckpointSupplier) throws IOException { From 5a4e957bc40e178c79111aeb62d1473c47810243 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 7 Mar 2018 22:26:40 +0100 Subject: [PATCH 21/25] roll back changes to 20_translog.yml test --- .../rest-api-spec/test/indices.stats/20_translog.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml index 6c008974d689d..5c9ec3e597ad9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.stats/20_translog.yml @@ -29,7 +29,9 @@ setup: metric: [ translog ] - gt: { indices.test.primaries.translog.size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.operations: 1 } - - gt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size } +# we can't check this yet as creation size will contain two empty translog generations. A single +# non empty generation with one op may be smaller or larger than that. +# - gt: { indices.test.primaries.translog.uncommitted_size_in_bytes: $creation_size } - match: { indices.test.primaries.translog.uncommitted_operations: 1 } - do: From b8ecd0ebbb82cd50833c710358d1fa635b05ba1d Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 11 Mar 2018 16:38:03 +0100 Subject: [PATCH 22/25] java docs and dedicated uni tests --- .../index/engine/EngineDiskUtils.java | 15 +- .../indices/recovery/RecoveryTarget.java | 2 +- .../index/engine/EngineDiskUtilsTests.java | 186 ++++++++++++++++++ .../index/engine/InternalEngineTests.java | 168 ---------------- .../index/engine/EngineTestCase.java | 18 ++ 5 files changed, 219 insertions(+), 170 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java index 7ba8de093d544..0bc4ae5199acc 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java @@ -45,6 +45,9 @@ public final class EngineDiskUtils { private EngineDiskUtils() { } + /** + * creates an empty lucene index and a corresponding empty translog. Any existing data will be deleted. + */ public static void createEmpty(final Directory dir, final Path translogPath, final ShardId shardId) throws IOException { try (IndexWriter writer = newIndexWriter(true, dir)) { final String translogUuid = Translog.createEmptyTranslog(translogPath, SequenceNumbers.NO_OPS_PERFORMED, shardId); @@ -60,6 +63,10 @@ public static void createEmpty(final Directory dir, final Path translogPath, fin } + /** + * Converts an existing lucene index and marks it with a new history uuid. Also creates a new empty translog file. + * This is used to make sure no existing shard will recovery from this index using ops based recovery. + */ public static void bootstrapNewHistoryFromLuceneIndex(final Directory dir, final Path translogPath, final ShardId shardId) throws IOException { try (IndexWriter writer = newIndexWriter(false, dir)) { @@ -75,6 +82,9 @@ public static void bootstrapNewHistoryFromLuceneIndex(final Directory dir, final } } + /** + * Creates a new empty translog and associates it with an existing lucene index. + */ public static void createNewTranslog(final Directory dir, final Path translogPath, long initialGlobalCheckpoint, final ShardId shardId) throws IOException { if (Assertions.ENABLED) { @@ -96,7 +106,10 @@ public static void createNewTranslog(final Directory dir, final Path translogPat } - public static void verifyHasHistoryUUID(final Directory dir) throws IOException { + /** + * Checks that the Lucene index contains a history uuid marker. If not, a new one is generated and committed. + */ + public static void ensureIndexHasHistoryUUID(final Directory dir) throws IOException { try (IndexWriter writer = newIndexWriter(false, dir)) { final Map userData = getUserData(writer); if (userData.containsKey(Engine.HISTORY_UUID_KEY) == false) { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index d2ec92b48847a..1b1a2802b52bd 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -441,7 +441,7 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa try { store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); if (indexShard.indexSettings().getIndexVersionCreated().before(Version.V_6_0_0_rc1)) { - EngineDiskUtils.verifyHasHistoryUUID(store.directory()); + EngineDiskUtils.ensureIndexHasHistoryUUID(store.directory()); } // TODO: Assign the global checkpoint to the max_seqno of the safe commit if the index version >= 6.2 EngineDiskUtils.createNewTranslog(store.directory(), indexShard.shardPath().resolveTranslog(), diff --git a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java new file mode 100644 index 0000000000000..2e65c03b99358 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java @@ -0,0 +1,186 @@ +package org.elasticsearch.index.engine; + +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.search.IndexSearcher; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; +import org.elasticsearch.test.IndexSettingsModule; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; + +public class EngineDiskUtilsTests extends EngineTestCase { + + + public void testHistoryUUIDIsSetIfMissing() throws IOException { + final int numDocs = randomIntBetween(0, 3); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.IndexResult index = engine.index(firstIndexRequest); + assertThat(index.getVersion(), equalTo(1L)); + } + assertVisibleCount(engine, numDocs); + engine.close(); + + IndexWriterConfig iwc = new IndexWriterConfig(null) + .setCommitOnClose(false) + // we don't want merges to happen here - we call maybe merge on the engine + // later once we stared it up otherwise we would need to wait for it here + // we also don't specify a codec here and merges should use the engines for this index + .setMergePolicy(NoMergePolicy.INSTANCE) + .setOpenMode(IndexWriterConfig.OpenMode.APPEND); + try (IndexWriter writer = new IndexWriter(store.directory(), iwc)) { + Map newCommitData = new HashMap<>(); + for (Map.Entry entry : writer.getLiveCommitData()) { + if (entry.getKey().equals(Engine.HISTORY_UUID_KEY) == false) { + newCommitData.put(entry.getKey(), entry.getValue()); + } + } + writer.setLiveCommitData(newCommitData.entrySet()); + writer.commit(); + } + + EngineDiskUtils.ensureIndexHasHistoryUUID(store.directory()); + + final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() + .put(defaultSettings.getSettings()) + .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0_beta1) + .build()); + + EngineConfig config = engine.config(); + EngineConfig newConfig = new EngineConfig( + shardId, allocationId.getId(), + threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), + new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), + config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), + new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); + engine = new InternalEngine(newConfig); + engine.recoverFromTranslog(); + assertVisibleCount(engine, numDocs, false); + assertThat(engine.getHistoryUUID(), notNullValue()); + } + + public void testCurrentTranslogIDisCommitted() throws IOException { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + try (Store store = createStore()) { + EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); + + // create + { + EngineDiskUtils.createEmpty(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); + ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + + try (InternalEngine engine = createEngine(config)) { + engine.index(firstIndexRequest); + globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); + expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + // open and recover tlog + { + for (int i = 0; i < 2; i++) { + try (InternalEngine engine = new InternalEngine(config)) { + assertTrue(engine.isRecovering()); + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + if (i == 0) { + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + } else { + // creating an empty index will create the first translog gen and commit it + // opening the empty index will make the second translog file but not commit it + // opening the engine again (i=0) will make the third translog file, which then be committed + assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + } + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + } + // open index with new tlog + { + EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), + SequenceNumbers.NO_OPS_PERFORMED, shardId); + try (InternalEngine engine = new InternalEngine(config)) { + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + assertEquals(2, engine.getTranslog().currentFileGeneration()); + assertEquals(0L, engine.getTranslog().uncommittedOperations()); + } + } + + // open and recover tlog with empty tlog + { + for (int i = 0; i < 2; i++) { + try (InternalEngine engine = new InternalEngine(config)) { + Map userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + engine.recoverFromTranslog(); + userData = engine.getLastCommittedSegmentInfos().getUserData(); + assertEquals("no changes - nothing to commit", "1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); + assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); + } + } + } + } + } + + public void testHistoryUUIDCanBeForced() throws IOException { + final int numDocs = randomIntBetween(0, 3); + for (int i = 0; i < numDocs; i++) { + ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.IndexResult index = engine.index(firstIndexRequest); + assertThat(index.getVersion(), equalTo(1L)); + } + assertVisibleCount(engine, numDocs); + final String oldHistoryUUID = engine.getHistoryUUID(); + engine.close(); + EngineConfig config = engine.config(); + EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); + + EngineConfig newConfig = new EngineConfig( + shardId, allocationId.getId(), + threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), + new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), + config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), + new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); + engine = new InternalEngine(newConfig); + engine.recoverFromTranslog(); + assertVisibleCount(engine, 0, false); + assertThat(engine.getHistoryUUID(), not(equalTo(oldHistoryUUID))); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 36e7d8b8e9f11..4e68a8c5715dd 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -81,7 +81,6 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.logging.Loggers; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.VersionsAndSeqNoResolver; @@ -119,7 +118,6 @@ import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; -import org.elasticsearch.test.IndexSettingsModule; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; @@ -133,7 +131,6 @@ import java.util.Base64; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; @@ -2421,21 +2418,6 @@ public void testTranslogReplayWithFailure() throws IOException { } } - private static void assertVisibleCount(InternalEngine engine, int numDocs) throws IOException { - assertVisibleCount(engine, numDocs, true); - } - - private static void assertVisibleCount(InternalEngine engine, int numDocs, boolean refresh) throws IOException { - if (refresh) { - engine.refresh("test"); - } - try (Searcher searcher = engine.acquireSearcher("test")) { - final TotalHitCountCollector collector = new TotalHitCountCollector(); - searcher.searcher().search(new MatchAllDocsQuery(), collector); - assertThat(collector.getTotalHits(), equalTo(numDocs)); - } - } - public void testTranslogCleanUpPostCommitCrash() throws Exception { IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(), defaultSettings.getNodeSettings(), defaultSettings.getScopedSettings()); @@ -2641,83 +2623,6 @@ public void testRecoverFromForeignTranslog() throws IOException { assertVisibleCount(engine, numDocs, false); } - public void testHistoryUUIDIsSetIfMissing() throws IOException { - final int numDocs = randomIntBetween(0, 3); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - Engine.IndexResult index = engine.index(firstIndexRequest); - assertThat(index.getVersion(), equalTo(1L)); - } - assertVisibleCount(engine, numDocs); - engine.close(); - - IndexWriterConfig iwc = new IndexWriterConfig(null) - .setCommitOnClose(false) - // we don't want merges to happen here - we call maybe merge on the engine - // later once we stared it up otherwise we would need to wait for it here - // we also don't specify a codec here and merges should use the engines for this index - .setMergePolicy(NoMergePolicy.INSTANCE) - .setOpenMode(IndexWriterConfig.OpenMode.APPEND); - try (IndexWriter writer = new IndexWriter(store.directory(), iwc)) { - Map newCommitData = new HashMap<>(); - for (Map.Entry entry: writer.getLiveCommitData()) { - if (entry.getKey().equals(Engine.HISTORY_UUID_KEY) == false) { - newCommitData.put(entry.getKey(), entry.getValue()); - } - } - writer.setLiveCommitData(newCommitData.entrySet()); - writer.commit(); - } - - EngineDiskUtils.verifyHasHistoryUUID(store.directory()); - - final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder() - .put(defaultSettings.getSettings()) - .put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_6_0_0_beta1) - .build()); - - EngineConfig config = engine.config(); - EngineConfig newConfig = new EngineConfig( - shardId, allocationId.getId(), - threadPool, indexSettings, null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), - config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); - engine = new InternalEngine(newConfig); - engine.recoverFromTranslog(); - assertVisibleCount(engine, numDocs, false); - assertThat(engine.getHistoryUUID(), notNullValue()); - } - - public void testHistoryUUIDCanBeForced() throws IOException { - final int numDocs = randomIntBetween(0, 3); - for (int i = 0; i < numDocs; i++) { - ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - Engine.IndexResult index = engine.index(firstIndexRequest); - assertThat(index.getVersion(), equalTo(1L)); - } - assertVisibleCount(engine, numDocs); - final String oldHistoryUUID = engine.getHistoryUUID(); - engine.close(); - EngineConfig config = engine.config(); - EngineDiskUtils.bootstrapNewHistoryFromLuceneIndex(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); - - EngineConfig newConfig = new EngineConfig( - shardId, allocationId.getId(), - threadPool, config.getIndexSettings(), null, store, newMergePolicy(), config.getAnalyzer(), config.getSimilarity(), - new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), - IndexSearcher.getDefaultQueryCachingPolicy(), config.getTranslogConfig(), TimeValue.timeValueMinutes(5), - config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, config.getTranslogRecoveryRunner(), - new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED); - engine = new InternalEngine(newConfig); - engine.recoverFromTranslog(); - assertVisibleCount(engine, 0, false); - assertThat(engine.getHistoryUUID(), not(equalTo(oldHistoryUUID))); - } - public void testShardNotAvailableExceptionWhenEngineClosedConcurrently() throws IOException, InterruptedException { AtomicReference exception = new AtomicReference<>(); String operation = randomFrom("optimize", "refresh", "flush"); @@ -2810,79 +2715,6 @@ protected void doRun() throws Exception { } } - public void testCurrentTranslogIDisCommitted() throws IOException { - final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); - try (Store store = createStore()) { - EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get); - - // create - { - EngineDiskUtils.createEmpty(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); - ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); - - try (InternalEngine engine = createEngine(config)){ - engine.index(firstIndexRequest); - globalCheckpoint.set(engine.getLocalCheckpointTracker().getCheckpoint()); - expectThrows(IllegalStateException.class, () -> engine.recoverFromTranslog()); - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - // open and recover tlog - { - for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(config)) { - assertTrue(engine.isRecovering()); - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - if (i == 0) { - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - } else { - // creating an empty index will create the first translog gen and commit it - // opening the empty index will make the second translog file but not commit it - // opening the engine again (i=0) will make the third translog file, which then be committed - assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - } - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("3", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - } - // open index with new tlog - { - EngineDiskUtils.createNewTranslog(store.directory(), config.getTranslogConfig().getTranslogPath(), - SequenceNumbers.NO_OPS_PERFORMED, shardId); - try (InternalEngine engine = new InternalEngine(config)) { - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - assertEquals(2, engine.getTranslog().currentFileGeneration()); - assertEquals(0L, engine.getTranslog().uncommittedOperations()); - } - } - - // open and recover tlog with empty tlog - { - for (int i = 0; i < 2; i++) { - try (InternalEngine engine = new InternalEngine(config)) { - Map userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - engine.recoverFromTranslog(); - userData = engine.getLastCommittedSegmentInfos().getUserData(); - assertEquals("no changes - nothing to commit", "1", userData.get(Translog.TRANSLOG_GENERATION_KEY)); - assertEquals(engine.getTranslog().getTranslogUUID(), userData.get(Translog.TRANSLOG_UUID_KEY)); - } - } - } - } - } - private static class ThrowingIndexWriter extends IndexWriter { private AtomicReference> failureToThrow = new AtomicReference<>(); 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 55f6e34fbcdda..14c7bb1f10ddc 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 @@ -31,8 +31,10 @@ import org.apache.lucene.index.MergePolicy; import org.apache.lucene.index.Term; import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.Sort; +import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.store.Directory; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; @@ -88,6 +90,7 @@ import static java.util.Collections.emptyList; import static org.elasticsearch.index.translog.TranslogDeletionPolicies.createTranslogDeletionPolicy; +import static org.hamcrest.Matchers.equalTo; public abstract class EngineTestCase extends ESTestCase { @@ -108,6 +111,21 @@ public abstract class EngineTestCase extends ESTestCase { protected Path primaryTranslogDir; protected Path replicaTranslogDir; + protected static void assertVisibleCount(Engine engine, int numDocs) throws IOException { + assertVisibleCount(engine, numDocs, true); + } + + protected static void assertVisibleCount(Engine engine, int numDocs, boolean refresh) throws IOException { + if (refresh) { + engine.refresh("test"); + } + try (Engine.Searcher searcher = engine.acquireSearcher("test")) { + final TotalHitCountCollector collector = new TotalHitCountCollector(); + searcher.searcher().search(new MatchAllDocsQuery(), collector); + assertThat(collector.getTotalHits(), equalTo(numDocs)); + } + } + @Override @Before public void setUp() throws Exception { From f8ff0aa58f0161de5e3d7d46d07ca17d9b8e5bcb Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 11 Mar 2018 17:01:15 +0100 Subject: [PATCH 23/25] license FTW --- .../index/engine/EngineDiskUtilsTests.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java index 2e65c03b99358..8949f8a524c27 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java @@ -1,3 +1,22 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.elasticsearch.index.engine; import org.apache.lucene.index.IndexWriter; From 37853c9d5ea67c6f1243ee39c1a50936e564ac28 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Sun, 11 Mar 2018 17:18:54 +0100 Subject: [PATCH 24/25] lint --- .../elasticsearch/index/engine/EngineDiskUtilsTests.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java index 8949f8a524c27..c57af9b448671 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/EngineDiskUtilsTests.java @@ -56,7 +56,8 @@ public void testHistoryUUIDIsSetIfMissing() throws IOException { final int numDocs = randomIntBetween(0, 3); for (int i = 0; i < numDocs; i++) { ParsedDocument doc = testParsedDocument(Integer.toString(i), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); Engine.IndexResult index = engine.index(firstIndexRequest); assertThat(index.getVersion(), equalTo(1L)); } @@ -111,7 +112,8 @@ public void testCurrentTranslogIDisCommitted() throws IOException { { EngineDiskUtils.createEmpty(store.directory(), config.getTranslogConfig().getTranslogPath(), shardId); ParsedDocument doc = testParsedDocument(Integer.toString(0), null, testDocument(), new BytesArray("{}"), null); - Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); + Engine.Index firstIndexRequest = new Engine.Index(newUid(doc), doc, SequenceNumbers.UNASSIGNED_SEQ_NO, 0, + Versions.MATCH_DELETED, VersionType.INTERNAL, PRIMARY, System.nanoTime(), -1, false); try (InternalEngine engine = createEngine(config)) { engine.index(firstIndexRequest); From 713911674653b6730ccca0bc5d0a4e84bb1b2a24 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Wed, 14 Mar 2018 13:25:06 +0100 Subject: [PATCH 25/25] feedback --- .../org/elasticsearch/index/engine/EngineDiskUtils.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java index 0bc4ae5199acc..f7f3aa8e9fe1d 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java +++ b/server/src/main/java/org/elasticsearch/index/engine/EngineDiskUtils.java @@ -40,10 +40,10 @@ import java.util.Map; -public final class EngineDiskUtils { - - private EngineDiskUtils() { - } +/** + * This class contains utility methods for mutating the shard lucene index and translog as a preparation to be opened. + */ +public abstract class EngineDiskUtils { /** * creates an empty lucene index and a corresponding empty translog. Any existing data will be deleted.