From 05975af69e6db63cb95f3e40d25bfa7174e006ea Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Mon, 12 Jan 2015 18:44:29 +0100 Subject: [PATCH 01/56] Add ShadowEngine --- .../cluster/metadata/IndexMetaData.java | 1 + .../elasticsearch/env/NodeEnvironment.java | 16 +- .../gateway/GatewayMetaState.java | 14 +- .../org/elasticsearch/index/IndexService.java | 4 +- .../elasticsearch/index/engine/Engine.java | 55 ++- .../index/engine/InternalEngine.java | 48 +- .../index/engine/ShadowEngine.java | 440 ++++++++++++++++++ .../index/engine/ShadowEngineFactory.java | 30 ++ .../index/gateway/IndexShardGateway.java | 34 +- .../elasticsearch/index/shard/IndexShard.java | 49 +- .../index/shard/IndexShardModule.java | 29 +- .../org/elasticsearch/index/store/Store.java | 23 +- .../cluster/IndicesClusterStateService.java | 20 +- .../indices/recovery/RecoverySource.java | 10 + .../indices/recovery/RecoveryTarget.java | 4 +- .../env/NodeEnvironmentTests.java | 4 +- .../index/IndexWithShadowReplicasTests.java | 112 +++++ .../index/shard/IndexShardModuleTests.java | 55 +++ .../elasticsearch/index/store/StoreTest.java | 4 +- 19 files changed, 855 insertions(+), 97 deletions(-) create mode 100644 src/main/java/org/elasticsearch/index/engine/ShadowEngine.java create mode 100644 src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java create mode 100644 src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java create mode 100644 src/test/java/org/elasticsearch/index/shard/IndexShardModuleTests.java diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 1d2e9646ae4b4..71bcdd4497387 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -157,6 +157,7 @@ public static State fromString(String state) { public static final String SETTING_NUMBER_OF_SHARDS = "index.number_of_shards"; public static final String SETTING_NUMBER_OF_REPLICAS = "index.number_of_replicas"; + public static final String SETTING_SHADOW_REPLICAS = "index.shadow_replicas"; public static final String SETTING_AUTO_EXPAND_REPLICAS = "index.auto_expand_replicas"; public static final String SETTING_READ_ONLY = "index.blocks.read_only"; public static final String SETTING_BLOCKS_READ = "index.blocks.read"; diff --git a/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/src/main/java/org/elasticsearch/env/NodeEnvironment.java index ab8cb91a322db..2a46ce1c49fae 100644 --- a/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -196,6 +196,12 @@ public void deleteShardDirectorySafe(ShardId shardId, @IndexSettings Settings in public void deleteShardDirectoryUnderLock(ShardLock lock, @IndexSettings Settings indexSettings) throws IOException { assert indexSettings != ImmutableSettings.EMPTY; final ShardId shardId = lock.getShardId(); + + if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { + logger.trace("skipping shard deletion because {} uses shadow replicas", shardId); + return; + } + assert isShardLocked(shardId) : "shard " + shardId + " is not locked"; final Path[] paths = shardPaths(shardId); IOUtils.rm(paths); @@ -224,11 +230,19 @@ private boolean isShardLocked(ShardId id) { * * @param index the index to delete * @param lockTimeoutMS how long to wait for acquiring the indices shard locks + * @param indexSettings settings for the index being deleted + * @param force boolean flag to delete directory even if shadow replicas are used for the index * @throws Exception if any of the shards data directories can't be locked or deleted */ - public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, @IndexSettings Settings indexSettings) throws IOException { + public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, @IndexSettings Settings indexSettings, boolean force) throws IOException { // This is to ensure someone doesn't use ImmutableSettings.EMPTY assert indexSettings != ImmutableSettings.EMPTY; + // TODO still need to delete the directory on the primary + if (force == false && indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { + logger.trace("skipping index directory deletion because {} uses shadow replicas", index); + return; + } + final List locks = lockAllForIndex(index, lockTimeoutMS); try { final Path[] indexPaths = indexPaths(index); diff --git a/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index b97d55592a23f..c6379e6be7648 100644 --- a/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -33,8 +33,8 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.cluster.routing.HashFunction; import org.elasticsearch.cluster.routing.DjbHashFunction; +import org.elasticsearch.cluster.routing.HashFunction; import org.elasticsearch.cluster.routing.SimpleHashFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; @@ -55,7 +55,9 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; -import java.nio.file.*; +import java.nio.file.DirectoryStream; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.Set; @@ -267,7 +269,7 @@ public void clusterChanged(ClusterChangedEvent event) { // it may take a couple of seconds for outstanding shard reference // to release their refs (for example, on going recoveries) // we are working on a better solution see: https://github.com/elasticsearch/elasticsearch/pull/8608 - nodeEnv.deleteIndexDirectorySafe(idx, deleteTimeout.millis(), current.settings()); + nodeEnv.deleteIndexDirectorySafe(idx, deleteTimeout.millis(), current.settings(), false); } catch (LockObtainFailedException ex) { logger.debug("[{}] failed to delete index - at least one shards is still locked", ex, current.index()); } catch (Exception ex) { @@ -318,7 +320,7 @@ public void clusterChanged(ClusterChangedEvent event) { final List shardLocks = nodeEnv.lockAllForIndex(index, 0); if (shardLocks.isEmpty()) { // no shards - try to remove the directory - nodeEnv.deleteIndexDirectorySafe(index, 0, indexMetaData.settings()); + nodeEnv.deleteIndexDirectorySafe(index, 0, indexMetaData.settings(), false); continue; } IOUtils.closeWhileHandlingException(shardLocks); @@ -332,7 +334,7 @@ public void clusterChanged(ClusterChangedEvent event) { } else if (danglingTimeout.millis() == 0) { logger.info("[{}] dangling index, exists on local file system, but not in cluster metadata, timeout set to 0, deleting now", indexName); try { - nodeEnv.deleteIndexDirectorySafe(index, 0, indexMetaData.settings()); + nodeEnv.deleteIndexDirectorySafe(index, 0, indexMetaData.settings(), false); } catch (LockObtainFailedException ex) { logger.debug("[{}] failed to delete index - at least one shards is still locked", ex, indexName); } catch (Exception ex) { @@ -592,7 +594,7 @@ public void run() { try { MetaDataStateFormat.deleteMetaState(nodeEnv.indexPaths(index)); - nodeEnv.deleteIndexDirectorySafe(index, 0, indexSettings); + nodeEnv.deleteIndexDirectorySafe(index, 0, indexSettings, true); } catch (Exception ex) { logger.debug("failed to delete dangling index", ex); } diff --git a/src/main/java/org/elasticsearch/index/IndexService.java b/src/main/java/org/elasticsearch/index/IndexService.java index f6f608f201030..72121ac02d059 100644 --- a/src/main/java/org/elasticsearch/index/IndexService.java +++ b/src/main/java/org/elasticsearch/index/IndexService.java @@ -279,7 +279,7 @@ public String indexUUID() { return indexSettings.get(IndexMetaData.SETTING_UUID, IndexMetaData.INDEX_UUID_NA_VALUE); } - public synchronized IndexShard createShard(int sShardId) throws ElasticsearchException { + public synchronized IndexShard createShard(int sShardId, boolean primary) throws ElasticsearchException { /* * TODO: we execute this in parallel but it's a synced method. Yet, we might * be able to serialize the execution via the cluster state in the future. for now we just @@ -304,7 +304,7 @@ public synchronized IndexShard createShard(int sShardId) throws ElasticsearchExc ModulesBuilder modules = new ModulesBuilder(); modules.add(new ShardsPluginsModule(indexSettings, pluginsService)); - modules.add(new IndexShardModule(shardId, indexSettings)); + modules.add(new IndexShardModule(shardId, primary, indexSettings)); modules.add(new ShardIndexingModule()); modules.add(new ShardSearchModule()); modules.add(new ShardGetModule()); diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index f86e203c961ee..2bdc50d564da1 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -32,17 +32,20 @@ import org.elasticsearch.common.Preconditions; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.uid.Versions; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.ParseContext.Document; import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import java.io.Closeable; @@ -50,6 +53,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; @@ -59,8 +63,15 @@ */ public abstract class Engine implements Closeable { + protected final ShardId shardId; protected final ESLogger logger; protected final EngineConfig engineConfig; + protected final Store store; + protected final AtomicBoolean isClosed = new AtomicBoolean(false); + protected final FailedEngineListener failedEngineListener; + protected final SnapshotDeletionPolicy deletionPolicy; + + protected volatile Throwable failedEngine = null; protected Engine(EngineConfig engineConfig) { Preconditions.checkNotNull(engineConfig.getStore(), "Store must be provided to the engine"); @@ -68,7 +79,11 @@ protected Engine(EngineConfig engineConfig) { Preconditions.checkNotNull(engineConfig.getTranslog(), "Translog must be provided to the engine"); this.engineConfig = engineConfig; + this.shardId = engineConfig.getShardId(); + this.store = engineConfig.getStore(); this.logger = Loggers.getLogger(getClass(), engineConfig.getIndexSettings(), engineConfig.getShardId()); + this.failedEngineListener = engineConfig.getFailedEngineListener(); + this.deletionPolicy = engineConfig.getDeletionPolicy(); } /** Returns 0 in the case where accountable is null, otherwise returns {@code ramBytesUsed()} */ @@ -181,6 +196,35 @@ public Condition newCondition() { public abstract void delete(DeleteByQuery delete) throws EngineException; + protected GetResult getFromSearcher(Get get) throws EngineException { + final Searcher searcher = acquireSearcher("get"); + final Versions.DocIdAndVersion docIdAndVersion; + try { + docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid()); + } catch (Throwable e) { + Releasables.closeWhileHandlingException(searcher); + //TODO: A better exception goes here + throw new EngineException(shardId, "Couldn't resolve version", e); + } + + if (docIdAndVersion != null) { + if (get.versionType().isVersionConflictForReads(docIdAndVersion.version, get.version())) { + Releasables.close(searcher); + Uid uid = Uid.createUid(get.uid().text()); + throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), docIdAndVersion.version, get.version()); + } + } + + if (docIdAndVersion != null) { + // don't release the searcher on this path, it is the + // responsibility of the caller to call GetResult.release + return new GetResult(searcher, docIdAndVersion); + } else { + Releasables.close(searcher); + return GetResult.NOT_EXISTS; + } + } + public abstract GetResult get(Get get) throws EngineException; /** @@ -192,6 +236,13 @@ public Condition newCondition() { */ public abstract Searcher acquireSearcher(String source) throws EngineException; + + protected void ensureOpen() { + if (isClosed.get()) { + throw new EngineClosedException(shardId, failedEngine); + } + } + /** * Global stats on segments. */ @@ -232,7 +283,7 @@ public Condition newCondition() { /** * Optimizes to 1 segment */ - abstract void forceMerge(boolean flush, boolean waitForMerge); + public abstract void forceMerge(boolean flush, boolean waitForMerge); /** * Triggers a forced merge on this engine diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 599299ac98354..2874f711db2c8 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -34,18 +34,14 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.math.MathUtils; -import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ReleasableLock; -import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.mapper.Uid; @@ -54,8 +50,6 @@ import org.elasticsearch.index.merge.policy.MergePolicyProvider; import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.search.nested.IncludeNestedDocsQuery; -import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.store.Store; import org.elasticsearch.index.translog.Translog; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.threadpool.ThreadPool; @@ -75,7 +69,6 @@ */ public class InternalEngine extends Engine { - protected final ShardId shardId; private final FailEngineOnMergeFailure mergeSchedulerFailureListener; private final MergeSchedulerListener mergeSchedulerListener; @@ -85,8 +78,6 @@ public class InternalEngine extends Engine { private final ShardIndexingService indexingService; @Nullable private final IndicesWarmer warmer; - private final Store store; - private final SnapshotDeletionPolicy deletionPolicy; private final Translog translog; private final MergePolicyProvider mergePolicyProvider; private final MergeSchedulerProvider mergeScheduler; @@ -100,7 +91,6 @@ public class InternalEngine extends Engine { private final SearcherFactory searcherFactory; private final SearcherManager searcherManager; - private final AtomicBoolean isClosed = new AtomicBoolean(false); private final AtomicBoolean optimizeMutex = new AtomicBoolean(); // we use flushNeeded here, since if there are no changes, then the commit won't write // will not really happen, and then the commitUserData and the new translog will not be reflected @@ -114,9 +104,7 @@ public class InternalEngine extends Engine { private final LiveVersionMap versionMap; private final Object[] dirtyLocks; - private volatile Throwable failedEngine = null; private final ReentrantLock failEngineLock = new ReentrantLock(); - private final FailedEngineListener failedEngineListener; private final AtomicLong translogIdGenerator = new AtomicLong(); private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean(); @@ -127,8 +115,6 @@ public class InternalEngine extends Engine { public InternalEngine(EngineConfig engineConfig) throws EngineException { super(engineConfig); - this.store = engineConfig.getStore(); - this.shardId = engineConfig.getShardId(); this.versionMap = new LiveVersionMap(); store.incRef(); IndexWriter writer = null; @@ -139,7 +125,6 @@ public InternalEngine(EngineConfig engineConfig) throws EngineException { this.lastDeleteVersionPruneTimeMSec = engineConfig.getThreadPool().estimatedTimeInMillis(); this.indexingService = engineConfig.getIndexingService(); this.warmer = engineConfig.getWarmer(); - this.deletionPolicy = engineConfig.getDeletionPolicy(); this.translog = engineConfig.getTranslog(); this.mergePolicyProvider = engineConfig.getMergePolicyProvider(); this.mergeScheduler = engineConfig.getMergeScheduler(); @@ -148,7 +133,6 @@ public InternalEngine(EngineConfig engineConfig) throws EngineException { dirtyLocks[i] = new Object(); } - this.failedEngineListener = engineConfig.getFailedEngineListener(); throttle = new IndexThrottle(); this.searcherFactory = new SearchFactory(engineConfig); try { @@ -252,31 +236,7 @@ public GetResult get(Get get) throws EngineException { } // no version, get the version from the index, we know that we refresh on flush - final Searcher searcher = acquireSearcher("get"); - final Versions.DocIdAndVersion docIdAndVersion; - try { - docIdAndVersion = Versions.loadDocIdAndVersion(searcher.reader(), get.uid()); - } catch (Throwable e) { - Releasables.closeWhileHandlingException(searcher); - //TODO: A better exception goes here - throw new EngineException(shardId, "Couldn't resolve version", e); - } - - if (docIdAndVersion != null) { - if (get.versionType().isVersionConflictForReads(docIdAndVersion.version, get.version())) { - Releasables.close(searcher); - Uid uid = Uid.createUid(get.uid().text()); - throw new VersionConflictEngineException(shardId, uid.type(), uid.id(), docIdAndVersion.version, get.version()); - } - } - - if (docIdAndVersion != null) { - // don't release the searcher on this path, it is the responsability of the caller to call GetResult.release - return new GetResult(searcher, docIdAndVersion); - } else { - Releasables.close(searcher); - return GetResult.NOT_EXISTS; - } + return getFromSearcher(get); } } @@ -776,12 +736,6 @@ private void flush(boolean commitTranslog, boolean force, boolean waitIfOngoing) } } - private void ensureOpen() { - if (isClosed.get()) { - throw new EngineClosedException(shardId, failedEngine); - } - } - private void pruneDeletedTombstones() { long timeMSec = engineConfig.getThreadPool().estimatedTimeInMillis(); diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java new file mode 100644 index 0000000000000..edf72050a3d48 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -0,0 +1,440 @@ +/* + * 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.LeafReaderContext; +import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentReader; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SearcherFactory; +import org.apache.lucene.search.SearcherManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.Accountables; +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; +import org.elasticsearch.common.util.concurrent.ReleasableLock; +import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; + +import java.io.IOException; +import java.util.*; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * TODO: document me! + */ +public class ShadowEngine extends Engine { + + private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); + private final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); + private final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); + private final Lock failReleasableLock = new ReentrantLock(); + + private final RecoveryCounter onGoingRecoveries; + private volatile boolean closedOrFailed = false; + private volatile SearcherManager searcherManager; + + private DirectoryReader indexReader = null; + + public ShadowEngine(EngineConfig engineConfig) { + super(engineConfig); + store.incRef(); + SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); + this.onGoingRecoveries = new RecoveryCounter(store); + try { + openNewReader(); + this.searcherManager = new SearcherManager(this.indexReader, searcherFactory); + } catch (IOException e) { + logger.warn("failed to create new reader", e); + store.decRef(); + } + } + + private final void openNewReader() throws IOException { + try (ReleasableLock _ = writeLock.acquire()) { + if (indexReader == null) { + Directory d = store.directory(); + indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(d), shardId); + } else { + // TODO might not need this, the reader is never re-opened here + DirectoryReader oldReader = indexReader; + DirectoryReader newReader = DirectoryReader.openIfChanged(indexReader); + if (newReader != null) { + indexReader = ElasticsearchDirectoryReader.wrap(newReader, shardId); + oldReader.close(); + } + } + } + } + + @Override + public void create(Create create) throws EngineException { + // no-op + logger.info("cowardly refusing to CREATE"); + } + + @Override + public void index(Index index) throws EngineException { + // no-op + logger.info("cowardly refusing to INDEX"); + } + + @Override + public void delete(Delete delete) throws EngineException { + // no-op + logger.info("cowardly refusing to DELETE"); + } + + @Override + public void delete(DeleteByQuery delete) throws EngineException { + // no-op + logger.info("cowardly refusing to DELETE-BY-QUERY"); + } + + @Override + public void flush() throws EngineException { + flush(false, false); + } + + @Override + public void flush(boolean force, boolean waitIfOngoing) throws EngineException { + logger.info("cowardly refusing to FLUSH"); + } + + @Override + public void forceMerge(boolean flush, boolean waitForMerge) { + forceMerge(flush, waitForMerge, 1, false, false); + } + + @Override + public void forceMerge(boolean flush, boolean waitForMerge, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { + logger.info("cowardly refusing to FORCE_MERGE, since the since the primary will do it"); + } + + @Override + public GetResult get(Get get) throws EngineException { + // There is no translog, so we can get it directly from the searcher + return getFromSearcher(get); + } + + protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) { + return new EngineSearcher(source, searcher, manager, store, logger); + } + + @Override + public Searcher acquireSearcher(String source) throws EngineException { + boolean success = false; + /* Acquire order here is store -> manager since we need + * to make sure that the store is not closed before + * the searcher is acquired. */ + store.incRef(); + try { + final SearcherManager manager = this.searcherManager; // can never be null + assert manager != null : "SearcherManager is null"; + /* This might throw NPE but that's fine we will run ensureOpen() + * in the catch block and throw the right exception */ + final IndexSearcher searcher = manager.acquire(); + try { + final Searcher retVal = newSearcher(source, searcher, manager); + success = true; + return retVal; + } finally { + if (!success) { + manager.release(searcher); + } + } + } catch (EngineClosedException ex) { + throw ex; + } catch (Throwable ex) { + ensureOpen(); // throw EngineCloseException here if we are already closed + logger.error("failed to acquire searcher, source {}", ex, source); + throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); + } finally { + if (!success) { // release the ref in the case of an error... + store.decRef(); + } + } + } + + @Override + public SegmentsStats segmentsStats() { + ensureOpen(); + try (final Searcher searcher = acquireSearcher("segments_stats")) { + SegmentsStats stats = new SegmentsStats(); + for (LeafReaderContext reader : searcher.reader().leaves()) { + // TODO refactor segmentReader into abstract or utility class? + final SegmentReader segmentReader = InternalEngine.segmentReader(reader.reader()); + stats.add(1, segmentReader.ramBytesUsed()); + stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader())); + stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader())); + stats.addTermVectorsMemoryInBytes(guardedRamBytesUsed(segmentReader.getTermVectorsReader())); + stats.addNormsMemoryInBytes(guardedRamBytesUsed(segmentReader.getNormsReader())); + stats.addDocValuesMemoryInBytes(guardedRamBytesUsed(segmentReader.getDocValuesReader())); + } + // No version map for shadow engine + stats.addVersionMapMemoryInBytes(0); + // Since there is no IndexWriter, these are 0 + stats.addIndexWriterMemoryInBytes(0); + stats.addIndexWriterMaxMemoryInBytes(0); + return stats; + } + } + + @Override + public List segments(boolean verbose) { + try (ReleasableLock _ = readLock.acquire()) { + ensureOpen(); + Map segments = new HashMap<>(); + + // first, go over and compute the search ones... + Searcher searcher = acquireSearcher("segments"); + try { + for (LeafReaderContext reader : searcher.reader().leaves()) { + SegmentCommitInfo info = InternalEngine.segmentReader(reader.reader()).getSegmentInfo(); + assert !segments.containsKey(info.info.name); + Segment segment = new Segment(info.info.name); + segment.search = true; + segment.docCount = reader.reader().numDocs(); + segment.delDocCount = reader.reader().numDeletedDocs(); + segment.version = info.info.getVersion(); + segment.compound = info.info.getUseCompoundFile(); + try { + segment.sizeInBytes = info.sizeInBytes(); + } catch (IOException e) { + logger.trace("failed to get size for [{}]", e, info.info.name); + } + final SegmentReader segmentReader = InternalEngine.segmentReader(reader.reader()); + segment.memoryInBytes = segmentReader.ramBytesUsed(); + if (verbose) { + segment.ramTree = Accountables.namedAccountable("root", segmentReader); + } + // TODO: add more fine grained mem stats values to per segment info here + segments.put(info.info.name, segment); + } + } finally { + searcher.close(); + } + + Segment[] segmentsArr = segments.values().toArray(new Segment[segments.values().size()]); + Arrays.sort(segmentsArr, new Comparator() { + @Override + public int compare(Segment o1, Segment o2) { + return (int) (o1.getGeneration() - o2.getGeneration()); + } + }); + + // fill in the merges flag + // TODO uncomment me +// Set onGoingMerges = mergeScheduler.onGoingMerges(); +// for (OnGoingMerge onGoingMerge : onGoingMerges) { +// for (SegmentCommitInfo segmentInfoPerCommit : onGoingMerge.getMergedSegments()) { +// for (Segment segment : segmentsArr) { +// if (segment.getName().equals(segmentInfoPerCommit.info.name)) { +// segment.mergeId = onGoingMerge.getId(); +// break; +// } +// } +// } +// } + + return Arrays.asList(segmentsArr); + } + } + + @Override + public boolean refreshNeeded() { + if (store.tryIncRef()) { + /* + we need to inc the store here since searcherManager.isSearcherCurrent() + acquires a searcher internally and that might keep a file open on the + store. this violates the assumption that all files are closed when + the store is closed so we need to make sure we increment it here + */ + try { + // if a merge has finished, we should refresh + return searcherManager.isSearcherCurrent() == false; + } catch (IOException e) { + logger.error("failed to access searcher manager", e); + failEngine("failed to access searcher manager", e); + throw new EngineException(shardId, "failed to access searcher manager", e); + } finally { + store.decRef(); + } + } + return false; + } + + @Override + public void refresh(String source) throws EngineException { + // we obtain a read lock here, since we don't want a flush to happen while we are refreshing + // since it flushes the index as well (though, in terms of concurrency, we are allowed to do it) + try (ReleasableLock _ = readLock.acquire()) { + ensureOpen(); + searcherManager.maybeRefreshBlocking(); + } catch (AlreadyClosedException e) { + ensureOpen(); + } catch (EngineClosedException e) { + throw e; + } catch (Throwable t) { + failEngine("refresh failed", t); + throw new RefreshFailedEngineException(shardId, t); + } + } + + @Override + public SnapshotIndexCommit snapshotIndex() throws EngineException { + // we have to flush outside of the readlock otherwise we might have a problem upgrading + // the to a write lock when we fail the engine in this operation + flush(false, true); + try (ReleasableLock _ = readLock.acquire()) { + ensureOpen(); + return deletionPolicy.snapshot(); + } catch (IOException e) { + throw new SnapshotFailedEngineException(shardId, e); + } + } + + // TODO refactor into abstract helper + private boolean maybeFailEngine(Throwable t, String source) { + if (Lucene.isCorruptionException(t)) { + if (engineConfig.isFailEngineOnCorruption()) { + failEngine("corrupt file detected source: [" + source + "]", t); + return true; + } else { + logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, + EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, engineConfig.isFailEngineOnCorruption()); + } + } else if (ExceptionsHelper.isOOM(t)) { + failEngine("out of memory", t); + return true; + } + return false; + } + + // TODO refactor into abstract helper + private Throwable wrapIfClosed(Throwable t) { + if (closedOrFailed) { + if (t != failedEngine && failedEngine != null) { + t.addSuppressed(failedEngine); + } + return new EngineClosedException(shardId, t); + } + return t; + } + + @Override + public void recover(RecoveryHandler recoveryHandler) throws EngineException { + // take a write lock here so it won't happen while a flush is in progress + // this means that next commits will not be allowed once the lock is released + try (ReleasableLock _ = writeLock.acquire()) { + if (closedOrFailed) { + throw new EngineClosedException(shardId, failedEngine); + } + onGoingRecoveries.startRecovery(); + } + + SnapshotIndexCommit phase1Snapshot; + try { + phase1Snapshot = deletionPolicy.snapshot(); + } catch (Throwable e) { + maybeFailEngine(e, "recovery"); + Releasables.closeWhileHandlingException(onGoingRecoveries); + throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e); + } + + boolean success = false; + try { + recoveryHandler.phase1(phase1Snapshot); + success = true; + } catch (Throwable e) { + maybeFailEngine(e, "recovery phase 1"); + Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot); + throw new RecoveryEngineException(shardId, 1, "Execution failed", wrapIfClosed(e)); + } finally { + Releasables.close(success, onGoingRecoveries, writeLock, phase1Snapshot); + } + + // Since operations cannot be replayed from a translog on a shadow + // engine, there is no phase2 and phase3 of recovery + } + + @Override + public void failEngine(String reason, Throwable failure) { + // Note, there is no IndexWriter, so nothing to rollback here + assert failure != null; + if (failReleasableLock.tryLock()) { + try { + try { + // we first mark the store as corrupted before we notify any listeners + // this must happen first otherwise we might try to reallocate so quickly + // on the same node that we don't see the corrupted marker file when + // the shard is initializing + if (Lucene.isCorruptionException(failure)) { + try { + store.markStoreCorrupted(ExceptionsHelper.unwrapCorruption(failure)); + } catch (IOException e) { + logger.warn("Couldn't marks store corrupted", e); + } + } + } finally { + if (failedEngine != null) { + logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); + return; + } + logger.warn("failed engine [{}]", failure, reason); + // we must set a failure exception, generate one if not supplied + failedEngine = failure; + failedEngineListener.onFailedEngine(shardId, reason, failure); + } + } catch (Throwable t) { + // don't bubble up these exceptions up + logger.warn("failEngine threw exception", t); + } finally { + closedOrFailed = true; + } + } else { + logger.debug("tried to fail engine but could not acquire lock - engine should be failed by now [{}]", reason, failure); + } + } + + @Override + public void close() throws IOException { + logger.debug("shadow replica close now acquiring writeLock"); + try (ReleasableLock _ = writeLock.acquire()) { + logger.debug("shadow replica close acquired writeLock"); + if (isClosed.compareAndSet(false, true)) { + try { + logger.debug("shadow replica close searcher manager"); + IOUtils.close(searcherManager); + } catch (Throwable t) { + logger.warn("shadow replica failed to close searcher manager", t); + } finally { + store.decRef(); + } + } + } + } +} diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java new file mode 100644 index 0000000000000..8606eac78de32 --- /dev/null +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java @@ -0,0 +1,30 @@ +/* + * 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; + +/** + * TODO: document me! + */ +public class ShadowEngineFactory implements EngineFactory { + @Override + public Engine newEngine(EngineConfig config) { + return new ShadowEngine(config); + } +} diff --git a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java index 49da62e51121b..35751d2047a0f 100644 --- a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java +++ b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java @@ -29,20 +29,19 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.gateway.IndexShardGateway; -import org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.*; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.rest.RestStatus; @@ -100,7 +99,17 @@ public IndexShardGateway(ShardId shardId, @IndexSettings Settings indexSettings, } } - public void recover(boolean indexShouldExists, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException { + public void recover(boolean indexShouldExist, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException { + recover(this.indexShard, this.logger, this.mappingUpdatedAction, this.cancellableThreads, + indexShouldExist, recoveryState, this.waitForMappingUpdatePostRecovery); + } + + public static void recover(IndexShard indexShard, final ESLogger logger, + final MappingUpdatedAction mappingUpdatedAction, + final CancellableThreads cancellableThreads, + boolean indexShouldExists, RecoveryState recoveryState, + final TimeValue waitForMappingUpdatePostRecovery) + throws IndexShardGatewayRecoveryException { recoveryState.getIndex().startTime(System.currentTimeMillis()); recoveryState.setStage(RecoveryState.Stage.INDEX); long version = -1; @@ -121,7 +130,8 @@ public void recover(boolean indexShouldExists, RecoveryState recoveryState) thro files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")"; } if (indexShouldExists) { - throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e); + throw new IndexShardGatewayRecoveryException(indexShard.shardId(), + "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e); } } if (si != null) { @@ -147,7 +157,8 @@ public void recover(boolean indexShouldExists, RecoveryState recoveryState) thro } } } catch (Throwable e) { - throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e); + throw new IndexShardGatewayRecoveryException(indexShard.shardId(), + "failed to fetch index version after copying it over", e); } recoveryState.getIndex().updateVersion(version); recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime()); @@ -282,7 +293,8 @@ public void recover(boolean indexShouldExists, RecoveryState recoveryState) thro } } catch (Throwable e) { IOUtils.closeWhileHandlingException(indexShard.translog()); - throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e); + throw new IndexShardGatewayRecoveryException(indexShard.shardId(), + "failed to recover shard", e); } finally { IOUtils.closeWhileHandlingException(in); } @@ -298,7 +310,11 @@ public void recover(boolean indexShouldExists, RecoveryState recoveryState) thro } for (final String type : typesToUpdate) { final CountDownLatch latch = new CountDownLatch(1); - mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() { + IndexService indexService = indexShard.indexService(); + mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), + indexService.mapperService().documentMapper(type), + indexService.indexUUID(), + new MappingUpdatedAction.MappingUpdateListener() { @Override public void onMappingUpdate() { latch.countDown(); diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index b64fe608c293c..fbfd5ce96513d 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -34,6 +34,8 @@ import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; +import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.common.Booleans; @@ -44,12 +46,12 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.VersionType; @@ -69,6 +71,7 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.flush.FlushStats; +import org.elasticsearch.index.gateway.IndexShardGateway; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.indexing.IndexingStats; @@ -89,9 +92,9 @@ import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreStats; -import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.suggest.stats.ShardSuggestService; import org.elasticsearch.index.suggest.stats.SuggestStats; import org.elasticsearch.index.termvectors.ShardTermVectorsService; @@ -149,12 +152,14 @@ public class IndexShard extends AbstractIndexShardComponent { private final IndexService indexService; private final ShardSuggestService shardSuggestService; private final ShardBitsetFilterCache shardBitsetFilterCache; + private final MappingUpdatedAction mappingUpdatedAction; + private final CancellableThreads cancellableThreads = new CancellableThreads(); private final Object mutex = new Object(); private final String checkIndexOnStartup; private final EngineConfig config; - private final EngineFactory engineFactory; private long checkIndexTook = 0; + private volatile EngineFactory engineFactory; private volatile IndexShardState state; private TimeValue refreshInterval; @@ -176,11 +181,19 @@ public class IndexShard extends AbstractIndexShardComponent { private final MapperAnalyzer mapperAnalyzer; @Inject - public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, - ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, - ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, - ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, - @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, AnalysisService analysisService, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { + public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, + IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, + ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, + IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, + ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, + ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, + PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, + CodecService codecService, ShardTermVectorsService termVectorsService, + IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, + ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, + @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, AnalysisService analysisService, + SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, + MappingUpdatedAction mappingUpdatedAction) { super(shardId, indexSettings); Preconditions.checkNotNull(store, "Store must be provided to the index shard"); Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard"); @@ -211,6 +224,7 @@ public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexS this.codecService = codecService; this.shardSuggestService = shardSuggestService; this.shardBitsetFilterCache = shardBitsetFilterCache; + this.mappingUpdatedAction = mappingUpdatedAction; state = IndexShardState.CREATED; this.refreshInterval = indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL); indexSettingsService.addListener(applyRefreshSettings); @@ -310,6 +324,22 @@ public IndexShard routingEntry(ShardRouting newRouting) { if (currentRouting.equals(newRouting)) { return this; } + + // check for a shadow replica that now needs to be transformed into + // a normal primary + if (currentRouting.primary() == false && // currently a replica + newRouting.primary() == true && // becoming a primary + indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { + this.engineFactory = new InternalEngineFactory(); + + // Recovery creates a new engine, we only need to preset the + // state to RECOVERING + state = IndexShardState.RECOVERING; + IndexShardGateway.recover(this, this.logger, this.mappingUpdatedAction, + this.cancellableThreads, true, new RecoveryState(shardId()), + // TODO make this configurable, or get it from IndexShardGateway + TimeValue.timeValueSeconds(30)); + } } if (state == IndexShardState.POST_RECOVERY) { @@ -1144,7 +1174,8 @@ private void createNewEngine() { if (state == IndexShardState.CLOSED) { throw new EngineClosedException(shardId); } - assert this.currentEngineReference.get() == null; + // TODO we need to remove this assert, but should it be replaced with something? + // assert this.currentEngineReference.get() == null; this.currentEngineReference.set(engineFactory.newEngine(config)); } } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index 0932d12eb4379..c6bbc90004aa9 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -19,34 +19,55 @@ package org.elasticsearch.index.shard; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; +import org.elasticsearch.index.engine.ShadowEngineFactory; import org.elasticsearch.index.warmer.ShardIndexWarmerService; /** - * + * The {@code IndexShardModule} module is responsible for binding the correct + * shard id, index shard, engine factory, and warming service for a newly + * created shard. */ public class IndexShardModule extends AbstractModule { public static final String ENGINE_FACTORY = "index.engine.factory"; + public static final String SHADOW_ENGINE_FACTORY = "index.shadow_engine.factory"; private static final Class DEFAULT_ENGINE_FACTORY_CLASS = InternalEngineFactory.class; + private static final Class SHADOW_ENGINE_FACTORY_CLASS = ShadowEngineFactory.class; + + private static final String ENGINE_PREFIX = "org.elasticsearch.index.engine."; + private static final String ENGINE_SUFFIX = "EngineFactory"; private final ShardId shardId; private final Settings settings; + private final boolean primary; - public IndexShardModule(ShardId shardId, Settings settings) { + public IndexShardModule(ShardId shardId, boolean primary, Settings settings) { this.settings = settings; this.shardId = shardId; + this.primary = primary; + } + + /** Return true if a shadow engine should be used */ + protected boolean useShadowEngine() { + return primary == false && settings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false); } @Override protected void configure() { bind(ShardId.class).toInstance(shardId); bind(IndexShard.class).asEagerSingleton(); - bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, - "org.elasticsearch.index.engine.", "EngineFactory")); + Class engineFactory = DEFAULT_ENGINE_FACTORY_CLASS; + String factorySetting = ENGINE_FACTORY; + if (useShadowEngine()) { + engineFactory = SHADOW_ENGINE_FACTORY_CLASS; + factorySetting = SHADOW_ENGINE_FACTORY; + } + bind(EngineFactory.class).to(settings.getAsClass(factorySetting, engineFactory, ENGINE_PREFIX, ENGINE_SUFFIX)); bind(ShardIndexWarmerService.class).asEagerSingleton(); } diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index ecb5ef818e8d1..280c19268c849 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -28,6 +28,7 @@ import org.apache.lucene.util.*; import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; @@ -539,24 +540,34 @@ private static final void failIfCorrupted(Directory directory, ShardId shardId) /** * This method deletes every file in this store that is not contained in the given source meta data or is a * legacy checksum file. After the delete it pulls the latest metadata snapshot from the store and compares it - * to the given snapshot. If the snapshots are inconsistent an illegal state exception is thrown + * to the given snapshot. If the snapshots are inconsistent an illegal state exception is thrown. + * + * If the store is part of a shadow replica, extra files are not cleaned up because they could be in use + * by a shared filesystem. * * @param reason the reason for this cleanup operation logged for each deleted file * @param sourceMetaData the metadata used for cleanup. all files in this metadata should be kept around. + * @param indexSettings the settings for the index that is being cleaned up * @throws IOException if an IOException occurs * @throws ElasticsearchIllegalStateException if the latest snapshot in this store differs from the given one after the cleanup. */ - public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData) throws IOException { + public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData, @IndexSettings Settings indexSettings) throws IOException { failIfCorrupted(); metadataLock.writeLock().lock(); try { final StoreDirectory dir = directory; + final boolean shadowReplicasInUse = indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false); for (String existingFile : dir.listAll()) { + if (shadowReplicasInUse) { + logger.debug("skipping store cleanup of [{}] because shadow replicas are in use", existingFile); + continue; + } // don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum) - if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile)) { + // we also don't want to deleted IndexWriter's write.lock + // files, since it could be a shared filesystem + if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile) && !Store.isEngineLock(existingFile)) { try { dir.deleteFile(reason, existingFile); - dir.deleteFile(existingFile); } catch (Exception e) { // ignore, we don't really care, will get deleted later on } @@ -1083,6 +1094,10 @@ public static final boolean isChecksum(String name) { return name.startsWith(CHECKSUMS_PREFIX) || name.endsWith(".cks"); // bwcomapt - .cks used to be a previous checksum file } + public static final boolean isEngineLock(String name) { + return name.equals("write.lock"); + } + /** * Produces a string representation of the given digest value. */ diff --git a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 5c53320239478..957e0d34890fc 100644 --- a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -47,6 +47,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexShardAlreadyExistsException; import org.elasticsearch.index.IndexShardMissingException; import org.elasticsearch.index.aliases.IndexAlias; @@ -56,18 +57,20 @@ import org.elasticsearch.index.gateway.IndexShardGatewayService; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.settings.IndexSettingsService; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.threadpool.ThreadPool; -import java.util.*; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; @@ -555,7 +558,7 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela continue; } - IndexShard indexShard = indexService.shard(shardId); + final IndexShard indexShard = indexService.shard(shardId); if (indexShard != null) { ShardRouting currentRoutingEntry = indexShard.routingEntry(); // if the current and global routing are initializing, but are still not the same, its a different "shard" being allocated @@ -574,7 +577,8 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela // we have an ongoing recovery, find the source based on current routing and compare them DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting); if (!recoveryState.getSourceNode().equals(sourceNode)) { - logger.debug("[{}][{}] removing shard (recovery source changed), current [{}], global [{}])", shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting); + logger.debug("[{}][{}] removing shard (recovery source changed), current [{}], global [{}])", + shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting); // closing the shard will also cancel any ongoing recovery. indexService.removeShard(shardRouting.id(), "removing shard (recovery source node changed)"); shardHasBeenRemoved = true; @@ -587,9 +591,9 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela indexService.shardInjectorSafe(shardId).getInstance(IndexShardGatewayService.class).routingStateChanged(); } } - if (shardRouting.initializing()) { - applyInitializingShard(routingTable, nodes, indexMetaData, routingTable.index(shardRouting.index()).shard(shardRouting.id()), shardRouting); + applyInitializingShard(routingTable, nodes, indexMetaData, + routingTable.index(shardRouting.index()).shard(shardRouting.id()), shardRouting); } } } @@ -688,7 +692,7 @@ private void applyInitializingShard(final RoutingTable routingTable, final Disco if (logger.isDebugEnabled()) { logger.debug("[{}][{}] creating shard", shardRouting.index(), shardId); } - IndexShard indexShard = indexService.createShard(shardId); + IndexShard indexShard = indexService.createShard(shardId, shardRouting.primary()); indexShard.routingEntry(shardRouting); indexShard.addFailedEngineListener(failedEngineHandler); } catch (IndexShardAlreadyExistsException e) { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index e6af0d9c42dce..5d3749447329d 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -19,9 +19,11 @@ package org.elasticsearch.indices.recovery; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; @@ -102,6 +104,14 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { break; } } + if (shard.routingEntry().primary() && + targetShardRouting.primary() && // must be primary-to-primary relocation + shard.indexSettings().getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { + // TODO better exception here + logger.info("aborting recovery of shadow primary to shadow primary"); + shard.engine().failEngine("attempted to relocate primary shard for shadow index", + new ElasticsearchException("aborting recovery of shadow primary to shadow primary")); + } if (targetShardRouting == null) { logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), request.targetNode()); throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 07260f0ab37e5..2e444565d0c5b 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -24,6 +24,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.StopWatch; @@ -397,7 +398,8 @@ public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel recoveryStatus.legacyChecksums().write(store); Store.MetadataSnapshot sourceMetaData = request.sourceMetaSnapshot(); try { - store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); + IndexMetaData indexMeta = clusterService.state().getMetaData().index(request.shardId().getIndex()); + store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData, indexMeta.settings()); } catch (Exception ex) { throw new RecoveryFailedException(recoveryStatus.state(), "failed to clean after recovery", ex); } diff --git a/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index 3e4ab88051b68..dd2ce09948961 100644 --- a/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -180,7 +180,7 @@ public void testDeleteSafe() throws IOException, InterruptedException { } try { - env.deleteIndexDirectorySafe(new Index("foo"), randomIntBetween(0, 10), idxSettings); + env.deleteIndexDirectorySafe(new Index("foo"), randomIntBetween(0, 10), idxSettings, false); fail("shard is locked"); } catch (LockObtainFailedException ex) { // expected @@ -223,7 +223,7 @@ protected void doRun() throws Exception { start.countDown(); blockLatch.await(); - env.deleteIndexDirectorySafe(new Index("foo"), 5000, idxSettings); + env.deleteIndexDirectorySafe(new Index("foo"), 5000, idxSettings, false); assertNull(threadException.get()); diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java new file mode 100644 index 0000000000000..1fd49cc730c65 --- /dev/null +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -0,0 +1,112 @@ +/* + * 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; + +import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; +import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.store.MockFSDirectoryService; +import org.junit.Test; + +import java.nio.file.Path; + +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.equalTo; + +/** + * TODO: document me! + */ +@ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) +public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { + + @Test + @TestLogging("_root:DEBUG,env:TRACE") + public void leesFavoriteTest() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + // check-on-close for shadow replicas does *NOT* work + .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) + // disable all the mock stuff, this isn't how to use this though... + .put(InternalTestCluster.TESTS_ENABLE_MOCK_MODULES, false) + .build(); + + String node1 = internalCluster().startNode(nodeSettings); + String node2 = internalCluster().startNode(nodeSettings); + + final String IDX = "test"; + final Path dataPath = newTempDirPath(); + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + ensureGreen(IDX); + + // So basically, the primary should fail and the replica will need to + // replay the translog, this is what this tests + client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + flushAndRefresh(IDX); + client().prepareIndex(IDX, "doc", "3").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "4").setSource("foo", "bar").get(); + refresh(); + + // Check that we can get doc 1 and 2 + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + + logger.info("--> restarting both nodes"); + if (randomBoolean()) { + internalCluster().rollingRestart(); + } else { + internalCluster().fullRestart(); + } + + assertBusy(new Runnable() { + @Override + public void run() { + logger.info("--> waiting for green..."); + assertThat(client().admin().cluster().prepareHealth().setWaitForNodes("2").get().getStatus(), equalTo(ClusterHealthStatus.GREEN)); + } + }); + + logger.info("--> performing query"); + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, 4); + + logger.info("--> deleting index"); + assertAcked(client().admin().indices().prepareDelete(IDX)); + } +} diff --git a/src/test/java/org/elasticsearch/index/shard/IndexShardModuleTests.java b/src/test/java/org/elasticsearch/index/shard/IndexShardModuleTests.java new file mode 100644 index 0000000000000..dff682569a6d7 --- /dev/null +++ b/src/test/java/org/elasticsearch/index/shard/IndexShardModuleTests.java @@ -0,0 +1,55 @@ +/* + * 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.shard; + +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ElasticsearchTestCase; +import org.junit.Test; + +/** Unit test(s) for IndexShardModule */ +public class IndexShardModuleTests extends ElasticsearchTestCase { + + @Test + public void testDetermineShadowEngineShouldBeUsed() { + ShardId shardId = new ShardId("myindex", 0); + Settings regularSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .build(); + + Settings shadowSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .build(); + + IndexShardModule ism1 = new IndexShardModule(shardId, true, regularSettings); + IndexShardModule ism2 = new IndexShardModule(shardId, false, regularSettings); + IndexShardModule ism3 = new IndexShardModule(shardId, true, shadowSettings); + IndexShardModule ism4 = new IndexShardModule(shardId, false, shadowSettings); + + assertFalse("no shadow replicas for normal settings", ism1.useShadowEngine()); + assertFalse("no shadow replicas for normal settings", ism2.useShadowEngine()); + assertFalse("no shadow replicas for primary shard with shadow settings", ism3.useShadowEngine()); + assertTrue("shadow replicas for replica shards with shadow settings", ism4.useShadowEngine()); + } +} diff --git a/src/test/java/org/elasticsearch/index/store/StoreTest.java b/src/test/java/org/elasticsearch/index/store/StoreTest.java index 88272ca9d14bb..656a29a3e6a5d 100644 --- a/src/test/java/org/elasticsearch/index/store/StoreTest.java +++ b/src/test/java/org/elasticsearch/index/store/StoreTest.java @@ -967,7 +967,7 @@ public void testCleanupFromSnapshot() throws IOException { checksums.write(store); // write one checksum file here - we expect it to survive all the cleanups if (randomBoolean()) { - store.cleanupAndVerify("test", firstMeta); + store.cleanupAndVerify("test", firstMeta, ImmutableSettings.EMPTY); String[] strings = store.directory().listAll(); int numChecksums = 0; int numNotFound = 0; @@ -983,7 +983,7 @@ public void testCleanupFromSnapshot() throws IOException { assertTrue("at least one file must not be in here since we have two commits?", numNotFound > 0); assertEquals("we wrote one checksum but it's gone now? - checksums are supposed to be kept", numChecksums, 1); } else { - store.cleanupAndVerify("test", secondMeta); + store.cleanupAndVerify("test", secondMeta, ImmutableSettings.EMPTY); String[] strings = store.directory().listAll(); int numChecksums = 0; int numNotFound = 0; From 38135af0c1991b88f168ece0efb72ffe9498ff59 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 10 Feb 2015 22:25:11 +0100 Subject: [PATCH 02/56] make tests pass --- .../elasticsearch/index/engine/Engine.java | 64 ++++++++-- .../index/engine/InternalEngine.java | 63 +--------- .../index/engine/ShadowEngine.java | 118 ++++-------------- .../elasticsearch/index/shard/IndexShard.java | 8 +- .../index/IndexWithShadowReplicasTests.java | 2 + 5 files changed, 97 insertions(+), 158 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 2bdc50d564da1..444cacd1681a0 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -49,6 +49,7 @@ import org.elasticsearch.index.translog.Translog; import java.io.Closeable; +import java.io.IOException; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -122,7 +123,7 @@ protected static boolean isMergedSegment(LeafReader reader) { } protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) { - return new EngineSearcher(source, searcher, manager, engineConfig.getStore(), logger); + return new EngineSearcher(source, searcher, manager, store, logger); } public final EngineConfig config() { @@ -234,8 +235,39 @@ protected GetResult getFromSearcher(Get get) throws EngineException { * * @see Searcher#close() */ - public abstract Searcher acquireSearcher(String source) throws EngineException; - + public final Searcher acquireSearcher(String source) throws EngineException { + boolean success = false; + /* Acquire order here is store -> manager since we need + * to make sure that the store is not closed before + * the searcher is acquired. */ + store.incRef(); + try { + final SearcherManager manager = getSearcherManager(); // can never be null + assert manager != null : "SearcherManager is null"; + /* This might throw NPE but that's fine we will run ensureOpen() + * in the catch block and throw the right exception */ + final IndexSearcher searcher = manager.acquire(); + try { + final Searcher retVal = newSearcher(source, searcher, manager); + success = true; + return retVal; + } finally { + if (!success) { + manager.release(searcher); + } + } + } catch (EngineClosedException ex) { + throw ex; + } catch (Throwable ex) { + ensureOpen(); // throw EngineCloseException here if we are already closed + logger.error("failed to acquire searcher, source {}", ex, source); + throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); + } finally { + if (!success) { // release the ref in the case of an error... + store.decRef(); + } + } + } protected void ensureOpen() { if (isClosed.get()) { @@ -253,10 +285,26 @@ protected void ensureOpen() { */ public abstract List segments(boolean verbose); - /** - * Returns true if a refresh is really needed. - */ - public abstract boolean refreshNeeded(); + public final boolean refreshNeeded() { + if (store.tryIncRef()) { + /* + we need to inc the store here since searcherManager.isSearcherCurrent() + acquires a searcher internally and that might keep a file open on the + store. this violates the assumption that all files are closed when + the store is closed so we need to make sure we increment it here + */ + try { + return !getSearcherManager().isSearcherCurrent(); + } catch (IOException e) { + logger.error("failed to access searcher manager", e); + failEngine("failed to access searcher manager", e); + throw new EngineException(shardId, "failed to access searcher manager", e); + } finally { + store.decRef(); + } + } + return false; + } /** * Refreshes the engine for new search operations to reflect the latest @@ -816,4 +864,6 @@ public void release() { } } } + + protected abstract SearcherManager getSearcherManager(); } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 2874f711db2c8..13d70aa9e2c82 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -540,63 +540,6 @@ public void delete(DeleteByQuery delete) throws EngineException { refresh("delete_by_query"); } - @Override - public final Searcher acquireSearcher(String source) throws EngineException { - boolean success = false; - /* Acquire order here is store -> manager since we need - * to make sure that the store is not closed before - * the searcher is acquired. */ - store.incRef(); - try { - final SearcherManager manager = this.searcherManager; // can never be null - assert manager != null : "SearcherManager is null"; - /* This might throw NPE but that's fine we will run ensureOpen() - * in the catch block and throw the right exception */ - final IndexSearcher searcher = manager.acquire(); - try { - final Searcher retVal = newSearcher(source, searcher, manager); - success = true; - return retVal; - } finally { - if (!success) { - manager.release(searcher); - } - } - } catch (EngineClosedException ex) { - throw ex; - } catch (Throwable ex) { - ensureOpen(); // throw EngineCloseException here if we are already closed - logger.error("failed to acquire searcher, source {}", ex, source); - throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); - } finally { - if (!success) { // release the ref in the case of an error... - store.decRef(); - } - } - } - - @Override - public boolean refreshNeeded() { - if (store.tryIncRef()) { - /* - we need to inc the store here since searcherManager.isSearcherCurrent() - acquires a searcher internally and that might keep a file open on the - store. this violates the assumption that all files are closed when - the store is closed so we need to make sure we increment it here - */ - try { - return !searcherManager.isSearcherCurrent(); - } catch (IOException e) { - logger.error("failed to access searcher manager", e); - failEngine("failed to access searcher manager", e); - throw new EngineException(shardId, "failed to access searcher manager", e); - } finally { - store.decRef(); - } - } - return false; - } - @Override public void refresh(String source) throws EngineException { // we obtain a read lock here, since we don't want a flush to happen while we are refreshing @@ -834,7 +777,6 @@ protected void doRun() throws Exception { } } - @Override public SnapshotIndexCommit snapshotIndex() throws EngineException { // we have to flush outside of the readlock otherwise we might have a problem upgrading @@ -1138,6 +1080,11 @@ public void failEngine(String reason, Throwable failure) { } } + @Override + protected SearcherManager getSearcherManager() { + return searcherManager; + } + private Object dirtyLock(BytesRef uid) { int hash = DjbHashFunction.DJB_HASH(uid.bytes, uid.offset, uid.length); return dirtyLocks[MathUtils.mod(hash, dirtyLocks.length)]; diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index edf72050a3d48..f153d99f256ed 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -30,6 +30,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.Accountables; import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; @@ -57,39 +58,33 @@ public class ShadowEngine extends Engine { private volatile boolean closedOrFailed = false; private volatile SearcherManager searcherManager; - private DirectoryReader indexReader = null; - - public ShadowEngine(EngineConfig engineConfig) { + public ShadowEngine(EngineConfig engineConfig) { super(engineConfig); - store.incRef(); SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); this.onGoingRecoveries = new RecoveryCounter(store); try { - openNewReader(); - this.searcherManager = new SearcherManager(this.indexReader, searcherFactory); - } catch (IOException e) { - logger.warn("failed to create new reader", e); - store.decRef(); - } - } - - private final void openNewReader() throws IOException { - try (ReleasableLock _ = writeLock.acquire()) { - if (indexReader == null) { - Directory d = store.directory(); - indexReader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(d), shardId); - } else { - // TODO might not need this, the reader is never re-opened here - DirectoryReader oldReader = indexReader; - DirectoryReader newReader = DirectoryReader.openIfChanged(indexReader); - if (newReader != null) { - indexReader = ElasticsearchDirectoryReader.wrap(newReader, shardId); - oldReader.close(); + DirectoryReader reader = null; + store.incRef(); + boolean success = false; + try { + reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(store.directory()), shardId); + this.searcherManager = new SearcherManager(reader, searcherFactory); + success = true; + } catch (Throwable e) { + logger.warn("failed to create new reader", e); + throw e; + } finally { + if (success == false) { + IOUtils.closeWhileHandlingException(reader); + store.decRef(); } } + } catch (IOException ex) { + throw new ElasticsearchException("failed to open index reader", ex); } } + @Override public void create(Create create) throws EngineException { // no-op @@ -140,53 +135,13 @@ public GetResult get(Get get) throws EngineException { return getFromSearcher(get); } - protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) { - return new EngineSearcher(source, searcher, manager, store, logger); - } - - @Override - public Searcher acquireSearcher(String source) throws EngineException { - boolean success = false; - /* Acquire order here is store -> manager since we need - * to make sure that the store is not closed before - * the searcher is acquired. */ - store.incRef(); - try { - final SearcherManager manager = this.searcherManager; // can never be null - assert manager != null : "SearcherManager is null"; - /* This might throw NPE but that's fine we will run ensureOpen() - * in the catch block and throw the right exception */ - final IndexSearcher searcher = manager.acquire(); - try { - final Searcher retVal = newSearcher(source, searcher, manager); - success = true; - return retVal; - } finally { - if (!success) { - manager.release(searcher); - } - } - } catch (EngineClosedException ex) { - throw ex; - } catch (Throwable ex) { - ensureOpen(); // throw EngineCloseException here if we are already closed - logger.error("failed to acquire searcher, source {}", ex, source); - throw new EngineException(shardId, "failed to acquire searcher, source " + source, ex); - } finally { - if (!success) { // release the ref in the case of an error... - store.decRef(); - } - } - } - @Override public SegmentsStats segmentsStats() { ensureOpen(); try (final Searcher searcher = acquireSearcher("segments_stats")) { SegmentsStats stats = new SegmentsStats(); for (LeafReaderContext reader : searcher.reader().leaves()) { - // TODO refactor segmentReader into abstract or utility class? - final SegmentReader segmentReader = InternalEngine.segmentReader(reader.reader()); + final SegmentReader segmentReader = segmentReader(reader.reader()); stats.add(1, segmentReader.ramBytesUsed()); stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader())); stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader())); @@ -213,7 +168,7 @@ public List segments(boolean verbose) { Searcher searcher = acquireSearcher("segments"); try { for (LeafReaderContext reader : searcher.reader().leaves()) { - SegmentCommitInfo info = InternalEngine.segmentReader(reader.reader()).getSegmentInfo(); + SegmentCommitInfo info = segmentReader(reader.reader()).getSegmentInfo(); assert !segments.containsKey(info.info.name); Segment segment = new Segment(info.info.name); segment.search = true; @@ -245,7 +200,6 @@ public int compare(Segment o1, Segment o2) { return (int) (o1.getGeneration() - o2.getGeneration()); } }); - // fill in the merges flag // TODO uncomment me // Set onGoingMerges = mergeScheduler.onGoingMerges(); @@ -264,29 +218,6 @@ public int compare(Segment o1, Segment o2) { } } - @Override - public boolean refreshNeeded() { - if (store.tryIncRef()) { - /* - we need to inc the store here since searcherManager.isSearcherCurrent() - acquires a searcher internally and that might keep a file open on the - store. this violates the assumption that all files are closed when - the store is closed so we need to make sure we increment it here - */ - try { - // if a merge has finished, we should refresh - return searcherManager.isSearcherCurrent() == false; - } catch (IOException e) { - logger.error("failed to access searcher manager", e); - failEngine("failed to access searcher manager", e); - throw new EngineException(shardId, "failed to access searcher manager", e); - } finally { - store.decRef(); - } - } - return false; - } - @Override public void refresh(String source) throws EngineException { // we obtain a read lock here, since we don't want a flush to happen while we are refreshing @@ -420,6 +351,11 @@ public void failEngine(String reason, Throwable failure) { } } + @Override + protected SearcherManager getSearcherManager() { + return searcherManager; + } + @Override public void close() throws IOException { logger.debug("shadow replica close now acquiring writeLock"); @@ -427,7 +363,7 @@ public void close() throws IOException { logger.debug("shadow replica close acquired writeLock"); if (isClosed.compareAndSet(false, true)) { try { - logger.debug("shadow replica close searcher manager"); + logger.debug("shadow replica close searcher manager refCount: {}", store.refCount()); IOUtils.close(searcherManager); } catch (Throwable t) { logger.warn("shadow replica failed to close searcher manager", t); diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index fbfd5ce96513d..bbd987b6a1434 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -335,6 +335,11 @@ public IndexShard routingEntry(ShardRouting newRouting) { // Recovery creates a new engine, we only need to preset the // state to RECOVERING state = IndexShardState.RECOVERING; + try { + performRecoveryRestart(); + } catch (IOException ex) { + failShard("failed to perform recovery restart", ex); + } IndexShardGateway.recover(this, this.logger, this.mappingUpdatedAction, this.cancellableThreads, true, new RecoveryState(shardId()), // TODO make this configurable, or get it from IndexShardGateway @@ -1174,8 +1179,7 @@ private void createNewEngine() { if (state == IndexShardState.CLOSED) { throw new EngineClosedException(shardId); } - // TODO we need to remove this assert, but should it be replaced with something? - // assert this.currentEngineReference.get() == null; + assert this.currentEngineReference.get() == null; this.currentEngineReference.set(engineFactory.newEngine(config)); } } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 1fd49cc730c65..ac6b1f7befc7b 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index; +import com.carrotsearch.randomizedtesting.annotations.Repeat; import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.search.SearchResponse; @@ -46,6 +47,7 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test @TestLogging("_root:DEBUG,env:TRACE") + @Repeat(iterations = 5) public void leesFavoriteTest() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) From 7fcb373f0617050ca1a5a577b8cf32e32dc612b0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 10 Feb 2015 23:19:21 +0100 Subject: [PATCH 03/56] make test more evil --- .../elasticsearch/index/shard/IndexShard.java | 16 ++-------------- .../index/IndexWithShadowReplicasTests.java | 13 ++++++------- .../test/store/MockFSDirectoryService.java | 3 ++- 3 files changed, 10 insertions(+), 22 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index bbd987b6a1434..73853b0a46eba 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -330,20 +330,8 @@ public IndexShard routingEntry(ShardRouting newRouting) { if (currentRouting.primary() == false && // currently a replica newRouting.primary() == true && // becoming a primary indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - this.engineFactory = new InternalEngineFactory(); - - // Recovery creates a new engine, we only need to preset the - // state to RECOVERING - state = IndexShardState.RECOVERING; - try { - performRecoveryRestart(); - } catch (IOException ex) { - failShard("failed to perform recovery restart", ex); - } - IndexShardGateway.recover(this, this.logger, this.mappingUpdatedAction, - this.cancellableThreads, true, new RecoveryState(shardId()), - // TODO make this configurable, or get it from IndexShardGateway - TimeValue.timeValueSeconds(30)); + failShard("foobar", new RuntimeException("Can't promote shadow replica")); + return this; } } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index ac6b1f7befc7b..d78001deaf8f9 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -47,26 +47,23 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test @TestLogging("_root:DEBUG,env:TRACE") - @Repeat(iterations = 5) + @Repeat(iterations = 10) public void leesFavoriteTest() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) .put("node.enable_custom_paths", true) - // check-on-close for shadow replicas does *NOT* work - .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) - // disable all the mock stuff, this isn't how to use this though... - .put(InternalTestCluster.TESTS_ENABLE_MOCK_MODULES, false) .build(); String node1 = internalCluster().startNode(nodeSettings); String node2 = internalCluster().startNode(nodeSettings); + String node3 = internalCluster().startNode(nodeSettings); final String IDX = "test"; final Path dataPath = newTempDirPath(); Settings idxSettings = ImmutableSettings.builder() .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 2) .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) .build(); @@ -91,8 +88,10 @@ public void leesFavoriteTest() throws Exception { logger.info("--> restarting both nodes"); if (randomBoolean()) { + logger.info("--> rolling restart"); internalCluster().rollingRestart(); } else { + logger.info("--> full restart"); internalCluster().fullRestart(); } @@ -100,7 +99,7 @@ public void leesFavoriteTest() throws Exception { @Override public void run() { logger.info("--> waiting for green..."); - assertThat(client().admin().cluster().prepareHealth().setWaitForNodes("2").get().getStatus(), equalTo(ClusterHealthStatus.GREEN)); + assertThat(client().admin().cluster().prepareHealth().setWaitForNodes("3").get().getStatus(), equalTo(ClusterHealthStatus.GREEN)); } }); diff --git a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 8d840ac9bcc62..3b37dc8369a77 100644 --- a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.IndexShardException; import org.elasticsearch.index.shard.IndexShardState; @@ -81,7 +82,7 @@ public void beforeIndexShardClosed(ShardId sid, @Nullable IndexShard indexShard, @IndexSettings Settings indexSettings) { if (indexShard != null && shardId.equals(sid)) { logger.info("Shard state before potentially flushing is {}", indexShard.state()); - if (validCheckIndexStates.contains(indexShard.state())) { + if (validCheckIndexStates.contains(indexShard.state()) && indexShard.engine() instanceof InternalEngine) { canRun = true; // When the the internal engine closes we do a rollback, which removes uncommitted segments // By doing a commit flush we perform a Lucene commit, but don't clear the translog, From be02cabfeebaea74b51b212957a2a466cfbfb716 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 10 Feb 2015 22:04:24 -0700 Subject: [PATCH 04/56] Add test that restarts nodes to ensure shadow replicas recover --- .../index/engine/ShadowEngine.java | 14 +- .../elasticsearch/index/shard/IndexShard.java | 4 +- .../indices/recovery/RecoverySource.java | 1 + .../index/IndexWithShadowReplicasTests.java | 167 ++++++++++++++++-- 4 files changed, 163 insertions(+), 23 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index f153d99f256ed..ed62c5482cc25 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -23,11 +23,9 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SegmentCommitInfo; import org.apache.lucene.index.SegmentReader; -import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.Directory; import org.apache.lucene.util.Accountables; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; @@ -88,25 +86,25 @@ public ShadowEngine(EngineConfig engineConfig) { @Override public void create(Create create) throws EngineException { // no-op - logger.info("cowardly refusing to CREATE"); + logger.debug("cowardly refusing to CREATE"); } @Override public void index(Index index) throws EngineException { // no-op - logger.info("cowardly refusing to INDEX"); + logger.debug("cowardly refusing to INDEX"); } @Override public void delete(Delete delete) throws EngineException { // no-op - logger.info("cowardly refusing to DELETE"); + logger.debug("cowardly refusing to DELETE"); } @Override public void delete(DeleteByQuery delete) throws EngineException { // no-op - logger.info("cowardly refusing to DELETE-BY-QUERY"); + logger.debug("cowardly refusing to DELETE-BY-QUERY"); } @Override @@ -116,7 +114,7 @@ public void flush() throws EngineException { @Override public void flush(boolean force, boolean waitIfOngoing) throws EngineException { - logger.info("cowardly refusing to FLUSH"); + logger.debug("cowardly refusing to FLUSH"); } @Override @@ -126,7 +124,7 @@ public void forceMerge(boolean flush, boolean waitForMerge) { @Override public void forceMerge(boolean flush, boolean waitForMerge, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { - logger.info("cowardly refusing to FORCE_MERGE, since the since the primary will do it"); + logger.debug("cowardly refusing to FORCE_MERGE, since the since the primary will do it"); } @Override diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 73853b0a46eba..b1dd53c195c4a 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -71,7 +71,6 @@ import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.flush.FlushStats; -import org.elasticsearch.index.gateway.IndexShardGateway; import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.ShardGetService; import org.elasticsearch.index.indexing.IndexingStats; @@ -330,7 +329,8 @@ public IndexShard routingEntry(ShardRouting newRouting) { if (currentRouting.primary() == false && // currently a replica newRouting.primary() == true && // becoming a primary indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - failShard("foobar", new RuntimeException("Can't promote shadow replica")); + failShard("can't promote shadow replica to primary", + new RuntimeException("can't promote shadow replica to primary")); return this; } } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 5d3749447329d..3875935a9abab 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -105,6 +105,7 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { } } if (shard.routingEntry().primary() && + targetShardRouting != null && targetShardRouting.primary() && // must be primary-to-primary relocation shard.indexSettings().getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { // TODO better exception here diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index d78001deaf8f9..e300dea4d5671 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,36 +19,41 @@ package org.elasticsearch.index; -import com.carrotsearch.randomizedtesting.annotations.Repeat; -import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ElasticsearchIntegrationTest; -import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Test; import java.nio.file.Path; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.LongAdder; +import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; /** - * TODO: document me! + * Tests for indices that use shadow replicas and a shared filesystem */ @ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test @TestLogging("_root:DEBUG,env:TRACE") - @Repeat(iterations = 10) - public void leesFavoriteTest() throws Exception { + public void testIndexWithFewDocuments() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) .put("node.enable_custom_paths", true) @@ -95,13 +100,8 @@ public void leesFavoriteTest() throws Exception { internalCluster().fullRestart(); } - assertBusy(new Runnable() { - @Override - public void run() { - logger.info("--> waiting for green..."); - assertThat(client().admin().cluster().prepareHealth().setWaitForNodes("3").get().getStatus(), equalTo(ClusterHealthStatus.GREEN)); - } - }); + client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); + ensureGreen(IDX); logger.info("--> performing query"); SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); @@ -110,4 +110,145 @@ public void run() { logger.info("--> deleting index"); assertAcked(client().admin().indices().prepareDelete(IDX)); } + + @Test + @LuceneTestCase.Slow + public void testChaosMonkeyWithShadowReplicas() throws Exception { + final int initialNodeCount = scaledRandomIntBetween(3, 8); + + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + // We don't want closing a node to take forever, so disable this + .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) + .put("discovery.zen.minimum_master_nodes", (initialNodeCount / 2) + 1) + .build(); + + logger.info("--> starting up {} nodes...", initialNodeCount); + final List nodes = internalCluster().startNodesAsync(initialNodeCount, nodeSettings).get(); + + // Start up a client node + Settings clientNodeSettings = ImmutableSettings.builder() + .put("node.client", true) + .build(); + String clientNode = internalCluster().startNode(clientNodeSettings); + final Client client = client(clientNode); + + final String IDX = "test"; + final String NORMAL = "normal"; // the normal index setting + final Path dataPath = newTempDirPath(); + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, initialNodeCount)) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, initialNodeCount - 1)) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .build(); + + Settings normalSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, initialNodeCount)) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, initialNodeCount - 1)) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(NORMAL).setSettings(normalSettings).get(); + ensureGreen(TimeValue.timeValueMinutes(1), IDX, NORMAL); + + // Either we're going to fiddle with replicas, or we're going to + // restart nodes, but not both + final boolean fiddleWithReplicas = randomBoolean(); + + // Flag to signle to the threads to stop doing things + final AtomicBoolean running = new AtomicBoolean(true); + + final Runnable chaosRunner = new Runnable() { + @Override + public void run() { + try { + while (running.get()) { + Thread.sleep(randomIntBetween(4000, 9000)); + // Randomly either restart nodes and change replica count + if (fiddleWithReplicas) { + int newCount = randomIntBetween(1, initialNodeCount - 1); + logger.info("--> changing replica count to {}", newCount); + assertAcked(client.admin().indices().prepareUpdateSettings(IDX) + .setSettings(ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, + newCount))); + } else { + logger.info("--> randomly restarting a data node"); + internalCluster().restartRandomDataNode(); + } + } + } catch (Throwable t) { + logger.warn("exception in chaos monkey", t); + } + } + }; + + final LongAdder totalDocs = new LongAdder(); + final Runnable indexRunner = new Runnable() { + @Override + public void run() { + while (running.get()) { + List indexRequests = newArrayList(); + int batchSize = scaledRandomIntBetween(10, 100); + for (int i = 0; i < batchSize; i++) { + indexRequests.add(client.prepareIndex() + .setIndex(randomBoolean() ? IDX : NORMAL) // Randomly use different index + .setType("doc") + .setSource("body", randomRealisticUnicodeOfCodepointLengthBetween(10, 20)) + ); + } + try { + logger.info("--> indexing batch of {} documents...", batchSize); + indexRandom(true, true, true, indexRequests); + totalDocs.add(batchSize); + if (randomBoolean()) { + // Randomly flush the shadow index + flush(IDX); + } + } catch (Throwable t) { + logger.info("--> wasn't able to index that batch, we'll get 'em next time", t); + } + } + } + }; + + Thread chaosMonkey = new Thread(chaosRunner); + Thread indexingThread = new Thread(indexRunner); + + logger.info("--> starting indexing thread and chaos monkey"); + indexingThread.start(); + chaosMonkey.start(); + + try { + // Give it 30 seconds to index and restart nodes randomly + Thread.sleep(30 * 1000); + + logger.info("--> stopping indexing and chaos thread..."); + running.getAndSet(false); + chaosMonkey.join(15 * 1000); + indexingThread.join(15 * 1000); + } catch (Throwable t) { + fail("failed to go to sleep"); + } + + logger.info("--> waiting for cluster to recover"); + client.admin().cluster().prepareHealth().setWaitForNodes(initialNodeCount + 1 + "").get(); + logger.info("--> waiting for green"); + ensureGreen(IDX, NORMAL); + + logger.info("--> flushing indices..."); + flush(IDX, NORMAL); + logger.info("--> refreshing indices..."); + refresh(); + + logger.info("--> expecting [{}] total documents", totalDocs.longValue()); + + SearchResponse resp = client.prepareSearch(IDX, NORMAL).setSearchType(SearchType.COUNT).setQuery(matchAllQuery()).get(); + assertThat("there should be " + totalDocs.longValue() + " documents that were indexed", + resp.getHits().totalHits(), + equalTo(totalDocs.longValue())); + } } From 343dc0b527a7052acdc783ac5abcaad1ef78dbda Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Feb 2015 16:05:28 +0100 Subject: [PATCH 05/56] long adder is not available in java7 --- .../elasticsearch/index/IndexWithShadowReplicasTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index e300dea4d5671..fba578a8bec34 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -37,7 +37,7 @@ import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.LongAdder; +import java.util.concurrent.atomic.AtomicLong; import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; @@ -186,7 +186,7 @@ public void run() { } }; - final LongAdder totalDocs = new LongAdder(); + final AtomicLong totalDocs = new AtomicLong(); final Runnable indexRunner = new Runnable() { @Override public void run() { @@ -203,7 +203,7 @@ public void run() { try { logger.info("--> indexing batch of {} documents...", batchSize); indexRandom(true, true, true, indexRequests); - totalDocs.add(batchSize); + totalDocs.addAndGet(batchSize); if (randomBoolean()) { // Randomly flush the shadow index flush(IDX); From 24d36c92dd82adce650e7ac8e9f0b43c83b2dc53 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Feb 2015 17:08:08 +0100 Subject: [PATCH 06/56] utilize the new delete code --- .../elasticsearch/env/NodeEnvironment.java | 15 +----- .../org/elasticsearch/index/IndexService.java | 17 ++++-- .../elasticsearch/indices/IndicesService.java | 53 +++++++++++++------ .../env/NodeEnvironmentTests.java | 4 +- 4 files changed, 55 insertions(+), 34 deletions(-) diff --git a/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 2a46ce1c49fae..0dbd9985a88ff 100644 --- a/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -196,12 +196,6 @@ public void deleteShardDirectorySafe(ShardId shardId, @IndexSettings Settings in public void deleteShardDirectoryUnderLock(ShardLock lock, @IndexSettings Settings indexSettings) throws IOException { assert indexSettings != ImmutableSettings.EMPTY; final ShardId shardId = lock.getShardId(); - - if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - logger.trace("skipping shard deletion because {} uses shadow replicas", shardId); - return; - } - assert isShardLocked(shardId) : "shard " + shardId + " is not locked"; final Path[] paths = shardPaths(shardId); IOUtils.rm(paths); @@ -231,18 +225,11 @@ private boolean isShardLocked(ShardId id) { * @param index the index to delete * @param lockTimeoutMS how long to wait for acquiring the indices shard locks * @param indexSettings settings for the index being deleted - * @param force boolean flag to delete directory even if shadow replicas are used for the index * @throws Exception if any of the shards data directories can't be locked or deleted */ - public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, @IndexSettings Settings indexSettings, boolean force) throws IOException { + public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, @IndexSettings Settings indexSettings) throws IOException { // This is to ensure someone doesn't use ImmutableSettings.EMPTY assert indexSettings != ImmutableSettings.EMPTY; - // TODO still need to delete the directory on the primary - if (force == false && indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - logger.trace("skipping index directory deletion because {} uses shadow replicas", index); - return; - } - final List locks = lockAllForIndex(index, lockTimeoutMS); try { final Path[] indexPaths = indexPaths(index); diff --git a/src/main/java/org/elasticsearch/index/IndexService.java b/src/main/java/org/elasticsearch/index/IndexService.java index bc05a05d7e527..d20f198b6c1c6 100644 --- a/src/main/java/org/elasticsearch/index/IndexService.java +++ b/src/main/java/org/elasticsearch/index/IndexService.java @@ -403,6 +403,12 @@ private void closeShardInjector(String reason, ShardId sId, Injector shardInject // call this before we close the store, so we can release resources for it indicesLifecycle.afterIndexShardClosed(sId, indexShard, indexSettings); + if (indexShard.routingEntry().primary()) { + StoreCloseListener instance = (StoreCloseListener) shardInjector.getInstance(Store.OnClose.class); + // we are the primary - we own the shard from a writing perspective + // NOCOMMIT can we make this even nicer? + instance.setOwnsShard(true); + } } finally { try { shardInjector.getInstance(Store.class).close(); @@ -430,10 +436,10 @@ private void closeInjectorResource(ShardId shardId, Injector shardInjector, Clas } } - private void onShardClose(ShardLock lock) { + private void onShardClose(ShardLock lock, boolean ownsShard) { if (deleted.get()) { // we remove that shards content if this index has been deleted try { - indicesServices.deleteShardStore("delete index", lock, indexSettings); + indicesServices.deleteShardStore("delete index", lock, indexSettings, ownsShard); } catch (IOException e) { logger.warn("{} failed to delete shard content", e, lock.getShardId()); } @@ -442,15 +448,20 @@ private void onShardClose(ShardLock lock) { private class StoreCloseListener implements Store.OnClose { private final ShardId shardId; + private volatile boolean ownsShard = false; public StoreCloseListener(ShardId shardId) { this.shardId = shardId; } + public void setOwnsShard(boolean ownsShard) { + this.ownsShard = ownsShard; + } + @Override public void handle(ShardLock lock) { assert lock.getShardId().equals(shardId) : "shard Id mismatch, expected: " + shardId + " but got: " + lock.getShardId(); - onShardClose(lock); + onShardClose(lock, ownsShard); } } diff --git a/src/main/java/org/elasticsearch/indices/IndicesService.java b/src/main/java/org/elasticsearch/indices/IndicesService.java index a3b1df54db76b..a886ce3033ffe 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -481,7 +481,9 @@ private void deleteIndexStore(String reason, Index index, Settings indexSettings // the store metadata gets wiped anyway even without the lock this is just best effort since // every shards deletes its content under the shard lock it owns. logger.debug("{} deleting index store reason [{}]", index, reason); - nodeEnv.deleteIndexDirectorySafe(index, 0, indexSettings); + if (canDeleteIndexContents(index, indexSettings)) { + nodeEnv.deleteIndexDirectorySafe(index, 0, indexSettings); + } } catch (LockObtainFailedException ex) { logger.debug("{} failed to delete index store - at least one shards is still locked", ex, index); } catch (Exception ex) { @@ -500,13 +502,17 @@ private void deleteIndexStore(String reason, Index index, Settings indexSettings * @param indexSettings the shards index settings. * @throws IOException if an IOException occurs */ - public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException { + public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings, boolean ownsShard) throws IOException { ShardId shardId = lock.getShardId(); - if (canDeleteShardContent(shardId, indexSettings) == false) { - throw new ElasticsearchIllegalStateException("Can't delete shard " + shardId); + if (canDeleteShardContent(shardId, indexSettings, ownsShard) == false) { + if (canDeleteShardContent(shardId, indexSettings, true)) { + throw new ElasticsearchIllegalStateException("Can't delete shard " + shardId + " ownsShard: " + ownsShard); + } + logger.trace("{} skip deleting shard shard ownsShard: {} ", shardId, ownsShard); + } else { + logger.trace("{} deleting shard reason [{}]", shardId, reason); + nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings); } - logger.trace("{} deleting shard reason [{}]", shardId, reason); - nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings); } /** @@ -520,13 +526,26 @@ public void deleteShardStore(String reason, ShardLock lock, Settings indexSettin */ public void deleteShardStore(String reason, ShardId shardId, IndexMetaData metaData) throws IOException { final Settings indexSettings = buildIndexSettings(metaData); - if (canDeleteShardContent(shardId, indexSettings) == false) { + if (canDeleteShardContent(shardId, indexSettings, false) == false) { throw new ElasticsearchIllegalStateException("Can't delete shard " + shardId); } nodeEnv.deleteShardDirectorySafe(shardId, indexSettings); logger.trace("{} deleting shard reason [{}]", shardId, reason); } + //NOCOMMIT document this + public boolean canDeleteIndexContents(Index index, Settings indexSettings) { + final Tuple indexServiceInjectorTuple = this.indices.get(index); + if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false) == false) { + if (indexServiceInjectorTuple == null && nodeEnv.hasNodeFile()) { + return true; + } + } else { + logger.trace("{} skipping index directory deletion due to shadow replicas", index); + } + return false; + } + /** * Returns true iff the shards content for the given shard can be deleted. * This method will return false if: @@ -545,18 +564,22 @@ public boolean canDeleteShardContent(ShardId shardId, IndexMetaData metaData) { // The plan was to make it harder to miss-use and ask for metadata instead of simple settings assert shardId.getIndex().equals(metaData.getIndex()); final Settings indexSettings = buildIndexSettings(metaData); - return canDeleteShardContent(shardId, indexSettings); + return canDeleteShardContent(shardId, indexSettings, false); } - private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings) { + private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings, boolean ownsShard) { final Tuple indexServiceInjectorTuple = this.indices.get(shardId.getIndex()); // TODO add some protection here to prevent shard deletion if we are on a shard FS or have ShadowReplicas enabled. - if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) { - final IndexService indexService = indexServiceInjectorTuple.v1(); - return indexService.hasShard(shardId.id()) == false; - } else if (nodeEnv.hasNodeFile()) { - final Path[] shardLocations = nodeEnv.shardDataPaths(shardId, indexSettings); - return FileSystemUtils.exists(shardLocations); + if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, ownsShard) == false) { + if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) { + final IndexService indexService = indexServiceInjectorTuple.v1(); + return indexService.hasShard(shardId.id()) == false; + } else if (nodeEnv.hasNodeFile()) { + final Path[] shardLocations = nodeEnv.shardDataPaths(shardId, indexSettings); + return FileSystemUtils.exists(shardLocations); + } + } else { + logger.trace("{} skipping shard directory deletion due to shadow replicas", shardId); } return false; } diff --git a/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index ac8736fb67129..b8696fbdc593b 100644 --- a/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -181,7 +181,7 @@ public void testDeleteSafe() throws IOException, InterruptedException { } try { - env.deleteIndexDirectorySafe(new Index("foo"), randomIntBetween(0, 10), idxSettings, false); + env.deleteIndexDirectorySafe(new Index("foo"), randomIntBetween(0, 10), idxSettings); fail("shard is locked"); } catch (LockObtainFailedException ex) { // expected @@ -224,7 +224,7 @@ protected void doRun() throws Exception { start.countDown(); blockLatch.await(); - env.deleteIndexDirectorySafe(new Index("foo"), 5000, idxSettings, false); + env.deleteIndexDirectorySafe(new Index("foo"), 5000, idxSettings); assertNull(threadException.get()); From 2d42736fed3ed8afda7e4aff10b65d292e1c6f92 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 11 Feb 2015 17:51:22 +0100 Subject: [PATCH 07/56] shortcut recovery if we are on a shared FS - no need to compare files etc. --- .../cluster/metadata/IndexMetaData.java | 7 ++++++ .../org/elasticsearch/index/store/Store.java | 25 ++++++++----------- .../elasticsearch/indices/IndicesService.java | 5 ++-- .../recovery/RecoveriesCollection.java | 1 + .../indices/recovery/RecoveryTarget.java | 9 ++++++- .../index/IndexWithShadowReplicasTests.java | 5 ++++ 6 files changed, 33 insertions(+), 19 deletions(-) diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 71bcdd4497387..a6bd68a170322 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -158,6 +158,7 @@ public static State fromString(String state) { public static final String SETTING_NUMBER_OF_SHARDS = "index.number_of_shards"; public static final String SETTING_NUMBER_OF_REPLICAS = "index.number_of_replicas"; public static final String SETTING_SHADOW_REPLICAS = "index.shadow_replicas"; + public static final String SETTING_SHARED_FILESYSTEM = "index.shared_filesystem"; public static final String SETTING_AUTO_EXPAND_REPLICAS = "index.auto_expand_replicas"; public static final String SETTING_READ_ONLY = "index.blocks.read_only"; public static final String SETTING_BLOCKS_READ = "index.blocks.read"; @@ -785,4 +786,10 @@ public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws } } } + + // NOCOMMIT find a good place for this and document it + public static boolean usesSharedFilesystem(Settings settings) { + return settings.getAsBoolean(SETTING_SHARED_FILESYSTEM, settings.getAsBoolean(SETTING_SHADOW_REPLICAS, false)); + } + } diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index 280c19268c849..66b33e7b8d5bf 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -556,23 +556,18 @@ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData, @In metadataLock.writeLock().lock(); try { final StoreDirectory dir = directory; - final boolean shadowReplicasInUse = indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false); - for (String existingFile : dir.listAll()) { - if (shadowReplicasInUse) { - logger.debug("skipping store cleanup of [{}] because shadow replicas are in use", existingFile); - continue; - } - // don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum) - // we also don't want to deleted IndexWriter's write.lock - // files, since it could be a shared filesystem - if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile) && !Store.isEngineLock(existingFile)) { - try { - dir.deleteFile(reason, existingFile); - } catch (Exception e) { - // ignore, we don't really care, will get deleted later on + for (String existingFile : dir.listAll()) { + // don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum) + // we also don't want to deleted IndexWriter's write.lock + // files, since it could be a shared filesystem + if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile) && !Store.isEngineLock(existingFile)) { + try { + dir.deleteFile(reason, existingFile); + } catch (Exception e) { + // ignore, we don't really care, will get deleted later on + } } } - } final Store.MetadataSnapshot metadataOrEmpty = getMetadata(); verifyAfterCleanup(sourceMetaData, metadataOrEmpty); } finally { diff --git a/src/main/java/org/elasticsearch/indices/IndicesService.java b/src/main/java/org/elasticsearch/indices/IndicesService.java index a886ce3033ffe..14108fb5efb63 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -536,7 +536,7 @@ public void deleteShardStore(String reason, ShardId shardId, IndexMetaData metaD //NOCOMMIT document this public boolean canDeleteIndexContents(Index index, Settings indexSettings) { final Tuple indexServiceInjectorTuple = this.indices.get(index); - if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false) == false) { + if (IndexMetaData.usesSharedFilesystem(indexSettings) == false) { if (indexServiceInjectorTuple == null && nodeEnv.hasNodeFile()) { return true; } @@ -569,8 +569,7 @@ public boolean canDeleteShardContent(ShardId shardId, IndexMetaData metaData) { private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings, boolean ownsShard) { final Tuple indexServiceInjectorTuple = this.indices.get(shardId.getIndex()); - // TODO add some protection here to prevent shard deletion if we are on a shard FS or have ShadowReplicas enabled. - if (indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, ownsShard) == false) { + if (IndexMetaData.usesSharedFilesystem(indexSettings) == false || ownsShard) { if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) { final IndexService indexService = indexServiceInjectorTuple.v1(); return indexService.hasShard(shardId.id()) == false; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index cac4a57b979cf..b1312cee52365 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.ESLogger; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 2e444565d0c5b..a7e2f20c026ce 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -157,7 +157,13 @@ protected void retryRecovery(final RecoveryStatus recoveryStatus, final String r private void doRecovery(final RecoveryStatus recoveryStatus) { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; - + if (IndexMetaData.usesSharedFilesystem(recoveryStatus.indexShard().indexSettings())) { + // NOCOMMIT - this is a super shortcut we need to check if statistics are all on etc. + recoveryStatus.indexShard().performRecoveryPrepareForTranslog(); + recoveryStatus.indexShard().performRecoveryFinalization(false, recoveryStatus.state()); + onGoingRecoveries.markRecoveryAsDone(recoveryStatus.recoveryId()); + return; + } logger.trace("collecting local files for {}", recoveryStatus); final Map existingFiles; try { @@ -399,6 +405,7 @@ public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel Store.MetadataSnapshot sourceMetaData = request.sourceMetaSnapshot(); try { IndexMetaData indexMeta = clusterService.state().getMetaData().index(request.shardId().getIndex()); + assert IndexMetaData.usesSharedFilesystem(indexMeta.settings()) == false : "[" + indexMeta.getIndex() +"] index uses shared FS - can't recover / clean files"; store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData, indexMeta.settings()); } catch (Exception ex) { throw new RecoveryFailedException(recoveryStatus.state(), "failed to clean after recovery", ex); diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index fba578a8bec34..a377b95b9cd98 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index; +import com.carrotsearch.randomizedtesting.annotations.Repeat; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -32,6 +33,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; +import org.junit.Ignore; import org.junit.Test; import java.nio.file.Path; @@ -53,6 +55,7 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test @TestLogging("_root:DEBUG,env:TRACE") + @Repeat(iterations = 10) // NOCOMMIT public void testIndexWithFewDocuments() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) @@ -71,6 +74,7 @@ public void testIndexWithFewDocuments() throws Exception { .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 2) .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .build(); prepareCreate(IDX).setSettings(idxSettings).get(); @@ -113,6 +117,7 @@ public void testIndexWithFewDocuments() throws Exception { @Test @LuceneTestCase.Slow + @Ignore // NOCOMMIT for now public void testChaosMonkeyWithShadowReplicas() throws Exception { final int initialNodeCount = scaledRandomIntBetween(3, 8); From 67d7df41eac5e10a1dd63ddb31de74e326e9d38b Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 10:06:05 -0700 Subject: [PATCH 08/56] Add start of ShadowEngine unit tests --- .../index/engine/ShadowEngineTests.java | 1738 +++++++++++++++++ 1 file changed, 1738 insertions(+) create mode 100644 src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java new file mode 100644 index 0000000000000..1612bc34b98ec --- /dev/null +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -0,0 +1,1738 @@ +/* + * 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 com.carrotsearch.randomizedtesting.LifecycleScope; +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexDeletionPolicy; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.LiveIndexWriterConfig; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.store.Directory; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.ImmutableSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; +import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.indexing.ShardIndexingService; +import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService; +import org.elasticsearch.index.mapper.ParseContext; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.internal.SourceFieldMapper; +import org.elasticsearch.index.mapper.internal.UidFieldMapper; +import org.elasticsearch.index.merge.policy.LogByteSizeMergePolicyProvider; +import org.elasticsearch.index.merge.policy.MergePolicyProvider; +import org.elasticsearch.index.merge.scheduler.ConcurrentMergeSchedulerProvider; +import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; +import org.elasticsearch.index.settings.IndexSettingsService; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.store.DirectoryService; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.distributor.LeastUsedDistributor; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.fs.FsTranslog; +import org.elasticsearch.test.DummyShardLock; +import org.elasticsearch.test.ElasticsearchLuceneTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.hamcrest.MatcherAssert; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; + +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean; +import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; +import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; +import static org.elasticsearch.test.ElasticsearchTestCase.newTempDirPath; +import static org.elasticsearch.test.ElasticsearchTestCase.terminate; +import static org.hamcrest.Matchers.*; + +/** + * TODO: document me! + */ +public class ShadowEngineTests extends ElasticsearchLuceneTestCase { + + protected final ShardId shardId = new ShardId(new Index("index"), 1); + + protected ThreadPool threadPool; + + private Store store; + private Store storeReplica; + + protected Translog translog; + protected Translog replicaTranslog; + + protected Engine primaryEngine; + protected Engine replicaEngine; + + private Settings defaultSettings; + private int indexConcurrency; + private String codecName; + private Path dirPath; + + @Before + public void setUp() throws Exception { + super.setUp(); + CodecService codecService = new CodecService(shardId.index()); + indexConcurrency = randomIntBetween(1, 20); + String name = Codec.getDefault().getName(); + if (Arrays.asList(codecService.availableCodecs()).contains(name)) { + // some codecs are read only so we only take the ones that we have in the service and randomly + // selected by lucene test case. + codecName = name; + } else { + codecName = "default"; + } + defaultSettings = ImmutableSettings.builder() + .put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, randomBoolean()) + .put(EngineConfig.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us + .put(EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, randomBoolean()) + .put(EngineConfig.INDEX_CODEC_SETTING, codecName) + .put(EngineConfig.INDEX_CONCURRENCY_SETTING, indexConcurrency) + .build(); // TODO randomize more settings + threadPool = new ThreadPool(getClass().getName()); + dirPath = newTempDirPath(LifecycleScope.TEST); + store = createStore(dirPath); + store.deleteContent(); + storeReplica = createStore(dirPath); + storeReplica.deleteContent(); + translog = createTranslog(); + primaryEngine = createInternalEngine(store, translog); + LiveIndexWriterConfig currentIndexWriterConfig = ((InternalEngine)primaryEngine).getCurrentIndexWriterConfig(); + + assertEquals(primaryEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); + assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName()); + if (randomBoolean()) { + primaryEngine.config().setEnableGcDeletes(false); + } + + replicaTranslog = createTranslogReplica(); + replicaEngine = createEngine(storeReplica, replicaTranslog); + + assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); + if (randomBoolean()) { + replicaEngine.config().setEnableGcDeletes(false); + } + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + replicaEngine.close(); + storeReplica.close(); + + primaryEngine.close(); + store.close(); + terminate(threadPool); + } + + private ParseContext.Document testDocumentWithTextField() { + ParseContext.Document document = testDocument(); + document.add(new TextField("value", "test", Field.Store.YES)); + return document; + } + + private ParseContext.Document testDocument() { + return new ParseContext.Document(); + } + + + private ParsedDocument testParsedDocument(String uid, String id, String type, String routing, long timestamp, long ttl, ParseContext.Document document, BytesReference source, boolean mappingsModified) { + Field uidField = new Field("_uid", uid, UidFieldMapper.Defaults.FIELD_TYPE); + Field versionField = new NumericDocValuesField("_version", 0); + document.add(uidField); + document.add(versionField); + return new ParsedDocument(uidField, versionField, id, type, routing, timestamp, ttl, Arrays.asList(document), source, mappingsModified); + } + + protected Store createStore(Path p) throws IOException { + return createStore(newMockFSDirectory(p)); + } + + protected Store createStore(final Directory directory) throws IOException { + final DirectoryService directoryService = new DirectoryService(shardId, EMPTY_SETTINGS) { + @Override + public Directory[] build() throws IOException { + return new Directory[]{ directory }; + } + + @Override + public long throttleTimeInNanos() { + return 0; + } + }; + return new Store(shardId, EMPTY_SETTINGS, directoryService, new LeastUsedDistributor(directoryService), new DummyShardLock(shardId)); + } + + protected Translog createTranslog() throws IOException { + return new FsTranslog(shardId, EMPTY_SETTINGS, Paths.get("work/fs-translog/")); + } + + protected Translog createTranslogReplica() throws IOException { + return new FsTranslog(shardId, EMPTY_SETTINGS, Paths.get("work/fs-translog/")); + } + + protected IndexDeletionPolicy createIndexDeletionPolicy() { + return new KeepOnlyLastDeletionPolicy(shardId, EMPTY_SETTINGS); + } + + protected SnapshotDeletionPolicy createSnapshotDeletionPolicy() { + return new SnapshotDeletionPolicy(createIndexDeletionPolicy()); + } + + protected MergePolicyProvider createMergePolicy() { + return new LogByteSizeMergePolicyProvider(store, new IndexSettingsService(new Index("test"), EMPTY_SETTINGS)); + } + + protected MergeSchedulerProvider createMergeScheduler(IndexSettingsService indexSettingsService) { + return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService); + } + + protected ShadowEngine createEngine(Store store, Translog translog) { + IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); + return createShadowEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)); + } + + protected InternalEngine createInternalEngine(Store store, Translog translog) { + IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); + return createInternalEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)); + } + + protected ShadowEngine createShadowEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { + return new ShadowEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider)); + } + + protected InternalEngine createInternalEngine(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { + return new InternalEngine(config(indexSettingsService, store, translog, mergeSchedulerProvider)); + } + + public EngineConfig config(IndexSettingsService indexSettingsService, Store store, Translog translog, MergeSchedulerProvider mergeSchedulerProvider) { + IndexWriterConfig iwc = newIndexWriterConfig(); + EngineConfig config = new EngineConfig(shardId, false/*per default optimization for auto generated ids is disabled*/, threadPool, new ShardIndexingService(shardId, EMPTY_SETTINGS, new ShardSlowLogIndexingService(shardId, EMPTY_SETTINGS, indexSettingsService)), indexSettingsService + , null, store, createSnapshotDeletionPolicy(), translog, createMergePolicy(), mergeSchedulerProvider, + iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(shardId.index()), new Engine.FailedEngineListener() { + @Override + public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t) { + // we don't need to notify anybody in this test + } + }); + + + return config; + } + + protected Term newUid(String id) { + return new Term("_uid", id); + } + + protected static final BytesReference B_1 = new BytesArray(new byte[]{1}); + protected static final BytesReference B_2 = new BytesArray(new byte[]{2}); + protected static final BytesReference B_3 = new BytesArray(new byte[]{3}); + + @Test + public void testSegments() throws Exception { + // nocommit - the .isCommitted() check on segments fails here + + List segments = primaryEngine.segments(false); + assertThat(segments.isEmpty(), equalTo(true)); + assertThat(primaryEngine.segmentsStats().getCount(), equalTo(0l)); + assertThat(primaryEngine.segmentsStats().getMemoryInBytes(), equalTo(0l)); + final boolean defaultCompound = defaultSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, true); + + // create a doc and refresh + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + primaryEngine.create(new Engine.Create(null, newUid("2"), doc2)); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(false); + assertThat(segments.size(), equalTo(1)); + SegmentsStats stats = primaryEngine.segmentsStats(); + assertThat(stats.getCount(), equalTo(1l)); + assertThat(stats.getTermsMemoryInBytes(), greaterThan(0l)); + assertThat(stats.getStoredFieldsMemoryInBytes(), greaterThan(0l)); + assertThat(stats.getTermVectorsMemoryInBytes(), equalTo(0l)); + assertThat(stats.getNormsMemoryInBytes(), greaterThan(0l)); + assertThat(stats.getDocValuesMemoryInBytes(), greaterThan(0l)); + assertThat(segments.get(0).isCommitted(), equalTo(false)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(2)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + assertThat(segments.get(0).ramTree, nullValue()); + + // Check that the replica sees nothing + segments = replicaEngine.segments(false); + assertThat(segments.size(), equalTo(0)); + stats = replicaEngine.segmentsStats(); + assertThat(stats.getCount(), equalTo(0l)); + assertThat(stats.getTermsMemoryInBytes(), equalTo(0l)); + assertThat(stats.getStoredFieldsMemoryInBytes(), equalTo(0l)); + assertThat(stats.getTermVectorsMemoryInBytes(), equalTo(0l)); + assertThat(stats.getNormsMemoryInBytes(), equalTo(0l)); + assertThat(stats.getDocValuesMemoryInBytes(), equalTo(0l)); + assertThat(segments.size(), equalTo(0)); + + // flush the primary engine + primaryEngine.flush(); + // refresh the replica + replicaEngine.refresh("tests"); + + // Check that the primary AND replica sees segments now + segments = primaryEngine.segments(false); + assertThat(segments.size(), equalTo(1)); + assertThat(primaryEngine.segmentsStats().getCount(), equalTo(1l)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(2)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + + segments = replicaEngine.segments(false); + assertThat(segments.size(), equalTo(1)); + assertThat(replicaEngine.segmentsStats().getCount(), equalTo(1l)); + //assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(2)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + + + primaryEngine.config().setCompoundOnFlush(false); + + ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, false); + primaryEngine.create(new Engine.Create(null, newUid("3"), doc3)); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(false); + assertThat(segments.size(), equalTo(2)); + assertThat(primaryEngine.segmentsStats().getCount(), equalTo(2l)); + assertThat(primaryEngine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes())); + assertThat(primaryEngine.segmentsStats().getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes())); + assertThat(primaryEngine.segmentsStats().getTermVectorsMemoryInBytes(), equalTo(0l)); + assertThat(primaryEngine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes())); + assertThat(primaryEngine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes())); + assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(2)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isSearch(), equalTo(true)); + assertThat(segments.get(1).getNumDocs(), equalTo(1)); + assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(1).isCompound(), equalTo(false)); + + // Make visible to shadow replica + primaryEngine.flush(); + replicaEngine.refresh("test"); + + segments = replicaEngine.segments(false); + assertThat(segments.size(), equalTo(2)); + assertThat(replicaEngine.segmentsStats().getCount(), equalTo(2l)); + assertThat(replicaEngine.segmentsStats().getTermsMemoryInBytes(), greaterThan(stats.getTermsMemoryInBytes())); + assertThat(replicaEngine.segmentsStats().getStoredFieldsMemoryInBytes(), greaterThan(stats.getStoredFieldsMemoryInBytes())); + assertThat(replicaEngine.segmentsStats().getTermVectorsMemoryInBytes(), equalTo(0l)); + assertThat(replicaEngine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes())); + assertThat(replicaEngine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes())); + assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true)); + //assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(2)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isSearch(), equalTo(true)); + assertThat(segments.get(1).getNumDocs(), equalTo(1)); + assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(1).isCompound(), equalTo(false)); + + primaryEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(false); + assertThat(segments.size(), equalTo(2)); + assertThat(primaryEngine.segmentsStats().getCount(), equalTo(2l)); + assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(1)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(1)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + //assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isSearch(), equalTo(true)); + assertThat(segments.get(1).getNumDocs(), equalTo(1)); + assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(1).isCompound(), equalTo(false)); + + // Make visible to shadow replica + primaryEngine.flush(); + replicaEngine.refresh("test"); + + primaryEngine.config().setCompoundOnFlush(true); + ParsedDocument doc4 = testParsedDocument("4", "4", "test", null, -1, -1, testDocumentWithTextField(), B_3, false); + primaryEngine.create(new Engine.Create(null, newUid("4"), doc4)); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(false); + assertThat(segments.size(), equalTo(3)); + assertThat(primaryEngine.segmentsStats().getCount(), equalTo(3l)); + assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isSearch(), equalTo(true)); + assertThat(segments.get(0).getNumDocs(), equalTo(1)); + assertThat(segments.get(0).getDeletedDocs(), equalTo(1)); + assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); + + //assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isSearch(), equalTo(true)); + assertThat(segments.get(1).getNumDocs(), equalTo(1)); + assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(1).isCompound(), equalTo(false)); + + //assertThat(segments.get(2).isCommitted(), equalTo(false)); + assertThat(segments.get(2).isSearch(), equalTo(true)); + assertThat(segments.get(2).getNumDocs(), equalTo(1)); + assertThat(segments.get(2).getDeletedDocs(), equalTo(0)); + assertThat(segments.get(2).isCompound(), equalTo(true)); + } + + @Test + public void testVerboseSegments() throws Exception { + List segments = primaryEngine.segments(true); + assertThat(segments.isEmpty(), equalTo(true)); + + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(true); + assertThat(segments.size(), equalTo(1)); + assertThat(segments.get(0).ramTree, notNullValue()); + + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + primaryEngine.create(new Engine.Create(null, newUid("2"), doc2)); + primaryEngine.refresh("test"); + ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, false); + primaryEngine.create(new Engine.Create(null, newUid("3"), doc3)); + primaryEngine.refresh("test"); + + segments = primaryEngine.segments(true); + assertThat(segments.size(), equalTo(3)); + assertThat(segments.get(0).ramTree, notNullValue()); + assertThat(segments.get(1).ramTree, notNullValue()); + assertThat(segments.get(2).ramTree, notNullValue()); + + // Now make the changes visible to the replica + primaryEngine.flush(); + replicaEngine.refresh("test"); + + segments = replicaEngine.segments(true); + assertThat(segments.size(), equalTo(3)); + assertThat(segments.get(0).ramTree, notNullValue()); + assertThat(segments.get(1).ramTree, notNullValue()); + assertThat(segments.get(2).ramTree, notNullValue()); + + } + + +/* @Test + public void testSegmentsWithMergeFlag() throws Exception { + final Store store = createStore(); + ConcurrentMergeSchedulerProvider mergeSchedulerProvider = new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, new IndexSettingsService(shardId.index(), EMPTY_SETTINGS)); + final AtomicReference waitTillMerge = new AtomicReference<>(); + final AtomicReference waitForMerge = new AtomicReference<>(); + mergeSchedulerProvider.addListener(new MergeSchedulerProvider.Listener() { + @Override + public void beforeMerge(OnGoingMerge merge) { + try { + if (waitTillMerge.get() != null) { + waitTillMerge.get().countDown(); + } + if (waitForMerge.get() != null) { + waitForMerge.get().await(); + } + } catch (InterruptedException e) { + throw ExceptionsHelper.convertToRuntime(e); + } + } + + @Override + public void afterMerge(OnGoingMerge merge) { + } + }); + + IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); + final Engine engine = createShadowEngine(indexSettingsService, store, createTranslog(), mergeSchedulerProvider); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + engine.flush(); + assertThat(engine.segments(false).size(), equalTo(1)); + index = new Engine.Index(null, newUid("2"), doc); + engine.index(index); + engine.flush(); + List segments = engine.segments(false); + assertThat(segments.size(), equalTo(2)); + for (Segment segment : segments) { + assertThat(segment.getMergeId(), nullValue()); + } + index = new Engine.Index(null, newUid("3"), doc); + engine.index(index); + engine.flush(); + segments = engine.segments(false); + assertThat(segments.size(), equalTo(3)); + for (Segment segment : segments) { + assertThat(segment.getMergeId(), nullValue()); + } + + waitTillMerge.set(new CountDownLatch(1)); + waitForMerge.set(new CountDownLatch(1)); + engine.forceMerge(false, false); + waitTillMerge.get().await(); + + for (Segment segment : engine.segments(false)) { + assertThat(segment.getMergeId(), notNullValue()); + } + + waitForMerge.get().countDown(); + + index = new Engine.Index(null, newUid("4"), doc); + engine.index(index); + engine.flush(); + final long gen1 = store.readLastCommittedSegmentsInfo().getGeneration(); + // now, optimize and wait for merges, see that we have no merge flag + engine.forceMerge(true, true); + + for (Segment segment : engine.segments(false)) { + assertThat(segment.getMergeId(), nullValue()); + } + // we could have multiple underlying merges, so the generation may increase more than once + assertTrue(store.readLastCommittedSegmentsInfo().getGeneration() > gen1); + + final boolean flush = randomBoolean(); + final long gen2 = store.readLastCommittedSegmentsInfo().getGeneration(); + engine.forceMerge(flush, false); + waitTillMerge.get().await(); + for (Segment segment : engine.segments(false)) { + assertThat(segment.getMergeId(), nullValue()); + } + waitForMerge.get().countDown(); + + if (flush) { + awaitBusy(new Predicate() { + @Override + public boolean apply(Object o) { + try { + // we should have had just 1 merge, so last generation should be exact + return store.readLastCommittedSegmentsInfo().getLastGeneration() == gen2; + } catch (IOException e) { + throw ExceptionsHelper.convertToRuntime(e); + } + } + }); + } + + engine.close(); + store.close(); + } */ + + @Test + public void testSimpleOperations() throws Exception { + Engine.Searcher searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + searchResult.close(); + + // create a document + ParseContext.Document document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + + // its not there... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + + // not on the replica either... + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + + // but, we can still get it (in realtime) + Engine.GetResult getResult = primaryEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.source().source.toBytesArray(), equalTo(B_1.toBytesArray())); + assertThat(getResult.docIdAndVersion(), nullValue()); + getResult.release(); + + // can't get it from the replica, because it's not in the translog for a shadow replica + getResult = replicaEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + getResult.release(); + + // but, not there non realtime + getResult = primaryEngine.get(new Engine.Get(false, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + getResult.release(); + // refresh and it should be there + primaryEngine.refresh("test"); + + // now its there... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + + // also in non realtime + getResult = primaryEngine.get(new Engine.Get(false, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + + // still not in the replica because no flush + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + + // now do an update + document = testDocument(); + document.add(new TextField("value", "test1", Field.Store.YES)); + document.add(new Field(SourceFieldMapper.NAME, B_2.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); + doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_2, false); + primaryEngine.index(new Engine.Index(null, newUid("1"), doc)); + + // its not updated yet... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + + // but, we can still get it (in realtime) + getResult = primaryEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.source().source.toBytesArray(), equalTo(B_2.toBytesArray())); + assertThat(getResult.docIdAndVersion(), nullValue()); + getResult.release(); + + // refresh and it should be updated + primaryEngine.refresh("test"); + + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); + searchResult.close(); + + // flush, now shadow replica should have the files + primaryEngine.flush(); + + // still not in the replica because the replica hasn't refreshed + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + + replicaEngine.refresh("test"); + + // the replica finally sees it because primary has flushed and replica refreshed + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); + searchResult.close(); + + // now delete + primaryEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + + // its not deleted yet + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); + searchResult.close(); + + // but, get should not see it (in realtime) + getResult = primaryEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + getResult.release(); + + // refresh and it should be deleted + primaryEngine.refresh("test"); + + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + + // add it back + document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); + doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + + // its not there... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + + // refresh and it should be there + primaryEngine.refresh("test"); + + // now its there... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + + // now flush + primaryEngine.flush(); + + // and, verify get (in real time) + getResult = primaryEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.source(), nullValue()); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + + // the replica should see it if we refresh too! + replicaEngine.refresh("test"); + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + getResult = replicaEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.source(), nullValue()); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + + // make sure we can still work with the engine + // now do an update + document = testDocument(); + document.add(new TextField("value", "test1", Field.Store.YES)); + doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + primaryEngine.index(new Engine.Index(null, newUid("1"), doc)); + + // its not updated yet... + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 0)); + searchResult.close(); + + // refresh and it should be updated + primaryEngine.refresh("test"); + + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); + searchResult.close(); + + // Make visible to shadow replica + primaryEngine.flush(); + replicaEngine.refresh("test"); + + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test1")), 1)); + searchResult.close(); + } + +/* @Test + public void testSearchResultRelease() throws Exception { + Engine.Searcher searchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + searchResult.close(); + + // create a document + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + + // its not there... + searchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + + // refresh and it should be there + engine.refresh("test"); + + // now its there... + searchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + // don't release the search result yet... + + // delete, refresh and do a new search, it should not be there + engine.delete(new Engine.Delete("test", "1", newUid("1"))); + engine.refresh("test"); + Engine.Searcher updateSearchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(updateSearchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + updateSearchResult.close(); + + // the non release search result should not see the deleted yet... + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + } + + @Test + public void testFailEngineOnCorruption() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + engine.flush(); + final boolean failEngine = defaultSettings.getAsBoolean(EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, false); + final int failInPhase = randomIntBetween(1, 3); + try { + engine.recover(new Engine.RecoveryHandler() { + @Override + public void phase1(SnapshotIndexCommit snapshot) throws EngineException { + if (failInPhase == 1) { + throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); + } + } + + @Override + public void phase2(Translog.Snapshot snapshot) throws EngineException { + if (failInPhase == 2) { + throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); + } + } + + @Override + public void phase3(Translog.Snapshot snapshot) throws EngineException { + if (failInPhase == 3) { + throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); + } + } + }); + fail("exception expected"); + } catch (RuntimeException ex) { + + } + try { + Engine.Searcher searchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + engine.create(new Engine.Create(null, newUid("2"), doc2)); + engine.refresh("foo"); + + searchResult = engine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 2)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(2)); + searchResult.close(); + assertThat(failEngine, is(false)); + } catch (EngineClosedException ex) { + assertThat(failEngine, is(true)); + } + } + + + @Test + public void testSimpleRecover() throws Exception { + final ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + engine.flush(); + + engine.recover(new Engine.RecoveryHandler() { + @Override + public void phase1(SnapshotIndexCommit snapshot) throws EngineException { + try { + engine.flush(); + assertThat("flush is not allowed in phase 1", false, equalTo(true)); + } catch (FlushNotAllowedEngineException e) { + // all is well + } + } + + @Override + public void phase2(Translog.Snapshot snapshot) throws EngineException { + MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); + try { + engine.flush(); + assertThat("flush is not allowed in phase 2", false, equalTo(true)); + } catch (FlushNotAllowedEngineException e) { + // all is well + } + + // but we can index + engine.index(new Engine.Index(null, newUid("1"), doc)); + } + + @Override + public void phase3(Translog.Snapshot snapshot) throws EngineException { + MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); + try { + // we can do this here since we are on the same thread + engine.flush(); + assertThat("flush is not allowed in phase 3", false, equalTo(true)); + } catch (FlushNotAllowedEngineException e) { + // all is well + } + } + }); + // post recovery should flush the translog + MatcherAssert.assertThat(translog.snapshot(), TranslogSizeMatcher.translogSize(0)); + + engine.flush(); + engine.close(); + } + + @Test + public void testRecoverWithOperationsBetweenPhase1AndPhase2() throws Exception { + ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc1)); + engine.flush(); + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + engine.create(new Engine.Create(null, newUid("2"), doc2)); + + engine.recover(new Engine.RecoveryHandler() { + @Override + public void phase1(SnapshotIndexCommit snapshot) throws EngineException { + } + + @Override + public void phase2(Translog.Snapshot snapshot) throws EngineException { + Translog.Create create = (Translog.Create) snapshot.next(); + assertThat("translog snapshot should not read null", create != null, equalTo(true)); + assertThat(create.source().toBytesArray(), equalTo(B_2)); + assertThat(snapshot.next(), equalTo(null)); + } + + @Override + public void phase3(Translog.Snapshot snapshot) throws EngineException { + MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); + } + }); + + engine.flush(); + engine.close(); + } + + @Test + public void testRecoverWithOperationsBetweenPhase1AndPhase2AndPhase3() throws Exception { + ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc1)); + engine.flush(); + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + engine.create(new Engine.Create(null, newUid("2"), doc2)); + + engine.recover(new Engine.RecoveryHandler() { + @Override + public void phase1(SnapshotIndexCommit snapshot) throws EngineException { + } + + @Override + public void phase2(Translog.Snapshot snapshot) throws EngineException { + Translog.Create create = (Translog.Create) snapshot.next(); + assertThat(create != null, equalTo(true)); + assertThat(snapshot.next(), equalTo(null)); + assertThat(create.source().toBytesArray(), equalTo(B_2)); + + // add for phase3 + ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, false); + engine.create(new Engine.Create(null, newUid("3"), doc3)); + } + + @Override + public void phase3(Translog.Snapshot snapshot) throws EngineException { + Translog.Create create = (Translog.Create) snapshot.next(); + assertThat(create != null, equalTo(true)); + assertThat(snapshot.next(), equalTo(null)); + assertThat(create.source().toBytesArray(), equalTo(B_3)); + } + }); + + engine.flush(); + engine.close(); + } + + @Test + public void testVersioningNewCreate() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Create create = new Engine.Create(null, newUid("1"), doc); + engine.create(create); + assertThat(create.version(), equalTo(1l)); + + create = new Engine.Create(null, newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.create(create); + assertThat(create.version(), equalTo(1l)); + } + + @Test + public void testExternalVersioningNewCreate() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Create create = new Engine.Create(null, newUid("1"), doc, 12, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, 0); + engine.create(create); + assertThat(create.version(), equalTo(12l)); + + create = new Engine.Create(null, newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.create(create); + assertThat(create.version(), equalTo(12l)); + } + + @Test + public void testVersioningNewIndex() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + assertThat(index.version(), equalTo(1l)); + } + + @Test + public void testExternalVersioningNewIndex() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + engine.index(index); + assertThat(index.version(), equalTo(12l)); + + index = new Engine.Index(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + assertThat(index.version(), equalTo(12l)); + } + + @Test + public void testVersioningIndexConflict() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // future versions should not work as well + index = new Engine.Index(null, newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testExternalVersioningIndexConflict() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + engine.index(index); + assertThat(index.version(), equalTo(12l)); + + index = new Engine.Index(null, newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + engine.index(index); + assertThat(index.version(), equalTo(14l)); + + index = new Engine.Index(null, newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testVersioningIndexConflictWithFlush() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + engine.flush(); + + index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // future versions should not work as well + index = new Engine.Index(null, newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testExternalVersioningIndexConflictWithFlush() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); + engine.index(index); + assertThat(index.version(), equalTo(12l)); + + index = new Engine.Index(null, newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); + engine.index(index); + assertThat(index.version(), equalTo(14l)); + + engine.flush(); + + index = new Engine.Index(null, newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testVersioningDeleteConflict() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + try { + engine.delete(delete); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // future versions should not work as well + delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + try { + engine.delete(delete); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // now actually delete + delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + engine.delete(delete); + assertThat(delete.version(), equalTo(3l)); + + // now check if we can index to a delete doc with version + index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // we shouldn't be able to create as well + Engine.Create create = new Engine.Create(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.create(create); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testVersioningDeleteConflictWithFlush() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + engine.flush(); + + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); + try { + engine.delete(delete); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // future versions should not work as well + delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); + try { + engine.delete(delete); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + engine.flush(); + + // now actually delete + delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); + engine.delete(delete); + assertThat(delete.version(), equalTo(3l)); + + engine.flush(); + + // now check if we can index to a delete doc with version + index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // we shouldn't be able to create as well + Engine.Create create = new Engine.Create(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.create(create); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testVersioningCreateExistsException() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Create create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); + engine.create(create); + assertThat(create.version(), equalTo(1l)); + + create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.create(create); + fail(); + } catch (DocumentAlreadyExistsException e) { + // all is well + } + } + + @Test + public void testVersioningCreateExistsExceptionWithFlush() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Create create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); + engine.create(create); + assertThat(create.version(), equalTo(1l)); + + engine.flush(); + + create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); + try { + engine.create(create); + fail(); + } catch (DocumentAlreadyExistsException e) { + // all is well + } + } + + @Test + public void testVersioningReplicaConflict1() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + // apply the second index to the replica, should work fine + index = new Engine.Index(null, newUid("1"), doc, index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + assertThat(index.version(), equalTo(2l)); + + // now, the old one should not work + index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + try { + replicaEngine.index(index); + fail(); + } catch (VersionConflictEngineException e) { + // all is well + } + + // second version on replica should fail as well + try { + index = new Engine.Index(null, newUid("1"), doc, 2l + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + assertThat(index.version(), equalTo(2l)); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + @Test + public void testVersioningReplicaConflict2() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(1l)); + + // apply the first index to the replica, should work fine + index = new Engine.Index(null, newUid("1"), doc, 1l + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + assertThat(index.version(), equalTo(1l)); + + // index it again + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertThat(index.version(), equalTo(2l)); + + // now delete it + Engine.Delete delete = new Engine.Delete("test", "1", newUid("1")); + engine.delete(delete); + assertThat(delete.version(), equalTo(3l)); + + // apply the delete on the replica (skipping the second index) + delete = new Engine.Delete("test", "1", newUid("1"), 3l + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); + replicaEngine.delete(delete); + assertThat(delete.version(), equalTo(3l)); + + // second time delete with same version should fail + try { + delete = new Engine.Delete("test", "1", newUid("1"), 3l + , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); + replicaEngine.delete(delete); + fail("excepted VersionConflictEngineException to be thrown"); + } catch (VersionConflictEngineException e) { + // all is well + } + + // now do the second index on the replica, it should fail + try { + index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); + replicaEngine.index(index); + fail("excepted VersionConflictEngineException to be thrown"); + } catch (VersionConflictEngineException e) { + // all is well + } + } + + + @Test + public void testBasicCreatedFlag() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertTrue(index.created()); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertFalse(index.created()); + + engine.delete(new Engine.Delete(null, "1", newUid("1"))); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertTrue(index.created()); + } + + @Test + public void testCreatedFlagAfterFlush() { + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + Engine.Index index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertTrue(index.created()); + + engine.delete(new Engine.Delete(null, "1", newUid("1"))); + + engine.flush(); + + index = new Engine.Index(null, newUid("1"), doc); + engine.index(index); + assertTrue(index.created()); + } + + private static class MockAppender extends AppenderSkeleton { + public boolean sawIndexWriterMessage; + + public boolean sawIndexWriterIFDMessage; + + @Override + protected void append(LoggingEvent event) { + if (event.getLevel() == Level.TRACE && event.getMessage().toString().contains("[index][1] ")) { + if (event.getLoggerName().endsWith("lucene.iw") && + event.getMessage().toString().contains("IW: apply all deletes during flush")) { + sawIndexWriterMessage = true; + } + if (event.getLoggerName().endsWith("lucene.iw.ifd")) { + sawIndexWriterIFDMessage = true; + } + } + } + + @Override + public boolean requiresLayout() { + return false; + } + + @Override + public void close() { + } + } + + // #5891: make sure IndexWriter's infoStream output is + // sent to lucene.iw with log level TRACE: + + @Test + public void testIndexWriterInfoStream() { + MockAppender mockAppender = new MockAppender(); + + Logger rootLogger = Logger.getRootLogger(); + Level savedLevel = rootLogger.getLevel(); + rootLogger.addAppender(mockAppender); + rootLogger.setLevel(Level.DEBUG); + + try { + // First, with DEBUG, which should NOT log IndexWriter output: + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + engine.flush(); + assertFalse(mockAppender.sawIndexWriterMessage); + + // Again, with TRACE, which should log IndexWriter output: + rootLogger.setLevel(Level.TRACE); + engine.create(new Engine.Create(null, newUid("2"), doc)); + engine.flush(); + assertTrue(mockAppender.sawIndexWriterMessage); + + } finally { + rootLogger.removeAppender(mockAppender); + rootLogger.setLevel(savedLevel); + } + } + + // #8603: make sure we can separately log IFD's messages + public void testIndexWriterIFDInfoStream() { + MockAppender mockAppender = new MockAppender(); + + // Works when running this test inside Intellij: + Logger iwIFDLogger = LogManager.exists("org.elasticsearch.index.engine.lucene.iw.ifd"); + if (iwIFDLogger == null) { + // Works when running this test from command line: + iwIFDLogger = LogManager.exists("index.engine.lucene.iw.ifd"); + assertNotNull(iwIFDLogger); + } + + Level savedLevel = iwIFDLogger.getLevel(); + iwIFDLogger.addAppender(mockAppender); + iwIFDLogger.setLevel(Level.DEBUG); + + try { + // First, with DEBUG, which should NOT log IndexWriter output: + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + engine.create(new Engine.Create(null, newUid("1"), doc)); + engine.flush(); + assertFalse(mockAppender.sawIndexWriterMessage); + assertFalse(mockAppender.sawIndexWriterIFDMessage); + + // Again, with TRACE, which should only log IndexWriter IFD output: + iwIFDLogger.setLevel(Level.TRACE); + engine.create(new Engine.Create(null, newUid("2"), doc)); + engine.flush(); + assertFalse(mockAppender.sawIndexWriterMessage); + assertTrue(mockAppender.sawIndexWriterIFDMessage); + + } finally { + iwIFDLogger.removeAppender(mockAppender); + iwIFDLogger.setLevel(null); + } + } + + @Slow + @Test + public void testEnableGcDeletes() throws Exception { + + Store store = createStore(); + + + + IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); + Engine engine = new ShadowEngine(config(indexSettingsService, store, createTranslog(), createMergeScheduler(indexSettingsService))); + ((ShadowEngine)engine).config().setEnableGcDeletes(false); + + // Add document + ParseContext.Document document = testDocument(); + document.add(new TextField("value", "test1", Field.Store.YES)); + + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_2, false); + engine.index(new Engine.Index(null, newUid("1"), doc, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + + // Delete document we just added: + engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + + // Get should not find the document + Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + + // Give the gc pruning logic a chance to kick in + Thread.sleep(1000); + + if (randomBoolean()) { + engine.refresh("test"); + } + + // Delete non-existent document + engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); + + // Get should not find the document (we never indexed uid=2): + getResult = engine.get(new Engine.Get(true, newUid("2"))); + assertThat(getResult.exists(), equalTo(false)); + + // Try to index uid=1 with a too-old version, should fail: + try { + engine.index(new Engine.Index(null, newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + fail("did not hit expected exception"); + } catch (VersionConflictEngineException vcee) { + // expected + } + + // Get should still not find the document + getResult = engine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + + // Try to index uid=2 with a too-old version, should fail: + try { + engine.index(new Engine.Index(null, newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); + fail("did not hit expected exception"); + } catch (VersionConflictEngineException vcee) { + // expected + } + + // Get should not find the document + getResult = engine.get(new Engine.Get(true, newUid("2"))); + assertThat(getResult.exists(), equalTo(false)); + engine.close(); + store.close(); + } + + protected Term newUid(String id) { + return new Term("_uid", id); + } + + @Test + public void testExtractShardId() { + try (Engine.Searcher test = this.engine.acquireSearcher("test")) { + ShardId shardId = ShardUtils.extractShardId(test.reader()); + assertNotNull(shardId); + assertEquals(shardId, ((ShadowEngine) engine).config().getShardId()); + } + } + + *//** + * Random test that throws random exception and ensures all references are + * counted down / released and resources are closed. + *//* + @Test + public void testFailStart() throws IOException { + // this test fails if any reader, searcher or directory is not closed - MDW FTW + final int iters = scaledRandomIntBetween(10, 100); + for (int i = 0; i < iters; i++) { + MockDirectoryWrapper wrapper = newMockDirectory(); + wrapper.setFailOnOpenInput(randomBoolean()); + wrapper.setAllowRandomFileNotFoundException(randomBoolean()); + wrapper.setRandomIOExceptionRate(randomDouble()); + wrapper.setRandomIOExceptionRateOnOpen(randomDouble()); + try (Store store = createStore(wrapper)) { + int refCount = store.refCount(); + assertTrue("refCount: "+ store.refCount(), store.refCount() > 0); + Translog translog = createTranslog(); + ShadowEngine holder; + try { + holder = createEngine(store, translog); + } catch (EngineCreationFailureException ex) { + assertEquals(store.refCount(), refCount); + continue; + } + holder.config().setFailEngineOnCorruption(true); + assertEquals(store.refCount(), refCount+1); + final int numStarts = scaledRandomIntBetween(1, 5); + for (int j = 0; j < numStarts; j++) { + try { + assertEquals(store.refCount(), refCount + 1); + holder.close(); + holder = createEngine(store, translog); + holder.config().setFailEngineOnCorruption(true); + assertEquals(store.refCount(), refCount + 1); + } catch (EngineCreationFailureException ex) { + // all is fine + assertEquals(store.refCount(), refCount); + break; + } + } + translog.close(); + holder.close(); + assertEquals(store.refCount(), refCount); + } + } + } + + @Test + public void testSettings() { + ShadowEngine engine = (ShadowEngine) this.engine; + CodecService codecService = new CodecService(shardId.index()); + LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); + + assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName()); + assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName()); + assertEquals(engine.config().getIndexConcurrency(), indexConcurrency); + assertEquals(currentIndexWriterConfig.getMaxThreadStates(), indexConcurrency); + + + IndexDynamicSettingsModule settings = new IndexDynamicSettingsModule(); + assertTrue(settings.containsSetting(EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING)); + assertTrue(settings.containsSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH)); + assertTrue(settings.containsSetting(EngineConfig.INDEX_GC_DELETES_SETTING)); + assertTrue(settings.containsSetting(EngineConfig.INDEX_FAIL_ON_MERGE_FAILURE_SETTING)); + } + + @Test + public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { + + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + boolean canHaveDuplicates = false; + boolean autoGeneratedId = true; + + Engine.Create index = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + engine.create(index); + assertThat(index.version(), equalTo(1l)); + + index = new Engine.Create(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + replicaEngine.create(index); + assertThat(index.version(), equalTo(1l)); + + canHaveDuplicates = true; + index = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + engine.create(index); + assertThat(index.version(), equalTo(1l)); + engine.refresh("test"); + Engine.Searcher searcher = engine.acquireSearcher("test"); + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); + assertThat(topDocs.totalHits, equalTo(1)); + + index = new Engine.Create(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + try { + replicaEngine.create(index); + fail(); + } catch (VersionConflictEngineException e) { + // we ignore version conflicts on replicas, see TransportShardReplicationOperationAction.ignoreReplicaException + } + replicaEngine.refresh("test"); + Engine.Searcher replicaSearcher = replicaEngine.acquireSearcher("test"); + topDocs = replicaSearcher.searcher().search(new MatchAllDocsQuery(), 10); + assertThat(topDocs.totalHits, equalTo(1)); + searcher.close(); + replicaSearcher.close(); + } + + @Test + public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() throws IOException { + + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); + boolean canHaveDuplicates = true; + boolean autoGeneratedId = true; + + Engine.Create firstIndexRequest = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + engine.create(firstIndexRequest); + assertThat(firstIndexRequest.version(), equalTo(1l)); + + Engine.Create firstIndexRequestReplica = new Engine.Create(null, newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + replicaEngine.create(firstIndexRequestReplica); + assertThat(firstIndexRequestReplica.version(), equalTo(1l)); + + canHaveDuplicates = false; + Engine.Create secondIndexRequest = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + try { + engine.create(secondIndexRequest); + fail(); + } catch (DocumentAlreadyExistsException e) { + // we can ignore the exception. In case this happens because the retry request arrived first then this error will not be sent back anyway. + // in any other case this is an actual error + } + engine.refresh("test"); + Engine.Searcher searcher = engine.acquireSearcher("test"); + TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); + assertThat(topDocs.totalHits, equalTo(1)); + + Engine.Create secondIndexRequestReplica = new Engine.Create(null, newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); + try { + replicaEngine.create(secondIndexRequestReplica); + fail(); + } catch (VersionConflictEngineException e) { + // we ignore version conflicts on replicas, see TransportShardReplicationOperationAction.ignoreReplicaException. + } + replicaEngine.refresh("test"); + Engine.Searcher replicaSearcher = replicaEngine.acquireSearcher("test"); + topDocs = replicaSearcher.searcher().search(new MatchAllDocsQuery(), 10); + assertThat(topDocs.totalHits, equalTo(1)); + searcher.close(); + replicaSearcher.close(); + }*/ + +} From 1896feda9de69e4f9cf774ef6748a5c50e953946 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 10:29:12 -0700 Subject: [PATCH 09/56] Add testShadowEngineIgnoresWriteOperations and testSearchResultRelease --- .../index/engine/ShadowEngineTests.java | 119 ++++++++++++++++-- 1 file changed, 106 insertions(+), 13 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 1612bc34b98ec..1b89bad02816f 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -470,6 +470,86 @@ public void testVerboseSegments() throws Exception { } + @Test + public void testShadowEngineIgnoresWriteOperations() throws Exception { + // create a document + ParseContext.Document document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + replicaEngine.create(new Engine.Create(null, newUid("1"), doc)); + replicaEngine.refresh("test"); + + // its not there... + Engine.Searcher searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + Engine.GetResult getResult = replicaEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + getResult.release(); + + // index a document + document = testDocument(); + document.add(new TextField("value", "test1", Field.Store.YES)); + doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + replicaEngine.index(new Engine.Index(null, newUid("1"), doc)); + replicaEngine.refresh("test"); + + // its still not there... + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + getResult = replicaEngine.get(new Engine.Get(true, newUid("1"))); + assertThat(getResult.exists(), equalTo(false)); + getResult.release(); + + // Now, add a document to the primary so we can test shadow engine deletes + document = testDocumentWithTextField(); + document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); + doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + primaryEngine.flush(); + replicaEngine.refresh("test"); + + // Now the replica can see it + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + + // And the replica can retrieve it + getResult = replicaEngine.get(new Engine.Get(false, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + + // try to delete it on the replica + replicaEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + replicaEngine.flush(); + replicaEngine.refresh("test"); + primaryEngine.refresh("test"); + + // it's still there! + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + getResult = replicaEngine.get(new Engine.Get(false, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + + // it's still there on the primary also! + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + getResult = primaryEngine.get(new Engine.Get(false, newUid("1"))); + assertThat(getResult.exists(), equalTo(true)); + assertThat(getResult.docIdAndVersion(), notNullValue()); + getResult.release(); + } /* @Test public void testSegmentsWithMergeFlag() throws Exception { @@ -785,45 +865,58 @@ public void testSimpleOperations() throws Exception { searchResult.close(); } -/* @Test + @Test public void testSearchResultRelease() throws Exception { - Engine.Searcher searchResult = engine.acquireSearcher("test"); + Engine.Searcher searchResult = replicaEngine.acquireSearcher("test"); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); searchResult.close(); // create a document ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc)); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); // its not there... - searchResult = engine.acquireSearcher("test"); + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); + searchResult.close(); + searchResult = replicaEngine.acquireSearcher("test"); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 0)); searchResult.close(); - // refresh and it should be there - engine.refresh("test"); + // flush & refresh and it should everywhere + primaryEngine.flush(); + primaryEngine.refresh("test"); + replicaEngine.refresh("test"); // now its there... - searchResult = engine.acquireSearcher("test"); + searchResult = primaryEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + + searchResult = replicaEngine.acquireSearcher("test"); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); - // don't release the search result yet... + // don't release the replica search result yet... // delete, refresh and do a new search, it should not be there - engine.delete(new Engine.Delete("test", "1", newUid("1"))); - engine.refresh("test"); - Engine.Searcher updateSearchResult = engine.acquireSearcher("test"); + primaryEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + primaryEngine.flush(); + primaryEngine.refresh("test"); + replicaEngine.refresh("test"); + Engine.Searcher updateSearchResult = primaryEngine.acquireSearcher("test"); MatcherAssert.assertThat(updateSearchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(0)); updateSearchResult.close(); - // the non release search result should not see the deleted yet... + // the non released replica search result should not see the deleted yet... MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); searchResult.close(); } - @Test +/* @Test public void testFailEngineOnCorruption() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); engine.create(new Engine.Create(null, newUid("1"), doc)); From a95adbeded426d7f69f6ddc4cbd6712b6f6380b4 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 12:54:14 -0700 Subject: [PATCH 10/56] Remove tests that don't apply to ShadowEngine --- .../index/engine/ShadowEngineTests.java | 976 +----------------- 1 file changed, 17 insertions(+), 959 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 1b89bad02816f..6a550870300e2 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -30,6 +30,7 @@ import org.apache.lucene.index.Term; import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.MockDirectoryWrapper; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Nullable; @@ -53,6 +54,7 @@ import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.distributor.LeastUsedDistributor; @@ -64,6 +66,7 @@ import org.hamcrest.MatcherAssert; import org.junit.After; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.io.IOException; @@ -72,8 +75,7 @@ import java.util.Arrays; import java.util.List; -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean; -import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween; +import static com.carrotsearch.randomizedtesting.RandomizedTest.*; import static org.elasticsearch.common.settings.ImmutableSettings.Builder.EMPTY_SETTINGS; import static org.elasticsearch.test.ElasticsearchTestCase.newTempDirPath; import static org.elasticsearch.test.ElasticsearchTestCase.terminate; @@ -139,7 +141,7 @@ public void setUp() throws Exception { } replicaTranslog = createTranslogReplica(); - replicaEngine = createEngine(storeReplica, replicaTranslog); + replicaEngine = createShadowEngine(storeReplica, replicaTranslog); assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); if (randomBoolean()) { @@ -220,7 +222,7 @@ protected MergeSchedulerProvider createMergeScheduler(IndexSettingsService index return new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, indexSettingsService); } - protected ShadowEngine createEngine(Store store, Translog translog) { + protected ShadowEngine createShadowEngine(Store store, Translog translog) { IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); return createShadowEngine(indexSettingsService, store, translog, createMergeScheduler(indexSettingsService)); } @@ -551,107 +553,6 @@ public void testShadowEngineIgnoresWriteOperations() throws Exception { getResult.release(); } -/* @Test - public void testSegmentsWithMergeFlag() throws Exception { - final Store store = createStore(); - ConcurrentMergeSchedulerProvider mergeSchedulerProvider = new ConcurrentMergeSchedulerProvider(shardId, EMPTY_SETTINGS, threadPool, new IndexSettingsService(shardId.index(), EMPTY_SETTINGS)); - final AtomicReference waitTillMerge = new AtomicReference<>(); - final AtomicReference waitForMerge = new AtomicReference<>(); - mergeSchedulerProvider.addListener(new MergeSchedulerProvider.Listener() { - @Override - public void beforeMerge(OnGoingMerge merge) { - try { - if (waitTillMerge.get() != null) { - waitTillMerge.get().countDown(); - } - if (waitForMerge.get() != null) { - waitForMerge.get().await(); - } - } catch (InterruptedException e) { - throw ExceptionsHelper.convertToRuntime(e); - } - } - - @Override - public void afterMerge(OnGoingMerge merge) { - } - }); - - IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); - final Engine engine = createShadowEngine(indexSettingsService, store, createTranslog(), mergeSchedulerProvider); - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - engine.flush(); - assertThat(engine.segments(false).size(), equalTo(1)); - index = new Engine.Index(null, newUid("2"), doc); - engine.index(index); - engine.flush(); - List segments = engine.segments(false); - assertThat(segments.size(), equalTo(2)); - for (Segment segment : segments) { - assertThat(segment.getMergeId(), nullValue()); - } - index = new Engine.Index(null, newUid("3"), doc); - engine.index(index); - engine.flush(); - segments = engine.segments(false); - assertThat(segments.size(), equalTo(3)); - for (Segment segment : segments) { - assertThat(segment.getMergeId(), nullValue()); - } - - waitTillMerge.set(new CountDownLatch(1)); - waitForMerge.set(new CountDownLatch(1)); - engine.forceMerge(false, false); - waitTillMerge.get().await(); - - for (Segment segment : engine.segments(false)) { - assertThat(segment.getMergeId(), notNullValue()); - } - - waitForMerge.get().countDown(); - - index = new Engine.Index(null, newUid("4"), doc); - engine.index(index); - engine.flush(); - final long gen1 = store.readLastCommittedSegmentsInfo().getGeneration(); - // now, optimize and wait for merges, see that we have no merge flag - engine.forceMerge(true, true); - - for (Segment segment : engine.segments(false)) { - assertThat(segment.getMergeId(), nullValue()); - } - // we could have multiple underlying merges, so the generation may increase more than once - assertTrue(store.readLastCommittedSegmentsInfo().getGeneration() > gen1); - - final boolean flush = randomBoolean(); - final long gen2 = store.readLastCommittedSegmentsInfo().getGeneration(); - engine.forceMerge(flush, false); - waitTillMerge.get().await(); - for (Segment segment : engine.segments(false)) { - assertThat(segment.getMergeId(), nullValue()); - } - waitForMerge.get().countDown(); - - if (flush) { - awaitBusy(new Predicate() { - @Override - public boolean apply(Object o) { - try { - // we should have had just 1 merge, so last generation should be exact - return store.readLastCommittedSegmentsInfo().getLastGeneration() == gen2; - } catch (IOException e) { - throw ExceptionsHelper.convertToRuntime(e); - } - } - }); - } - - engine.close(); - store.close(); - } */ - @Test public void testSimpleOperations() throws Exception { Engine.Searcher searchResult = primaryEngine.acquireSearcher("test"); @@ -916,773 +817,26 @@ public void testSearchResultRelease() throws Exception { searchResult.close(); } -/* @Test - public void testFailEngineOnCorruption() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc)); - engine.flush(); - final boolean failEngine = defaultSettings.getAsBoolean(EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, false); - final int failInPhase = randomIntBetween(1, 3); - try { - engine.recover(new Engine.RecoveryHandler() { - @Override - public void phase1(SnapshotIndexCommit snapshot) throws EngineException { - if (failInPhase == 1) { - throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); - } - } - - @Override - public void phase2(Translog.Snapshot snapshot) throws EngineException { - if (failInPhase == 2) { - throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); - } - } - - @Override - public void phase3(Translog.Snapshot snapshot) throws EngineException { - if (failInPhase == 3) { - throw new RuntimeException("bar", new CorruptIndexException("Foo", "fake file description")); - } - } - }); - fail("exception expected"); - } catch (RuntimeException ex) { - - } - try { - Engine.Searcher searchResult = engine.acquireSearcher("test"); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); - searchResult.close(); - - ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); - engine.create(new Engine.Create(null, newUid("2"), doc2)); - engine.refresh("foo"); - - searchResult = engine.acquireSearcher("test"); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 2)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(2)); - searchResult.close(); - assertThat(failEngine, is(false)); - } catch (EngineClosedException ex) { - assertThat(failEngine, is(true)); - } - } - - - @Test - public void testSimpleRecover() throws Exception { - final ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc)); - engine.flush(); - - engine.recover(new Engine.RecoveryHandler() { - @Override - public void phase1(SnapshotIndexCommit snapshot) throws EngineException { - try { - engine.flush(); - assertThat("flush is not allowed in phase 1", false, equalTo(true)); - } catch (FlushNotAllowedEngineException e) { - // all is well - } - } - - @Override - public void phase2(Translog.Snapshot snapshot) throws EngineException { - MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); - try { - engine.flush(); - assertThat("flush is not allowed in phase 2", false, equalTo(true)); - } catch (FlushNotAllowedEngineException e) { - // all is well - } - - // but we can index - engine.index(new Engine.Index(null, newUid("1"), doc)); - } - - @Override - public void phase3(Translog.Snapshot snapshot) throws EngineException { - MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(1)); - try { - // we can do this here since we are on the same thread - engine.flush(); - assertThat("flush is not allowed in phase 3", false, equalTo(true)); - } catch (FlushNotAllowedEngineException e) { - // all is well - } - } - }); - // post recovery should flush the translog - MatcherAssert.assertThat(translog.snapshot(), TranslogSizeMatcher.translogSize(0)); - - engine.flush(); - engine.close(); - } - @Test - public void testRecoverWithOperationsBetweenPhase1AndPhase2() throws Exception { - ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc1)); - engine.flush(); - ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); - engine.create(new Engine.Create(null, newUid("2"), doc2)); - - engine.recover(new Engine.RecoveryHandler() { - @Override - public void phase1(SnapshotIndexCommit snapshot) throws EngineException { - } - - @Override - public void phase2(Translog.Snapshot snapshot) throws EngineException { - Translog.Create create = (Translog.Create) snapshot.next(); - assertThat("translog snapshot should not read null", create != null, equalTo(true)); - assertThat(create.source().toBytesArray(), equalTo(B_2)); - assertThat(snapshot.next(), equalTo(null)); - } - - @Override - public void phase3(Translog.Snapshot snapshot) throws EngineException { - MatcherAssert.assertThat(snapshot, TranslogSizeMatcher.translogSize(0)); - } - }); - - engine.flush(); - engine.close(); - } - - @Test - public void testRecoverWithOperationsBetweenPhase1AndPhase2AndPhase3() throws Exception { - ParsedDocument doc1 = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc1)); - engine.flush(); - ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); - engine.create(new Engine.Create(null, newUid("2"), doc2)); - - engine.recover(new Engine.RecoveryHandler() { - @Override - public void phase1(SnapshotIndexCommit snapshot) throws EngineException { - } - - @Override - public void phase2(Translog.Snapshot snapshot) throws EngineException { - Translog.Create create = (Translog.Create) snapshot.next(); - assertThat(create != null, equalTo(true)); - assertThat(snapshot.next(), equalTo(null)); - assertThat(create.source().toBytesArray(), equalTo(B_2)); - - // add for phase3 - ParsedDocument doc3 = testParsedDocument("3", "3", "test", null, -1, -1, testDocumentWithTextField(), B_3, false); - engine.create(new Engine.Create(null, newUid("3"), doc3)); - } - - @Override - public void phase3(Translog.Snapshot snapshot) throws EngineException { - Translog.Create create = (Translog.Create) snapshot.next(); - assertThat(create != null, equalTo(true)); - assertThat(snapshot.next(), equalTo(null)); - assertThat(create.source().toBytesArray(), equalTo(B_3)); - } - }); - - engine.flush(); - engine.close(); - } - - @Test - public void testVersioningNewCreate() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Create create = new Engine.Create(null, newUid("1"), doc); - engine.create(create); - assertThat(create.version(), equalTo(1l)); - - create = new Engine.Create(null, newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.create(create); - assertThat(create.version(), equalTo(1l)); - } - - @Test - public void testExternalVersioningNewCreate() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Create create = new Engine.Create(null, newUid("1"), doc, 12, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, 0); - engine.create(create); - assertThat(create.version(), equalTo(12l)); - - create = new Engine.Create(null, newUid("1"), doc, create.version(), create.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.create(create); - assertThat(create.version(), equalTo(12l)); - } - - @Test - public void testVersioningNewIndex() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - assertThat(index.version(), equalTo(1l)); - } - - @Test - public void testExternalVersioningNewIndex() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); - engine.index(index); - assertThat(index.version(), equalTo(12l)); - - index = new Engine.Index(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - assertThat(index.version(), equalTo(12l)); - } - - @Test - public void testVersioningIndexConflict() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL, Engine.Operation.Origin.PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // future versions should not work as well - index = new Engine.Index(null, newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testExternalVersioningIndexConflict() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); - engine.index(index); - assertThat(index.version(), equalTo(12l)); - - index = new Engine.Index(null, newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); - engine.index(index); - assertThat(index.version(), equalTo(14l)); - - index = new Engine.Index(null, newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testVersioningIndexConflictWithFlush() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - engine.flush(); - - index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // future versions should not work as well - index = new Engine.Index(null, newUid("1"), doc, 3l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testExternalVersioningIndexConflictWithFlush() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc, 12, VersionType.EXTERNAL, PRIMARY, 0); - engine.index(index); - assertThat(index.version(), equalTo(12l)); - - index = new Engine.Index(null, newUid("1"), doc, 14, VersionType.EXTERNAL, PRIMARY, 0); - engine.index(index); - assertThat(index.version(), equalTo(14l)); - - engine.flush(); - - index = new Engine.Index(null, newUid("1"), doc, 13, VersionType.EXTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testVersioningDeleteConflict() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertThat(delete.version(), equalTo(3l)); - - // now check if we can index to a delete doc with version - index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // we shouldn't be able to create as well - Engine.Create create = new Engine.Create(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.create(create); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testVersioningDeleteConflictWithFlush() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - engine.flush(); - - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1"), 1l, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // future versions should not work as well - delete = new Engine.Delete("test", "1", newUid("1"), 3l, VersionType.INTERNAL, PRIMARY, 0, false); - try { - engine.delete(delete); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - engine.flush(); - - // now actually delete - delete = new Engine.Delete("test", "1", newUid("1"), 2l, VersionType.INTERNAL, PRIMARY, 0, false); - engine.delete(delete); - assertThat(delete.version(), equalTo(3l)); - - engine.flush(); - - // now check if we can index to a delete doc with version - index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // we shouldn't be able to create as well - Engine.Create create = new Engine.Create(null, newUid("1"), doc, 2l, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.create(create); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testVersioningCreateExistsException() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Create create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); - engine.create(create); - assertThat(create.version(), equalTo(1l)); - - create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.create(create); - fail(); - } catch (DocumentAlreadyExistsException e) { - // all is well - } - } - - @Test - public void testVersioningCreateExistsExceptionWithFlush() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Create create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); - engine.create(create); - assertThat(create.version(), equalTo(1l)); - - engine.flush(); - - create = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, 0); - try { - engine.create(create); - fail(); - } catch (DocumentAlreadyExistsException e) { - // all is well - } - } - - @Test - public void testVersioningReplicaConflict1() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - // apply the second index to the replica, should work fine - index = new Engine.Index(null, newUid("1"), doc, index.version(), VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - assertThat(index.version(), equalTo(2l)); - - // now, the old one should not work - index = new Engine.Index(null, newUid("1"), doc, 1l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); - try { - replicaEngine.index(index); - fail(); - } catch (VersionConflictEngineException e) { - // all is well - } - - // second version on replica should fail as well - try { - index = new Engine.Index(null, newUid("1"), doc, 2l - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - assertThat(index.version(), equalTo(2l)); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - @Test - public void testVersioningReplicaConflict2() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(1l)); - - // apply the first index to the replica, should work fine - index = new Engine.Index(null, newUid("1"), doc, 1l - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - assertThat(index.version(), equalTo(1l)); - - // index it again - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertThat(index.version(), equalTo(2l)); - - // now delete it - Engine.Delete delete = new Engine.Delete("test", "1", newUid("1")); - engine.delete(delete); - assertThat(delete.version(), equalTo(3l)); - - // apply the delete on the replica (skipping the second index) - delete = new Engine.Delete("test", "1", newUid("1"), 3l - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); - replicaEngine.delete(delete); - assertThat(delete.version(), equalTo(3l)); - - // second time delete with same version should fail - try { - delete = new Engine.Delete("test", "1", newUid("1"), 3l - , VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0, false); - replicaEngine.delete(delete); - fail("excepted VersionConflictEngineException to be thrown"); - } catch (VersionConflictEngineException e) { - // all is well - } - - // now do the second index on the replica, it should fail - try { - index = new Engine.Index(null, newUid("1"), doc, 2l, VersionType.INTERNAL.versionTypeForReplicationAndRecovery(), REPLICA, 0); - replicaEngine.index(index); - fail("excepted VersionConflictEngineException to be thrown"); - } catch (VersionConflictEngineException e) { - // all is well - } - } - - - @Test - public void testBasicCreatedFlag() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertTrue(index.created()); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertFalse(index.created()); - - engine.delete(new Engine.Delete(null, "1", newUid("1"))); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertTrue(index.created()); - } - - @Test - public void testCreatedFlagAfterFlush() { - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - Engine.Index index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertTrue(index.created()); - - engine.delete(new Engine.Delete(null, "1", newUid("1"))); - - engine.flush(); - - index = new Engine.Index(null, newUid("1"), doc); - engine.index(index); - assertTrue(index.created()); - } - - private static class MockAppender extends AppenderSkeleton { - public boolean sawIndexWriterMessage; - - public boolean sawIndexWriterIFDMessage; - - @Override - protected void append(LoggingEvent event) { - if (event.getLevel() == Level.TRACE && event.getMessage().toString().contains("[index][1] ")) { - if (event.getLoggerName().endsWith("lucene.iw") && - event.getMessage().toString().contains("IW: apply all deletes during flush")) { - sawIndexWriterMessage = true; - } - if (event.getLoggerName().endsWith("lucene.iw.ifd")) { - sawIndexWriterIFDMessage = true; - } - } - } - - @Override - public boolean requiresLayout() { - return false; - } - - @Override - public void close() { - } - } - - // #5891: make sure IndexWriter's infoStream output is - // sent to lucene.iw with log level TRACE: - - @Test - public void testIndexWriterInfoStream() { - MockAppender mockAppender = new MockAppender(); - - Logger rootLogger = Logger.getRootLogger(); - Level savedLevel = rootLogger.getLevel(); - rootLogger.addAppender(mockAppender); - rootLogger.setLevel(Level.DEBUG); - - try { - // First, with DEBUG, which should NOT log IndexWriter output: - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc)); - engine.flush(); - assertFalse(mockAppender.sawIndexWriterMessage); - - // Again, with TRACE, which should log IndexWriter output: - rootLogger.setLevel(Level.TRACE); - engine.create(new Engine.Create(null, newUid("2"), doc)); - engine.flush(); - assertTrue(mockAppender.sawIndexWriterMessage); - - } finally { - rootLogger.removeAppender(mockAppender); - rootLogger.setLevel(savedLevel); - } - } - - // #8603: make sure we can separately log IFD's messages - public void testIndexWriterIFDInfoStream() { - MockAppender mockAppender = new MockAppender(); - - // Works when running this test inside Intellij: - Logger iwIFDLogger = LogManager.exists("org.elasticsearch.index.engine.lucene.iw.ifd"); - if (iwIFDLogger == null) { - // Works when running this test from command line: - iwIFDLogger = LogManager.exists("index.engine.lucene.iw.ifd"); - assertNotNull(iwIFDLogger); - } - - Level savedLevel = iwIFDLogger.getLevel(); - iwIFDLogger.addAppender(mockAppender); - iwIFDLogger.setLevel(Level.DEBUG); - - try { - // First, with DEBUG, which should NOT log IndexWriter output: - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); - engine.create(new Engine.Create(null, newUid("1"), doc)); - engine.flush(); - assertFalse(mockAppender.sawIndexWriterMessage); - assertFalse(mockAppender.sawIndexWriterIFDMessage); - - // Again, with TRACE, which should only log IndexWriter IFD output: - iwIFDLogger.setLevel(Level.TRACE); - engine.create(new Engine.Create(null, newUid("2"), doc)); - engine.flush(); - assertFalse(mockAppender.sawIndexWriterMessage); - assertTrue(mockAppender.sawIndexWriterIFDMessage); - - } finally { - iwIFDLogger.removeAppender(mockAppender); - iwIFDLogger.setLevel(null); - } - } - - @Slow - @Test - public void testEnableGcDeletes() throws Exception { - - Store store = createStore(); - - - - IndexSettingsService indexSettingsService = new IndexSettingsService(shardId.index(), ImmutableSettings.builder().put(defaultSettings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()); - Engine engine = new ShadowEngine(config(indexSettingsService, store, createTranslog(), createMergeScheduler(indexSettingsService))); - ((ShadowEngine)engine).config().setEnableGcDeletes(false); - - // Add document - ParseContext.Document document = testDocument(); - document.add(new TextField("value", "test1", Field.Store.YES)); - - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_2, false); - engine.index(new Engine.Index(null, newUid("1"), doc, 1, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); - - // Delete document we just added: - engine.delete(new Engine.Delete("test", "1", newUid("1"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); - - // Get should not find the document - Engine.GetResult getResult = engine.get(new Engine.Get(true, newUid("1"))); - assertThat(getResult.exists(), equalTo(false)); - - // Give the gc pruning logic a chance to kick in - Thread.sleep(1000); - - if (randomBoolean()) { - engine.refresh("test"); - } - - // Delete non-existent document - engine.delete(new Engine.Delete("test", "2", newUid("2"), 10, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime(), false)); - - // Get should not find the document (we never indexed uid=2): - getResult = engine.get(new Engine.Get(true, newUid("2"))); - assertThat(getResult.exists(), equalTo(false)); - - // Try to index uid=1 with a too-old version, should fail: - try { - engine.index(new Engine.Index(null, newUid("1"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); - fail("did not hit expected exception"); - } catch (VersionConflictEngineException vcee) { - // expected - } - - // Get should still not find the document - getResult = engine.get(new Engine.Get(true, newUid("1"))); - assertThat(getResult.exists(), equalTo(false)); - - // Try to index uid=2 with a too-old version, should fail: - try { - engine.index(new Engine.Index(null, newUid("2"), doc, 2, VersionType.EXTERNAL, Engine.Operation.Origin.PRIMARY, System.nanoTime())); - fail("did not hit expected exception"); - } catch (VersionConflictEngineException vcee) { - // expected - } - - // Get should not find the document - getResult = engine.get(new Engine.Get(true, newUid("2"))); - assertThat(getResult.exists(), equalTo(false)); - engine.close(); - store.close(); - } - - protected Term newUid(String id) { - return new Term("_uid", id); + public void testFailEngineOnCorruption() { + // nocommit - figure out how to implement me for shadow replica } @Test public void testExtractShardId() { - try (Engine.Searcher test = this.engine.acquireSearcher("test")) { + try (Engine.Searcher test = replicaEngine.acquireSearcher("test")) { ShardId shardId = ShardUtils.extractShardId(test.reader()); assertNotNull(shardId); - assertEquals(shardId, ((ShadowEngine) engine).config().getShardId()); + assertEquals(shardId, replicaEngine.config().getShardId()); } } - *//** + /** * Random test that throws random exception and ensures all references are * counted down / released and resources are closed. - *//* + */ @Test + @Ignore // nocommit - fails with IndexNotFoundException: no segments* file found in store public void testFailStart() throws IOException { // this test fails if any reader, searcher or directory is not closed - MDW FTW final int iters = scaledRandomIntBetween(10, 100); @@ -1698,7 +852,7 @@ public void testFailStart() throws IOException { Translog translog = createTranslog(); ShadowEngine holder; try { - holder = createEngine(store, translog); + holder = createShadowEngine(store, translog); } catch (EngineCreationFailureException ex) { assertEquals(store.refCount(), refCount); continue; @@ -1710,7 +864,7 @@ public void testFailStart() throws IOException { try { assertEquals(store.refCount(), refCount + 1); holder.close(); - holder = createEngine(store, translog); + holder = createShadowEngine(store, translog); holder.config().setFailEngineOnCorruption(true); assertEquals(store.refCount(), refCount + 1); } catch (EngineCreationFailureException ex) { @@ -1728,104 +882,8 @@ public void testFailStart() throws IOException { @Test public void testSettings() { - ShadowEngine engine = (ShadowEngine) this.engine; CodecService codecService = new CodecService(shardId.index()); - LiveIndexWriterConfig currentIndexWriterConfig = engine.getCurrentIndexWriterConfig(); - - assertEquals(engine.config().getCodec().getName(), codecService.codec(codecName).getName()); - assertEquals(currentIndexWriterConfig.getCodec().getName(), codecService.codec(codecName).getName()); - assertEquals(engine.config().getIndexConcurrency(), indexConcurrency); - assertEquals(currentIndexWriterConfig.getMaxThreadStates(), indexConcurrency); - - - IndexDynamicSettingsModule settings = new IndexDynamicSettingsModule(); - assertTrue(settings.containsSetting(EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING)); - assertTrue(settings.containsSetting(EngineConfig.INDEX_COMPOUND_ON_FLUSH)); - assertTrue(settings.containsSetting(EngineConfig.INDEX_GC_DELETES_SETTING)); - assertTrue(settings.containsSetting(EngineConfig.INDEX_FAIL_ON_MERGE_FAILURE_SETTING)); - } - - @Test - public void testRetryWithAutogeneratedIdWorksAndNoDuplicateDocs() throws IOException { - - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - boolean canHaveDuplicates = false; - boolean autoGeneratedId = true; - - Engine.Create index = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - engine.create(index); - assertThat(index.version(), equalTo(1l)); - - index = new Engine.Create(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - replicaEngine.create(index); - assertThat(index.version(), equalTo(1l)); - - canHaveDuplicates = true; - index = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - engine.create(index); - assertThat(index.version(), equalTo(1l)); - engine.refresh("test"); - Engine.Searcher searcher = engine.acquireSearcher("test"); - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); - assertThat(topDocs.totalHits, equalTo(1)); - - index = new Engine.Create(null, newUid("1"), doc, index.version(), index.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - try { - replicaEngine.create(index); - fail(); - } catch (VersionConflictEngineException e) { - // we ignore version conflicts on replicas, see TransportShardReplicationOperationAction.ignoreReplicaException - } - replicaEngine.refresh("test"); - Engine.Searcher replicaSearcher = replicaEngine.acquireSearcher("test"); - topDocs = replicaSearcher.searcher().search(new MatchAllDocsQuery(), 10); - assertThat(topDocs.totalHits, equalTo(1)); - searcher.close(); - replicaSearcher.close(); + assertEquals(replicaEngine.config().getCodec().getName(), codecService.codec(codecName).getName()); + assertEquals(replicaEngine.config().getIndexConcurrency(), indexConcurrency); } - - @Test - public void testRetryWithAutogeneratedIdsAndWrongOrderWorksAndNoDuplicateDocs() throws IOException { - - ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocument(), B_1, false); - boolean canHaveDuplicates = true; - boolean autoGeneratedId = true; - - Engine.Create firstIndexRequest = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - engine.create(firstIndexRequest); - assertThat(firstIndexRequest.version(), equalTo(1l)); - - Engine.Create firstIndexRequestReplica = new Engine.Create(null, newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - replicaEngine.create(firstIndexRequestReplica); - assertThat(firstIndexRequestReplica.version(), equalTo(1l)); - - canHaveDuplicates = false; - Engine.Create secondIndexRequest = new Engine.Create(null, newUid("1"), doc, Versions.MATCH_ANY, VersionType.INTERNAL, PRIMARY, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - try { - engine.create(secondIndexRequest); - fail(); - } catch (DocumentAlreadyExistsException e) { - // we can ignore the exception. In case this happens because the retry request arrived first then this error will not be sent back anyway. - // in any other case this is an actual error - } - engine.refresh("test"); - Engine.Searcher searcher = engine.acquireSearcher("test"); - TopDocs topDocs = searcher.searcher().search(new MatchAllDocsQuery(), 10); - assertThat(topDocs.totalHits, equalTo(1)); - - Engine.Create secondIndexRequestReplica = new Engine.Create(null, newUid("1"), doc, firstIndexRequest.version(), firstIndexRequest.versionType().versionTypeForReplicationAndRecovery(), REPLICA, System.nanoTime(), canHaveDuplicates, autoGeneratedId); - try { - replicaEngine.create(secondIndexRequestReplica); - fail(); - } catch (VersionConflictEngineException e) { - // we ignore version conflicts on replicas, see TransportShardReplicationOperationAction.ignoreReplicaException. - } - replicaEngine.refresh("test"); - Engine.Searcher replicaSearcher = replicaEngine.acquireSearcher("test"); - topDocs = replicaSearcher.searcher().search(new MatchAllDocsQuery(), 10); - assertThat(topDocs.totalHits, equalTo(1)); - searcher.close(); - replicaSearcher.close(); - }*/ - } From 52e9cd1b8334a5dd228d5d68bd03fd0040e9c8e9 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 13:45:05 -0700 Subject: [PATCH 11/56] Add a test for replica -> primary promotion --- .../index/IndexWithShadowReplicasTests.java | 49 ++++++++++++++++++- 1 file changed, 48 insertions(+), 1 deletion(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index a377b95b9cd98..aaeb4948213d5 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index; -import com.carrotsearch.randomizedtesting.annotations.Repeat; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -31,6 +30,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Ignore; @@ -115,6 +115,53 @@ public void testIndexWithFewDocuments() throws Exception { assertAcked(client().admin().indices().prepareDelete(IDX)); } + @Test + @Ignore // nocommit - this currently fails because of the way that we promote shadow replicas into primaries + public void testReplicaToPrimaryPromotion() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + .build(); + + String node1 = internalCluster().startNode(nodeSettings); + Path dataPath = newTempDirPath(); + String IDX = "test"; + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + ensureYellow(IDX); + client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + + // Node1 has the primary, now node2 has the replica + String node2 = internalCluster().startNode(nodeSettings); + ensureYellow(IDX); + + flushAndRefresh(IDX); + + logger.info("--> stopping node1 [{}]", node1); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node1)); + + ensureYellow(IDX); + + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + + logger.info("--> performing query"); + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, 2); + + } + @Test @LuceneTestCase.Slow @Ignore // NOCOMMIT for now From 2378fbb917b467e79c0262d7a41c23321bbeb147 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 13:45:44 -0700 Subject: [PATCH 12/56] Fix missing import --- .../org/elasticsearch/index/IndexWithShadowReplicasTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index aaeb4948213d5..d514b84d5ead0 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index; +import com.carrotsearch.randomizedtesting.annotations.Repeat; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; From 5e33eeaca971807b342f9be51a6a566eee005251 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 14:22:59 -0700 Subject: [PATCH 13/56] Remove overly-complex test --- .../index/IndexWithShadowReplicasTests.java | 152 ------------------ 1 file changed, 152 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index d514b84d5ead0..365568bb560b5 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -20,29 +20,19 @@ package org.elasticsearch.index; import com.carrotsearch.randomizedtesting.annotations.Repeat; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.elasticsearch.test.store.MockFSDirectoryService; import org.junit.Ignore; import org.junit.Test; import java.nio.file.Path; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; -import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -162,146 +152,4 @@ public void testReplicaToPrimaryPromotion() throws Exception { assertHitCount(resp, 2); } - - @Test - @LuceneTestCase.Slow - @Ignore // NOCOMMIT for now - public void testChaosMonkeyWithShadowReplicas() throws Exception { - final int initialNodeCount = scaledRandomIntBetween(3, 8); - - Settings nodeSettings = ImmutableSettings.builder() - .put("node.add_id_to_custom_path", false) - .put("node.enable_custom_paths", true) - // We don't want closing a node to take forever, so disable this - .put(MockFSDirectoryService.CHECK_INDEX_ON_CLOSE, false) - .put("discovery.zen.minimum_master_nodes", (initialNodeCount / 2) + 1) - .build(); - - logger.info("--> starting up {} nodes...", initialNodeCount); - final List nodes = internalCluster().startNodesAsync(initialNodeCount, nodeSettings).get(); - - // Start up a client node - Settings clientNodeSettings = ImmutableSettings.builder() - .put("node.client", true) - .build(); - String clientNode = internalCluster().startNode(clientNodeSettings); - final Client client = client(clientNode); - - final String IDX = "test"; - final String NORMAL = "normal"; // the normal index setting - final Path dataPath = newTempDirPath(); - - Settings idxSettings = ImmutableSettings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, initialNodeCount)) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, initialNodeCount - 1)) - .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) - .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) - .build(); - - Settings normalSettings = ImmutableSettings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, initialNodeCount)) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, initialNodeCount - 1)) - .build(); - - prepareCreate(IDX).setSettings(idxSettings).get(); - prepareCreate(NORMAL).setSettings(normalSettings).get(); - ensureGreen(TimeValue.timeValueMinutes(1), IDX, NORMAL); - - // Either we're going to fiddle with replicas, or we're going to - // restart nodes, but not both - final boolean fiddleWithReplicas = randomBoolean(); - - // Flag to signle to the threads to stop doing things - final AtomicBoolean running = new AtomicBoolean(true); - - final Runnable chaosRunner = new Runnable() { - @Override - public void run() { - try { - while (running.get()) { - Thread.sleep(randomIntBetween(4000, 9000)); - // Randomly either restart nodes and change replica count - if (fiddleWithReplicas) { - int newCount = randomIntBetween(1, initialNodeCount - 1); - logger.info("--> changing replica count to {}", newCount); - assertAcked(client.admin().indices().prepareUpdateSettings(IDX) - .setSettings(ImmutableSettings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, - newCount))); - } else { - logger.info("--> randomly restarting a data node"); - internalCluster().restartRandomDataNode(); - } - } - } catch (Throwable t) { - logger.warn("exception in chaos monkey", t); - } - } - }; - - final AtomicLong totalDocs = new AtomicLong(); - final Runnable indexRunner = new Runnable() { - @Override - public void run() { - while (running.get()) { - List indexRequests = newArrayList(); - int batchSize = scaledRandomIntBetween(10, 100); - for (int i = 0; i < batchSize; i++) { - indexRequests.add(client.prepareIndex() - .setIndex(randomBoolean() ? IDX : NORMAL) // Randomly use different index - .setType("doc") - .setSource("body", randomRealisticUnicodeOfCodepointLengthBetween(10, 20)) - ); - } - try { - logger.info("--> indexing batch of {} documents...", batchSize); - indexRandom(true, true, true, indexRequests); - totalDocs.addAndGet(batchSize); - if (randomBoolean()) { - // Randomly flush the shadow index - flush(IDX); - } - } catch (Throwable t) { - logger.info("--> wasn't able to index that batch, we'll get 'em next time", t); - } - } - } - }; - - Thread chaosMonkey = new Thread(chaosRunner); - Thread indexingThread = new Thread(indexRunner); - - logger.info("--> starting indexing thread and chaos monkey"); - indexingThread.start(); - chaosMonkey.start(); - - try { - // Give it 30 seconds to index and restart nodes randomly - Thread.sleep(30 * 1000); - - logger.info("--> stopping indexing and chaos thread..."); - running.getAndSet(false); - chaosMonkey.join(15 * 1000); - indexingThread.join(15 * 1000); - } catch (Throwable t) { - fail("failed to go to sleep"); - } - - logger.info("--> waiting for cluster to recover"); - client.admin().cluster().prepareHealth().setWaitForNodes(initialNodeCount + 1 + "").get(); - logger.info("--> waiting for green"); - ensureGreen(IDX, NORMAL); - - logger.info("--> flushing indices..."); - flush(IDX, NORMAL); - logger.info("--> refreshing indices..."); - refresh(); - - logger.info("--> expecting [{}] total documents", totalDocs.longValue()); - - SearchResponse resp = client.prepareSearch(IDX, NORMAL).setSearchType(SearchType.COUNT).setQuery(matchAllQuery()).get(); - assertThat("there should be " + totalDocs.longValue() + " documents that were indexed", - resp.getHits().totalHits(), - equalTo(totalDocs.longValue())); - } } From 80cf0e884c66eda7d59ac5d59235e1ce215af8f5 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 14:30:13 -0700 Subject: [PATCH 14/56] Remove nocommit in ShadowEngineTests#testFailStart() --- .../org/elasticsearch/index/engine/ShadowEngine.java | 3 +-- .../elasticsearch/index/engine/ShadowEngineTests.java | 9 ++++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index ed62c5482cc25..373ee2b461edf 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -28,7 +28,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.Accountables; import org.apache.lucene.util.IOUtils; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; @@ -78,7 +77,7 @@ public ShadowEngine(EngineConfig engineConfig) { } } } catch (IOException ex) { - throw new ElasticsearchException("failed to open index reader", ex); + throw new EngineCreationFailureException(shardId, "failed to open index reader", ex); } } diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 6a550870300e2..0a81076540d10 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -66,7 +66,6 @@ import org.hamcrest.MatcherAssert; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Test; import java.io.IOException; @@ -836,12 +835,16 @@ public void testExtractShardId() { * counted down / released and resources are closed. */ @Test - @Ignore // nocommit - fails with IndexNotFoundException: no segments* file found in store public void testFailStart() throws IOException { + // Need a commit point for this + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + primaryEngine.flush(); + // this test fails if any reader, searcher or directory is not closed - MDW FTW final int iters = scaledRandomIntBetween(10, 100); for (int i = 0; i < iters; i++) { - MockDirectoryWrapper wrapper = newMockDirectory(); + MockDirectoryWrapper wrapper = newMockFSDirectory(dirPath); wrapper.setFailOnOpenInput(randomBoolean()); wrapper.setAllowRandomFileNotFoundException(randomBoolean()); wrapper.setRandomIOExceptionRate(randomDouble()); From e4dbfb09a689b449f0edf6ee24222d7eaba2a215 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 15:08:18 -0700 Subject: [PATCH 15/56] Fix segment info for ShadowEngine, remove test nocommit --- .../elasticsearch/index/engine/Engine.java | 74 ++++++++++++++++- .../index/engine/InternalEngine.java | 67 +--------------- .../index/engine/ShadowEngine.java | 79 ++++++------------- .../index/engine/ShadowEngineTests.java | 14 ++-- 4 files changed, 101 insertions(+), 133 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 444cacd1681a0..9b884ee0ee460 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -26,6 +26,7 @@ import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.join.BitDocIdSetFilter; import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.Accountables; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.common.Nullable; @@ -50,9 +51,7 @@ import java.io.Closeable; import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Condition; @@ -280,6 +279,75 @@ protected void ensureOpen() { */ public abstract SegmentsStats segmentsStats(); + protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { + ensureOpen(); + Map segments = new HashMap<>(); + + // first, go over and compute the search ones... + Searcher searcher = acquireSearcher("segments"); + try { + for (LeafReaderContext reader : searcher.reader().leaves()) { + SegmentCommitInfo info = segmentReader(reader.reader()).getSegmentInfo(); + assert !segments.containsKey(info.info.name); + Segment segment = new Segment(info.info.name); + segment.search = true; + segment.docCount = reader.reader().numDocs(); + segment.delDocCount = reader.reader().numDeletedDocs(); + segment.version = info.info.getVersion(); + segment.compound = info.info.getUseCompoundFile(); + try { + segment.sizeInBytes = info.sizeInBytes(); + } catch (IOException e) { + logger.trace("failed to get size for [{}]", e, info.info.name); + } + final SegmentReader segmentReader = segmentReader(reader.reader()); + segment.memoryInBytes = segmentReader.ramBytesUsed(); + if (verbose) { + segment.ramTree = Accountables.namedAccountable("root", segmentReader); + } + // TODO: add more fine grained mem stats values to per segment info here + segments.put(info.info.name, segment); + } + } finally { + searcher.close(); + } + + // now, correlate or add the committed ones... + if (lastCommittedSegmentInfos != null) { + SegmentInfos infos = lastCommittedSegmentInfos; + for (SegmentCommitInfo info : infos) { + Segment segment = segments.get(info.info.name); + if (segment == null) { + segment = new Segment(info.info.name); + segment.search = false; + segment.committed = true; + segment.docCount = info.info.getDocCount(); + segment.delDocCount = info.getDelCount(); + segment.version = info.info.getVersion(); + segment.compound = info.info.getUseCompoundFile(); + try { + segment.sizeInBytes = info.sizeInBytes(); + } catch (IOException e) { + logger.trace("failed to get size for [{}]", e, info.info.name); + } + segments.put(info.info.name, segment); + } else { + segment.committed = true; + } + } + } + + Segment[] segmentsArr = segments.values().toArray(new Segment[segments.values().size()]); + Arrays.sort(segmentsArr, new Comparator() { + @Override + public int compare(Segment o1, Segment o2) { + return (int) (o1.getGeneration() - o2.getGeneration()); + } + }); + + return segmentsArr; + } + /** * The list of segments in the engine. */ diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index ba8f75316792e..60a5a754e35b0 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -25,7 +25,6 @@ import org.apache.lucene.search.*; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.LockObtainFailedException; -import org.apache.lucene.util.Accountables; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ElasticsearchException; @@ -905,70 +904,7 @@ public SegmentsStats segmentsStats() { @Override public List segments(boolean verbose) { try (ReleasableLock _ = readLock.acquire()) { - ensureOpen(); - Map segments = new HashMap<>(); - - // first, go over and compute the search ones... - Searcher searcher = acquireSearcher("segments"); - try { - for (LeafReaderContext reader : searcher.reader().leaves()) { - SegmentCommitInfo info = segmentReader(reader.reader()).getSegmentInfo(); - assert !segments.containsKey(info.info.name); - Segment segment = new Segment(info.info.name); - segment.search = true; - segment.docCount = reader.reader().numDocs(); - segment.delDocCount = reader.reader().numDeletedDocs(); - segment.version = info.info.getVersion(); - segment.compound = info.info.getUseCompoundFile(); - try { - segment.sizeInBytes = info.sizeInBytes(); - } catch (IOException e) { - logger.trace("failed to get size for [{}]", e, info.info.name); - } - final SegmentReader segmentReader = segmentReader(reader.reader()); - segment.memoryInBytes = segmentReader.ramBytesUsed(); - if (verbose) { - segment.ramTree = Accountables.namedAccountable("root", segmentReader); - } - // TODO: add more fine grained mem stats values to per segment info here - segments.put(info.info.name, segment); - } - } finally { - searcher.close(); - } - - // now, correlate or add the committed ones... - if (lastCommittedSegmentInfos != null) { - SegmentInfos infos = lastCommittedSegmentInfos; - for (SegmentCommitInfo info : infos) { - Segment segment = segments.get(info.info.name); - if (segment == null) { - segment = new Segment(info.info.name); - segment.search = false; - segment.committed = true; - segment.docCount = info.info.getDocCount(); - segment.delDocCount = info.getDelCount(); - segment.version = info.info.getVersion(); - segment.compound = info.info.getUseCompoundFile(); - try { - segment.sizeInBytes = info.sizeInBytes(); - } catch (IOException e) { - logger.trace("failed to get size for [{}]", e, info.info.name); - } - segments.put(info.info.name, segment); - } else { - segment.committed = true; - } - } - } - - Segment[] segmentsArr = segments.values().toArray(new Segment[segments.values().size()]); - Arrays.sort(segmentsArr, new Comparator() { - @Override - public int compare(Segment o1, Segment o2) { - return (int) (o1.getGeneration() - o2.getGeneration()); - } - }); + Segment[] segmentsArr = getSegmentInfo(lastCommittedSegmentInfos, verbose); // fill in the merges flag Set onGoingMerges = mergeScheduler.onGoingMerges(); @@ -982,7 +918,6 @@ public int compare(Segment o1, Segment o2) { } } } - return Arrays.asList(segmentsArr); } } diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 373ee2b461edf..44b697dbc2f5e 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -21,12 +21,11 @@ import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.LeafReaderContext; -import org.apache.lucene.index.SegmentCommitInfo; +import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentReader; import org.apache.lucene.search.SearcherFactory; import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.util.Accountables; import org.apache.lucene.util.IOUtils; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lease.Releasables; @@ -36,7 +35,8 @@ import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import java.io.IOException; -import java.util.*; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantReadWriteLock; @@ -50,11 +50,13 @@ public class ShadowEngine extends Engine { private final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); private final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); private final Lock failReleasableLock = new ReentrantLock(); - private final RecoveryCounter onGoingRecoveries; + private volatile boolean closedOrFailed = false; private volatile SearcherManager searcherManager; + private SegmentInfos lastCommittedSegmentInfos; + public ShadowEngine(EngineConfig engineConfig) { super(engineConfig); SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); @@ -66,6 +68,7 @@ public ShadowEngine(EngineConfig engineConfig) { try { reader = ElasticsearchDirectoryReader.wrap(DirectoryReader.open(store.directory()), shardId); this.searcherManager = new SearcherManager(reader, searcherFactory); + this.lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); success = true; } catch (Throwable e) { logger.warn("failed to create new reader", e); @@ -114,6 +117,17 @@ public void flush() throws EngineException { @Override public void flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.debug("cowardly refusing to FLUSH"); + // reread the last committed segment infos + try { + lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); + } catch (Throwable e) { + if (isClosed.get() == false) { + logger.warn("failed to read latest segment infos on flush", e); + if (Lucene.isCorruptionException(e)) { + throw new FlushFailedEngineException(shardId, e); + } + } + } } @Override @@ -158,59 +172,12 @@ public SegmentsStats segmentsStats() { @Override public List segments(boolean verbose) { try (ReleasableLock _ = readLock.acquire()) { - ensureOpen(); - Map segments = new HashMap<>(); - - // first, go over and compute the search ones... - Searcher searcher = acquireSearcher("segments"); - try { - for (LeafReaderContext reader : searcher.reader().leaves()) { - SegmentCommitInfo info = segmentReader(reader.reader()).getSegmentInfo(); - assert !segments.containsKey(info.info.name); - Segment segment = new Segment(info.info.name); - segment.search = true; - segment.docCount = reader.reader().numDocs(); - segment.delDocCount = reader.reader().numDeletedDocs(); - segment.version = info.info.getVersion(); - segment.compound = info.info.getUseCompoundFile(); - try { - segment.sizeInBytes = info.sizeInBytes(); - } catch (IOException e) { - logger.trace("failed to get size for [{}]", e, info.info.name); - } - final SegmentReader segmentReader = InternalEngine.segmentReader(reader.reader()); - segment.memoryInBytes = segmentReader.ramBytesUsed(); - if (verbose) { - segment.ramTree = Accountables.namedAccountable("root", segmentReader); - } - // TODO: add more fine grained mem stats values to per segment info here - segments.put(info.info.name, segment); - } - } finally { - searcher.close(); + Segment[] segmentsArr = getSegmentInfo(lastCommittedSegmentInfos, verbose); + for (int i = 0; i < segmentsArr.length; i++) { + // hard code all segments as committed, because they are in + // order for the shadow replica to see them + segmentsArr[i].committed = true; } - - Segment[] segmentsArr = segments.values().toArray(new Segment[segments.values().size()]); - Arrays.sort(segmentsArr, new Comparator() { - @Override - public int compare(Segment o1, Segment o2) { - return (int) (o1.getGeneration() - o2.getGeneration()); - } - }); - // fill in the merges flag - // TODO uncomment me -// Set onGoingMerges = mergeScheduler.onGoingMerges(); -// for (OnGoingMerge onGoingMerge : onGoingMerges) { -// for (SegmentCommitInfo segmentInfoPerCommit : onGoingMerge.getMergedSegments()) { -// for (Segment segment : segmentsArr) { -// if (segment.getName().equals(segmentInfoPerCommit.info.name)) { -// segment.mergeId = onGoingMerge.getId(); -// break; -// } -// } -// } -// } - return Arrays.asList(segmentsArr); } } diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 0a81076540d10..50bc4eb968d5e 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -264,8 +264,6 @@ protected Term newUid(String id) { @Test public void testSegments() throws Exception { - // nocommit - the .isCommitted() check on segments fails here - List segments = primaryEngine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(primaryEngine.segmentsStats().getCount(), equalTo(0l)); @@ -326,7 +324,7 @@ public void testSegments() throws Exception { segments = replicaEngine.segments(false); assertThat(segments.size(), equalTo(1)); assertThat(replicaEngine.segmentsStats().getCount(), equalTo(1l)); - //assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); assertThat(segments.get(0).isSearch(), equalTo(true)); assertThat(segments.get(0).getNumDocs(), equalTo(2)); assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); @@ -372,12 +370,12 @@ public void testSegments() throws Exception { assertThat(replicaEngine.segmentsStats().getNormsMemoryInBytes(), greaterThan(stats.getNormsMemoryInBytes())); assertThat(replicaEngine.segmentsStats().getDocValuesMemoryInBytes(), greaterThan(stats.getDocValuesMemoryInBytes())); assertThat(segments.get(0).getGeneration() < segments.get(1).getGeneration(), equalTo(true)); - //assertThat(segments.get(0).isCommitted(), equalTo(true)); + assertThat(segments.get(0).isCommitted(), equalTo(true)); assertThat(segments.get(0).isSearch(), equalTo(true)); assertThat(segments.get(0).getNumDocs(), equalTo(2)); assertThat(segments.get(0).getDeletedDocs(), equalTo(0)); assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); - assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isCommitted(), equalTo(true)); assertThat(segments.get(1).isSearch(), equalTo(true)); assertThat(segments.get(1).getNumDocs(), equalTo(1)); assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); @@ -395,7 +393,7 @@ public void testSegments() throws Exception { assertThat(segments.get(0).getNumDocs(), equalTo(1)); assertThat(segments.get(0).getDeletedDocs(), equalTo(1)); assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); - //assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isCommitted(), equalTo(true)); assertThat(segments.get(1).isSearch(), equalTo(true)); assertThat(segments.get(1).getNumDocs(), equalTo(1)); assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); @@ -420,13 +418,13 @@ public void testSegments() throws Exception { assertThat(segments.get(0).getDeletedDocs(), equalTo(1)); assertThat(segments.get(0).isCompound(), equalTo(defaultCompound)); - //assertThat(segments.get(1).isCommitted(), equalTo(false)); + assertThat(segments.get(1).isCommitted(), equalTo(true)); assertThat(segments.get(1).isSearch(), equalTo(true)); assertThat(segments.get(1).getNumDocs(), equalTo(1)); assertThat(segments.get(1).getDeletedDocs(), equalTo(0)); assertThat(segments.get(1).isCompound(), equalTo(false)); - //assertThat(segments.get(2).isCommitted(), equalTo(false)); + assertThat(segments.get(2).isCommitted(), equalTo(false)); assertThat(segments.get(2).isSearch(), equalTo(true)); assertThat(segments.get(2).getNumDocs(), equalTo(1)); assertThat(segments.get(2).getDeletedDocs(), equalTo(0)); From 06e2eb4496762130af87ce68a47d360962091697 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 15:21:32 -0700 Subject: [PATCH 16/56] Add a test checking that indices with shadow replicas clean up after themselves --- .../index/IndexWithShadowReplicasTests.java | 39 +++++++++++++++++++ .../indices/IndicesCustomDataPathTests.java | 19 --------- .../test/ElasticsearchIntegrationTest.java | 31 +++++++++++++++ 3 files changed, 70 insertions(+), 19 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 365568bb560b5..e667e1d003c34 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -150,6 +150,45 @@ public void testReplicaToPrimaryPromotion() throws Exception { logger.info("--> performing query"); SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); assertHitCount(resp, 2); + } + + @Test + public void testIndexWithShadowReplicasCleansUp() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + .build(); + + int nodeCount = randomIntBetween(2, 5); + internalCluster().startNodesAsync(nodeCount, nodeSettings); + Path dataPath = newTempDirPath(); + String IDX = "test"; + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, nodeCount - 1)) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + ensureYellow(IDX); + client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + flushAndRefresh(IDX); + + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + + logger.info("--> performing query"); + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, 2); + + assertAcked(client().admin().indices().prepareDelete(IDX)); + assertPathHasBeenCleared(dataPath); } } diff --git a/src/test/java/org/elasticsearch/indices/IndicesCustomDataPathTests.java b/src/test/java/org/elasticsearch/indices/IndicesCustomDataPathTests.java index 9e981d6ce5238..8bf52cbce7068 100644 --- a/src/test/java/org/elasticsearch/indices/IndicesCustomDataPathTests.java +++ b/src/test/java/org/elasticsearch/indices/IndicesCustomDataPathTests.java @@ -137,23 +137,4 @@ public void testIndexCreatedWithCustomPathAndTemplate() throws Exception { assertAcked(client().admin().indices().prepareDelete(INDEX)); assertPathHasBeenCleared(path); } - - private void assertPathHasBeenCleared(String path) throws Exception { - int count = 0; - StringBuilder sb = new StringBuilder(); - sb.append("["); - if (Files.exists(Paths.get(path))) { - try (DirectoryStream stream = Files.newDirectoryStream(Paths.get(path))) { - for (Path file : stream) { - if (Files.isRegularFile(file)) { - count++; - sb.append(file.toAbsolutePath().toString()); - sb.append("\n"); - } - } - } - } - sb.append("]"); - assertThat(count + " files exist that should have been cleaned:\n" + sb.toString(), count, equalTo(0)); - } } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 9739f26d3ae3c..50eb2476e15dc 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -125,8 +125,10 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.Paths; import java.util.*; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; @@ -1790,6 +1792,35 @@ public Set assertAllShardsOnNodes(String index, String... pattern) { return nodes; } + /** + * Asserts that there are no files in the specified path + */ + public void assertPathHasBeenCleared(String path) throws Exception { + assertPathHasBeenCleared(Paths.get(path)); + } + + /** + * Asserts that there are no files in the specified path + */ + public void assertPathHasBeenCleared(Path path) throws Exception { + int count = 0; + StringBuilder sb = new StringBuilder(); + sb.append("["); + if (Files.exists(path)) { + try (DirectoryStream stream = Files.newDirectoryStream(path)) { + for (Path file : stream) { + if (Files.isRegularFile(file)) { + count++; + sb.append(file.toAbsolutePath().toString()); + sb.append("\n"); + } + } + } + } + sb.append("]"); + assertThat(count + " files exist that should have been cleaned:\n" + sb.toString(), count, equalTo(0)); + } + protected static class NumShards { public final int numPrimaries; public final int numReplicas; From fdbe4133537eaeb768747c2200cfc91878afeb97 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 15:28:57 -0700 Subject: [PATCH 17/56] Use check for shared filesystem in primary -> primary relocation Also adds a nocommit --- .../org/elasticsearch/indices/recovery/RecoverySource.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 3875935a9abab..7fc2f01c2f4ec 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -107,8 +107,8 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { if (shard.routingEntry().primary() && targetShardRouting != null && targetShardRouting.primary() && // must be primary-to-primary relocation - shard.indexSettings().getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - // TODO better exception here + IndexMetaData.usesSharedFilesystem(shard.indexSettings())) { + // nocommit - is this really how we want to handle relocation? logger.info("aborting recovery of shadow primary to shadow primary"); shard.engine().failEngine("attempted to relocate primary shard for shadow index", new ElasticsearchException("aborting recovery of shadow primary to shadow primary")); From 5689b7d2f84ca1c41e4459030af56cb9c0151eff Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 15:58:19 -0700 Subject: [PATCH 18/56] Add testShadowReplicaNaturalRelocation --- .../index/IndexWithShadowReplicasTests.java | 75 +++++++++++++++++-- 1 file changed, 69 insertions(+), 6 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index e667e1d003c34..a6511815b9dff 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -20,9 +20,13 @@ package org.elasticsearch.index; import com.carrotsearch.randomizedtesting.annotations.Repeat; +import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ElasticsearchIntegrationTest; @@ -32,11 +36,14 @@ import org.junit.Test; import java.nio.file.Path; +import java.util.List; +import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; /** * Tests for indices that use shadow replicas and a shared filesystem @@ -53,9 +60,7 @@ public void testIndexWithFewDocuments() throws Exception { .put("node.enable_custom_paths", true) .build(); - String node1 = internalCluster().startNode(nodeSettings); - String node2 = internalCluster().startNode(nodeSettings); - String node3 = internalCluster().startNode(nodeSettings); + internalCluster().startNodesAsync(3, nodeSettings).get(); final String IDX = "test"; final Path dataPath = newTempDirPath(); @@ -133,7 +138,7 @@ public void testReplicaToPrimaryPromotion() throws Exception { // Node1 has the primary, now node2 has the replica String node2 = internalCluster().startNode(nodeSettings); - ensureYellow(IDX); + ensureGreen(IDX); flushAndRefresh(IDX); @@ -160,7 +165,7 @@ public void testIndexWithShadowReplicasCleansUp() throws Exception { .build(); int nodeCount = randomIntBetween(2, 5); - internalCluster().startNodesAsync(nodeCount, nodeSettings); + internalCluster().startNodesAsync(nodeCount, nodeSettings).get(); Path dataPath = newTempDirPath(); String IDX = "test"; @@ -173,7 +178,7 @@ public void testIndexWithShadowReplicasCleansUp() throws Exception { .build(); prepareCreate(IDX).setSettings(idxSettings).get(); - ensureYellow(IDX); + ensureGreen(IDX); client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); flushAndRefresh(IDX); @@ -191,4 +196,62 @@ public void testIndexWithShadowReplicasCleansUp() throws Exception { assertPathHasBeenCleared(dataPath); } + + /** + * Tests that shadow replicas can be "naturally" rebalanced and relocated + * around the cluster. By "naturally" I mean without using the reroute API + * @throws Exception + */ + @Test + public void testShadowReplicaNaturalRelocation() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + .build(); + + internalCluster().startNodesAsync(2, nodeSettings).get(); + Path dataPath = newTempDirPath(); + String IDX = "test"; + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 10) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + ensureGreen(IDX); + + int docCount = randomIntBetween(10, 100); + List builders = newArrayList(); + for (int i = 0; i < docCount; i++) { + builders.add(client().prepareIndex(IDX, "doc", i + "").setSource("body", "foo")); + } + indexRandom(true, true, true, builders); + flushAndRefresh(IDX); + + // start a third node, with 10 shards each on the other nodes, they + // should relocate some to the third node + final String node3 = internalCluster().startNode(nodeSettings); + + assertBusy(new Runnable() { + @Override + public void run() { + client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); + ClusterStateResponse resp = client().admin().cluster().prepareState().get(); + RoutingNodes nodes = resp.getState().getRoutingNodes(); + for (RoutingNode node : nodes) { + logger.info("--> node has {} shards", node.numberOfOwningShards()); + assertThat("at least 5 shards on node", node.numberOfOwningShards(), greaterThanOrEqualTo(5)); + } + } + }); + ensureGreen(IDX); + + logger.info("--> performing query"); + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, docCount); + } } From cf2fb807e7e243f1ad603a79bc9d5f31a499b769 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 11 Feb 2015 16:45:41 -0700 Subject: [PATCH 19/56] Make assertPathHasBeenCleared recursive --- .../elasticsearch/index/IndexWithShadowReplicasTests.java | 6 ++++-- .../elasticsearch/test/ElasticsearchIntegrationTest.java | 6 +++++- 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index a6511815b9dff..ec456b01e1fdc 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index; -import com.carrotsearch.randomizedtesting.annotations.Repeat; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; @@ -53,7 +52,6 @@ public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test @TestLogging("_root:DEBUG,env:TRACE") - @Repeat(iterations = 10) // NOCOMMIT public void testIndexWithFewDocuments() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) @@ -253,5 +251,9 @@ public void run() { logger.info("--> performing query"); SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); assertHitCount(resp, docCount); + + assertAcked(client().admin().indices().prepareDelete(IDX)); + + assertPathHasBeenCleared(dataPath); } } diff --git a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java index 50eb2476e15dc..3b1ea2a094080 100644 --- a/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java +++ b/src/test/java/org/elasticsearch/test/ElasticsearchIntegrationTest.java @@ -1803,13 +1803,17 @@ public void assertPathHasBeenCleared(String path) throws Exception { * Asserts that there are no files in the specified path */ public void assertPathHasBeenCleared(Path path) throws Exception { + logger.info("--> checking that [{}] has been cleared", path); int count = 0; StringBuilder sb = new StringBuilder(); sb.append("["); if (Files.exists(path)) { try (DirectoryStream stream = Files.newDirectoryStream(path)) { for (Path file : stream) { - if (Files.isRegularFile(file)) { + logger.info("--> found file: [{}]", file.toAbsolutePath().toString()); + if (Files.isDirectory(file)) { + assertPathHasBeenCleared(file); + } else if (Files.isRegularFile(file)) { count++; sb.append(file.toAbsolutePath().toString()); sb.append("\n"); From 4a367c07505b84b452807a58890f1cbe21711f27 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 12 Feb 2015 09:50:36 +0100 Subject: [PATCH 20/56] fix primary promotion --- .../elasticsearch/index/shard/IndexShard.java | 3 +- .../index/IndexWithShadowReplicasTests.java | 28 +++++++++++-------- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 4a6d0f7cda499..cedfe2e96980c 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -329,8 +329,9 @@ public IndexShard routingEntry(ShardRouting newRouting) { if (currentRouting.primary() == false && // currently a replica newRouting.primary() == true && // becoming a primary indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { + this.shardRouting = newRouting; // this is important otherwise we will not fail the shard right-away failShard("can't promote shadow replica to primary", - new RuntimeException("can't promote shadow replica to primary")); + new ElasticsearchIllegalStateException("can't promote shadow replica to primary")); return this; } } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index ec456b01e1fdc..9c49f07fc445a 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -48,10 +48,10 @@ * Tests for indices that use shadow replicas and a shared filesystem */ @ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) +@TestLogging("_root:DEBUG,env:TRACE") //nocommit public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test - @TestLogging("_root:DEBUG,env:TRACE") public void testIndexWithFewDocuments() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) @@ -59,7 +59,6 @@ public void testIndexWithFewDocuments() throws Exception { .build(); internalCluster().startNodesAsync(3, nodeSettings).get(); - final String IDX = "test"; final Path dataPath = newTempDirPath(); @@ -110,7 +109,6 @@ public void testIndexWithFewDocuments() throws Exception { } @Test - @Ignore // nocommit - this currently fails because of the way that we promote shadow replicas into primaries public void testReplicaToPrimaryPromotion() throws Exception { Settings nodeSettings = ImmutableSettings.builder() .put("node.add_id_to_custom_path", false) @@ -129,30 +127,38 @@ public void testReplicaToPrimaryPromotion() throws Exception { .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .build(); - prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); ensureYellow(IDX); client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertTrue(gResp1.isExists()); + assertTrue(gResp2.isExists()); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + // Node1 has the primary, now node2 has the replica String node2 = internalCluster().startNode(nodeSettings); ensureGreen(IDX); - + client().admin().cluster().prepareHealth().setWaitForNodes("2").get(); flushAndRefresh(IDX); logger.info("--> stopping node1 [{}]", node1); internalCluster().stopRandomNode(InternalTestCluster.nameFilter(node1)); - ensureYellow(IDX); - GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); - GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); - assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); - assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); - logger.info("--> performing query"); SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); assertHitCount(resp, 2); + + gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertTrue(gResp1.isExists()); + assertTrue(gResp2.toString(), gResp2.isExists()); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); } @Test From f2297199b7dd5d3f9f1f109d0ddf3dd83390b0d1 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 12 Feb 2015 12:41:32 +0100 Subject: [PATCH 21/56] first cut at catchup from primary make flush to a refresh factor our ShadowIndexShard to have IndexShard be idential to the master and least intrusive cleanup abstractions --- .../cluster/metadata/IndexMetaData.java | 7 +- .../deletionpolicy/SnapshotIndexCommit.java | 3 +- .../index/engine/ShadowEngine.java | 1 + .../elasticsearch/index/shard/IndexShard.java | 38 ++--- .../index/shard/IndexShardModule.java | 7 +- .../index/shard/ShadowIndexShard.java | 152 ++++++++++++++++++ .../cluster/IndicesClusterStateService.java | 14 +- .../recovery/FileSyncRecoveryHandler.java | 56 +++++++ .../indices/recovery/RecoverySource.java | 15 +- .../indices/recovery/RecoveryState.java | 3 +- .../indices/recovery/RecoveryTarget.java | 17 +- .../recovery/ShardRecoveryHandler.java | 15 +- .../index/IndexWithShadowReplicasTests.java | 94 ++++++++++- .../test/store/MockFSDirectoryService.java | 5 +- 14 files changed, 377 insertions(+), 50 deletions(-) create mode 100644 src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java create mode 100644 src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index a6bd68a170322..da265fb435afe 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -789,7 +789,12 @@ public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws // NOCOMMIT find a good place for this and document it public static boolean usesSharedFilesystem(Settings settings) { - return settings.getAsBoolean(SETTING_SHARED_FILESYSTEM, settings.getAsBoolean(SETTING_SHADOW_REPLICAS, false)); + return settings.getAsBoolean(SETTING_SHARED_FILESYSTEM, isIndexUsingShadowReplicas(settings)); + } + + // NOCOMMIT find a good place for this and document it + public static boolean isIndexUsingShadowReplicas(Settings settings) { + return settings.getAsBoolean(SETTING_SHADOW_REPLICAS, false); } } diff --git a/src/main/java/org/elasticsearch/index/deletionpolicy/SnapshotIndexCommit.java b/src/main/java/org/elasticsearch/index/deletionpolicy/SnapshotIndexCommit.java index 5c975789408e4..02f5658c40357 100644 --- a/src/main/java/org/elasticsearch/index/deletionpolicy/SnapshotIndexCommit.java +++ b/src/main/java/org/elasticsearch/index/deletionpolicy/SnapshotIndexCommit.java @@ -54,8 +54,7 @@ public String[] getFiles() { } /** - * Releases the current snapshot, returning true if it was - * actually released. + * Releases the current snapshot. */ public void close() { deletionPolicy.close(getGeneration()); diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 44b697dbc2f5e..bbec033981b17 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -118,6 +118,7 @@ public void flush() throws EngineException { public void flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.debug("cowardly refusing to FLUSH"); // reread the last committed segment infos + refresh("flush"); try { lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); } catch (Throwable e) { diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index cedfe2e96980c..52f47683f3555 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -34,8 +34,10 @@ import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.common.Booleans; @@ -46,17 +48,16 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.aliases.IndexAliasesService; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.filter.FilterCacheStats; @@ -104,7 +105,10 @@ import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.InternalIndicesLifecycle; +import org.elasticsearch.indices.cluster.IndicesClusterStateService; +import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat; import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.threadpool.ThreadPool; @@ -114,6 +118,7 @@ import java.nio.channels.ClosedByInterruptException; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -151,21 +156,19 @@ public class IndexShard extends AbstractIndexShardComponent { private final IndexService indexService; private final ShardSuggestService shardSuggestService; private final ShardBitsetFilterCache shardBitsetFilterCache; - private final MappingUpdatedAction mappingUpdatedAction; - private final CancellableThreads cancellableThreads = new CancellableThreads(); private final Object mutex = new Object(); private final String checkIndexOnStartup; private final EngineConfig config; + private final EngineFactory engineFactory; private long checkIndexTook = 0; - private volatile EngineFactory engineFactory; private volatile IndexShardState state; private TimeValue refreshInterval; private volatile ScheduledFuture refreshScheduledFuture; private volatile ScheduledFuture mergeScheduleFuture; - private volatile ShardRouting shardRouting; + protected volatile ShardRouting shardRouting; @Nullable private RecoveryState recoveryState; @@ -190,9 +193,8 @@ public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexS CodecService codecService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, - @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, AnalysisService analysisService, - SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, - MappingUpdatedAction mappingUpdatedAction) { + @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, + SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { super(shardId, indexSettings); Preconditions.checkNotNull(store, "Store must be provided to the index shard"); Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard"); @@ -223,7 +225,6 @@ public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexS this.codecService = codecService; this.shardSuggestService = shardSuggestService; this.shardBitsetFilterCache = shardBitsetFilterCache; - this.mappingUpdatedAction = mappingUpdatedAction; state = IndexShardState.CREATED; this.refreshInterval = indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL); indexSettingsService.addListener(applyRefreshSettings); @@ -323,17 +324,6 @@ public IndexShard routingEntry(ShardRouting newRouting) { if (currentRouting.equals(newRouting)) { return this; } - - // check for a shadow replica that now needs to be transformed into - // a normal primary - if (currentRouting.primary() == false && // currently a replica - newRouting.primary() == true && // becoming a primary - indexSettings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false)) { - this.shardRouting = newRouting; // this is important otherwise we will not fail the shard right-away - failShard("can't promote shadow replica to primary", - new ElasticsearchIllegalStateException("can't promote shadow replica to primary")); - return this; - } } if (state == IndexShardState.POST_RECOVERY) { @@ -891,7 +881,7 @@ private void writeAllowed(Engine.Operation.Origin origin) throws IllegalIndexSha } } - private void verifyStartedOrRecovering() throws IllegalIndexShardStateException { + protected final void verifyStartedOrRecovering() throws IllegalIndexShardStateException { IndexShardState state = this.state; // one time volatile read if (state != IndexShardState.STARTED && state != IndexShardState.RECOVERING && state != IndexShardState.POST_RECOVERY) { throw new IllegalIndexShardStateException(shardId, state, "operation only allowed when started/recovering"); @@ -905,7 +895,7 @@ private void verifyNotClosed() throws IllegalIndexShardStateException { } } - private void verifyStarted() throws IllegalIndexShardStateException { + protected final void verifyStarted() throws IllegalIndexShardStateException { IndexShardState state = this.state; // one time volatile read if (state != IndexShardState.STARTED) { throw new IndexShardNotStartedException(shardId, state); diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index c6bbc90004aa9..dde9088b7e498 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -60,7 +60,12 @@ protected boolean useShadowEngine() { @Override protected void configure() { bind(ShardId.class).toInstance(shardId); - bind(IndexShard.class).asEagerSingleton(); + if (useShadowEngine()) { + bind(IndexShard.class).to(ShadowIndexShard.class).asEagerSingleton(); + } else { + bind(IndexShard.class).asEagerSingleton(); + } + Class engineFactory = DEFAULT_ENGINE_FACTORY_CLASS; String factorySetting = ENGINE_FACTORY; if (useShadowEngine()) { diff --git a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java new file mode 100644 index 0000000000000..f75804632312e --- /dev/null +++ b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -0,0 +1,152 @@ +/* + * 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.shard; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchIllegalStateException; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.aliases.IndexAliasesService; +import org.elasticsearch.index.cache.IndexCache; +import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; +import org.elasticsearch.index.cache.filter.ShardFilterCache; +import org.elasticsearch.index.cache.query.ShardQueryCache; +import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineException; +import org.elasticsearch.index.engine.EngineFactory; +import org.elasticsearch.index.engine.ShadowEngine; +import org.elasticsearch.index.fielddata.IndexFieldDataService; +import org.elasticsearch.index.fielddata.ShardFieldData; +import org.elasticsearch.index.get.ShardGetService; +import org.elasticsearch.index.indexing.ShardIndexingService; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.merge.policy.MergePolicyProvider; +import org.elasticsearch.index.merge.scheduler.MergeSchedulerProvider; +import org.elasticsearch.index.percolator.PercolatorQueriesRegistry; +import org.elasticsearch.index.percolator.stats.ShardPercolateService; +import org.elasticsearch.index.query.IndexQueryParserService; +import org.elasticsearch.index.search.stats.ShardSearchService; +import org.elasticsearch.index.settings.IndexSettings; +import org.elasticsearch.index.settings.IndexSettingsService; +import org.elasticsearch.index.similarity.SimilarityService; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.suggest.stats.ShardSuggestService; +import org.elasticsearch.index.termvectors.ShardTermVectorsService; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.warmer.ShardIndexWarmerService; +import org.elasticsearch.indices.IndicesLifecycle; +import org.elasticsearch.indices.IndicesWarmer; +import org.elasticsearch.indices.cluster.IndicesClusterStateService; +import org.elasticsearch.indices.recovery.RecoveryFailedException; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.indices.recovery.RecoveryTarget; +import org.elasticsearch.threadpool.ThreadPool; + +import java.util.concurrent.CountDownLatch; + +/** + */ //nocommit document this! +public class ShadowIndexShard extends IndexShard { + private final RecoveryTarget recoveryTarget; + private final ClusterService clusterService; + + @Inject + public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, RecoveryTarget recoveryTarget, ClusterService clusterService) { + super(shardId, indexSettings, indexSettingsService, indicesLifecycle, store, mergeScheduler, translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, indexingService, getService, searchService, shardWarmerService, shardFilterCache, shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService, termVectorsService, indexFieldDataService, indexService, shardSuggestService, shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService, mergePolicyProvider, factory); + this.recoveryTarget = recoveryTarget; + this.clusterService = clusterService; + } + + public void flush(FlushRequest request) throws ElasticsearchException { + if (state() == IndexShardState.STARTED) { + syncFilesFromPrimary(); + } + super.flush(request); + } + + private void syncFilesFromPrimary() { + final ShardRouting shardRouting = routingEntry(); + if (IndexMetaData.usesSharedFilesystem(indexSettings()) == false && shardRouting.primary() == false) { + // nocommit - we are running a full recovery here I wonder if we should do this only do this if request.waitIfOngoing() == true? Or if we need a new parameter? + // I also wonder if we want to have an infrastructure for this instead that communicates with the primary etc? + ClusterState state = clusterService.state(); + final CountDownLatch latch = new CountDownLatch(1); + DiscoveryNode sourceNode = IndicesClusterStateService.findSourceNodeForPeerRecovery(state.routingTable(), state.nodes(), shardRouting, logger); + if (sourceNode != null) { + assert engine() instanceof ShadowEngine; + recoveryTarget.startFileSync(this, sourceNode, new RecoveryTarget.RecoveryListener() { + @Override + public void onRecoveryDone(RecoveryState state) { + latch.countDown(); + logger.info("shadow replica catchup done {}", state); + // nocommit + } + + @Override + public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) { + latch.countDown(); + logger.warn(" failed to catch up shadow replica can't find source node", e); + //nocommit + } + }); + try { + latch.await(); + } catch (InterruptedException e) { + // ignore + } + } else { + logger.warn(" failed to catch up shadow replica can't find source node", shardId); + } + } + } + + @Override + public IndexShard routingEntry(ShardRouting newRouting) { + ShardRouting shardRouting = this.routingEntry(); + super.routingEntry(newRouting); + // check for a shadow replica that now needs to be transformed into + // a normal primary today we simply fail it to force reallocation + if (shardRouting != null && shardRouting.primary() == false && // currently a replica + newRouting.primary() == true) {// becoming a primary + failShard("can't promote shadow replica to primary", + new ElasticsearchIllegalStateException("can't promote shadow replica to primary")); + } + return this; + } + + @Override + public void performRecoveryFinalization(boolean withFlush, RecoveryState recoveryState) throws ElasticsearchException { + if (recoveryState.getType() == RecoveryState.Type.FILE_SYNC) { + logger.debug("skipping recovery finalization file sync runs on a started engine"); + } else { + super.performRecoveryFinalization(withFlush, recoveryState); + } + } +} diff --git a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index eaf900a81d9c5..e2b36279c0852 100644 --- a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -44,6 +44,7 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; @@ -98,7 +99,6 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent failedShards = ConcurrentCollections.newConcurrentMap(); - private final NodeEnvironment nodeEnvironment; static class FailedShard { public final long version; @@ -114,15 +114,13 @@ static class FailedShard { private final FailedEngineHandler failedEngineHandler = new FailedEngineHandler(); private final boolean sendRefreshMapping; - private final AtomicLong recoveryIdGenerator = new AtomicLong(); @Inject public IndicesClusterStateService(Settings settings, IndicesService indicesService, ClusterService clusterService, ThreadPool threadPool, RecoveryTarget recoveryTarget, ShardStateAction shardStateAction, NodeIndexDeletedAction nodeIndexDeletedAction, - NodeMappingRefreshAction nodeMappingRefreshAction, - NodeEnvironment nodeEnvironment) { + NodeMappingRefreshAction nodeMappingRefreshAction) { super(settings); this.indicesService = indicesService; this.clusterService = clusterService; @@ -133,7 +131,6 @@ public IndicesClusterStateService(Settings settings, IndicesService indicesServi this.nodeMappingRefreshAction = nodeMappingRefreshAction; this.sendRefreshMapping = componentSettings.getAsBoolean("send_refresh_mapping", true); - this.nodeEnvironment = nodeEnvironment; } @Override @@ -573,7 +570,7 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela RecoveryState recoveryState = recoveryTarget.recoveryState(indexShard); if (recoveryState != null && recoveryState.getStage() != RecoveryState.Stage.DONE) { // we have an ongoing recovery, find the source based on current routing and compare them - DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting); + DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting, logger); if (!recoveryState.getSourceNode().equals(sourceNode)) { logger.debug("[{}][{}] removing shard (recovery source changed), current [{}], global [{}])", shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting); @@ -669,7 +666,7 @@ private void applyInitializingShard(final RoutingTable routingTable, final Disco // if we're in peer recovery, try to find out the source node now so in case it fails, we will not create the index shard DiscoveryNode sourceNode = null; if (isPeerRecovery(shardRouting)) { - sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting); + sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting, logger); if (sourceNode == null) { logger.trace("ignoring initializing shard {} - no source node can be found.", shardRouting.shardId()); return; @@ -767,7 +764,8 @@ public void onRecoveryFailed(IndexShardGatewayRecoveryException e) { * routing to *require* peer recovery, use {@link #isPeerRecovery(org.elasticsearch.cluster.routing.ShardRouting)} to * check if its needed or not. */ - private DiscoveryNode findSourceNodeForPeerRecovery(RoutingTable routingTable, DiscoveryNodes nodes, ShardRouting shardRouting) { + public static DiscoveryNode findSourceNodeForPeerRecovery(RoutingTable routingTable, DiscoveryNodes nodes, ShardRouting shardRouting, ESLogger logger) { + //nocommit factor this out somewhere useful DiscoveryNode sourceNode = null; if (!shardRouting.primary()) { IndexShardRoutingTable shardRoutingTable = routingTable.index(shardRouting.index()).shard(shardRouting.id()); diff --git a/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java new file mode 100644 index 0000000000000..d87885e453279 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java @@ -0,0 +1,56 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.indices.recovery; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.transport.TransportService; + +/** + * A recovery handler that skips phase 2 and 3 of the recovery + */ +public class FileSyncRecoveryHandler extends ShardRecoveryHandler { + + private IndexShard shard; + private final StartRecoveryRequest request; + + public FileSyncRecoveryHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ClusterService clusterService, IndicesService indicesService, MappingUpdatedAction mappingUpdatedAction, ESLogger logger) { + super(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + this.shard = shard; + this.request = request; + + } + + @Override + public void phase2(Translog.Snapshot snapshot) throws ElasticsearchException { + logger.trace("{} recovery [phase2] to {}: skipping translog for file sync", request.shardId(), request.targetNode()); + } + + @Override + protected int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { + logger.trace("{} recovery [phase3] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode()); + return 0; + } +} diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 7fc2f01c2f4ec..9496c3e5505d4 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -29,9 +29,12 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.settings.IndexSettings; +import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesLifecycle; @@ -117,15 +120,21 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), request.targetNode()); throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); } - if (!targetShardRouting.initializing()) { + if (request.recoveryType() != RecoveryState.Type.FILE_SYNC && !targetShardRouting.initializing()) { logger.debug("delaying recovery of {} as it is not listed as initializing on the target node {}. known shards state is [{}]", request.shardId(), request.targetNode(), targetShardRouting.state()); throw new DelayRecoveryException("source node has the state of the target shard to be [" + targetShardRouting.state() + "], expecting to be [initializing]"); } logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated()); - - final ShardRecoveryHandler handler = new ShardRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + final ShardRecoveryHandler handler; + if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { + logger.trace("{} taking snapshot before file sync recovery", shard.shardId()); + Releasables.close(shard.snapshotIndex()); // we have to take a snapshot here and close it right away so we can catchup from the latest docs committed + handler = new FileSyncRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + } else { + handler = new ShardRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + } ongoingRecoveries.add(shard, handler); try { shard.recover(handler); diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index e60541819aa30..1ffa83e7572e7 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -83,7 +83,8 @@ public static enum Type { GATEWAY((byte) 0), SNAPSHOT((byte) 1), REPLICA((byte) 2), - RELOCATION((byte) 3); + RELOCATION((byte) 3), + FILE_SYNC((byte) 4); private static final Type[] TYPES = new Type[Type.values().length]; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index a7e2f20c026ce..d73e487991d0d 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -22,6 +22,7 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexOutput; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -134,6 +135,21 @@ public void startRecovery(final IndexShard indexShard, final RecoveryState.Type logger.debug("{} ignore recovery. already in recovering process, {}", indexShard.shardId(), e.getMessage()); return; } + startRecoveryInternal(indexShard, recoveryType, sourceNode, listener); + } + + /** + * Runs a full recovery but skips phase 2 and 3 and only syncs the files that have been committed. + */ + public void startFileSync(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) { + if (indexShard.state() != IndexShardState.STARTED) { + throw new ElasticsearchIllegalStateException("Shard is not started can't sync files -state: " + indexShard.state()); + } + logger.trace("{} starting file sync with {}", indexShard.shardId(), sourceNode); + startRecoveryInternal(indexShard, RecoveryState.Type.FILE_SYNC, sourceNode, listener); + } + + private void startRecoveryInternal(IndexShard indexShard, RecoveryState.Type recoveryType, DiscoveryNode sourceNode, RecoveryListener listener) { // create a new recovery status, and process... RecoveryState recoveryState = new RecoveryState(indexShard.shardId()); recoveryState.setType(recoveryType); @@ -142,7 +158,6 @@ public void startRecovery(final IndexShard indexShard, final RecoveryState.Type recoveryState.setPrimary(indexShard.routingEntry().primary()); final long recoveryId = onGoingRecoveries.startRecovery(indexShard, sourceNode, recoveryState, listener, recoverySettings.activityTimeout()); threadPool.generic().execute(new RecoveryRunner(recoveryId)); - } protected void retryRecovery(final RecoveryStatus recoveryStatus, final String reason, TimeValue retryAfter, final StartRecoveryRequest currentRequest) { diff --git a/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java index a8c87cef1b7e6..19ba821bba11e 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java @@ -71,9 +71,9 @@ * everything relating to copying the segment files as well as sending translog * operations across the wire once the segments have been copied. */ -public final class ShardRecoveryHandler implements Engine.RecoveryHandler { +public class ShardRecoveryHandler implements Engine.RecoveryHandler { - private final ESLogger logger; + protected final ESLogger logger; // Shard that is going to be recovered (the "source") private final IndexShard shard; private final String indexName; @@ -375,6 +375,10 @@ public void run() throws InterruptedException { */ @Override public void phase2(Translog.Snapshot snapshot) throws ElasticsearchException { + if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { + logger.trace("{} recovery [phase2] to {}: skipping translog for file sync", request.shardId(), request.targetNode()); + return; + } if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } @@ -430,11 +434,12 @@ public void phase3(Translog.Snapshot snapshot) throws ElasticsearchException { throw new IndexShardClosedException(request.shardId()); } cancellableThreads.checkForCancel(); - logger.trace("[{}][{}] recovery [phase3] to {}: sending transaction log operations", indexName, shardId, request.targetNode()); StopWatch stopWatch = new StopWatch().start(); + final int totalOperations; + logger.trace("[{}][{}] recovery [phase3] to {}: sending transaction log operations", indexName, shardId, request.targetNode()); // Send the translog operations to the target node - int totalOperations = sendSnapshot(snapshot); + totalOperations = sendSnapshot(snapshot); cancellableThreads.execute(new Interruptable() { @Override @@ -538,7 +543,7 @@ public void run() throws InterruptedException { * * @return the total number of translog operations that were sent */ - private int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { + protected int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { int ops = 0; long size = 0; int totalOperations = 0; diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 9c49f07fc445a..4c50659036b3a 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -23,15 +23,15 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.RoutingNode; -import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.cluster.routing.*; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.junit.annotations.TestLogging; -import org.junit.Ignore; import org.junit.Test; import java.nio.file.Path; @@ -262,4 +262,92 @@ public void run() { assertPathHasBeenCleared(dataPath); } + + + public void testSimpleNonSharedFS() throws Exception { + internalCluster().startNodesAsync(3).get(); + final String IDX = "test"; + final Path dataPath = newTempDirPath(); + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, 2)) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, false) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).get(); + ensureGreen(IDX); + + IndexRequestBuilder[] builders = new IndexRequestBuilder[between(10, 20)]; + for (int i = 0; i < builders.length; i++) { + builders[i] = client().prepareIndex(IDX, "doc", Integer.toString(i)).setSource("foo", "bar"); + } + indexRandom(false, builders); + flush(IDX); + logger.info("--> performing query"); + for (int i = 0; i < 10; i++) { + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, builders.length); + } + + logger.info("--> restarting all nodes"); + if (randomBoolean()) { + logger.info("--> rolling restart"); + internalCluster().rollingRestart(); + } else { + logger.info("--> full restart"); + internalCluster().fullRestart(); + } + + client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); + ensureGreen(IDX); + + logger.info("--> performing query"); + for (int i = 0; i < 10; i++) { + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, builders.length); + for (SearchHit hit : resp.getHits().getHits()) { + assertEquals(hit.sourceAsMap().get("foo"), "bar"); + } + } + + // we reindex and check if the new updated docs are all available on the primary + for (int i = 0; i < builders.length; i++) { + builders[i] = client().prepareIndex(IDX, "doc", Integer.toString(i)).setSource("foo", "foobar"); + } + indexRandom(false, builders); + // only refresh no flush + refresh(); + ClusterState state = client().admin().cluster().prepareState().get().getState(); + GroupShardsIterator shardIterators = state.getRoutingNodes().getRoutingTable().activePrimaryShardsGrouped(new String[]{IDX}, false); + ShardRouting primary = shardIterators.iterator().next().nextOrNull(); + for (int i = 0; i < 10; i++) { + SearchResponse resp = client().prepareSearch(IDX).setExplain(true).setQuery(matchAllQuery()).get(); + assertHitCount(resp, builders.length); + for (SearchHit hit : resp.getHits().getHits()) { + if (hit.shard().getNodeId().equals(primary.currentNodeId())) { + // this comes from the primary so we have the latest + assertEquals(hit.sourceAsMap().get("foo"), "foobar"); + } else { + // this comes from the replica not caught up yet + assertEquals(hit.sourceAsMap().get("foo"), "bar"); + } + } + } + + flush(IDX); + logger.info("--> performing query"); + for (int i = 0; i < 10; i++) { + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, builders.length); + for (SearchHit hit : resp.getHits().getHits()) { + assertEquals(hit.sourceAsMap().get("foo"), "foobar"); + } + } + + logger.info("--> deleting index"); + assertAcked(client().admin().indices().prepareDelete(IDX)); + } } diff --git a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 3b37dc8369a77..651d4787f16c3 100644 --- a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -26,9 +26,12 @@ import org.apache.lucene.store.LockFactory; import org.apache.lucene.store.StoreRateLimiting; import org.apache.lucene.util.AbstractRandomizedTest; +import org.apache.lucene.util.IOUtils; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.Engine; @@ -87,7 +90,7 @@ public void beforeIndexShardClosed(ShardId sid, @Nullable IndexShard indexShard, // When the the internal engine closes we do a rollback, which removes uncommitted segments // By doing a commit flush we perform a Lucene commit, but don't clear the translog, // so that even in tests where don't flush we can check the integrity of the Lucene index - indexShard.engine().snapshotIndex(); // Keep translog for tests that rely on replaying it + Releasables.close(indexShard.engine().snapshotIndex()); // Keep translog for tests that rely on replaying it logger.info("flush finished in beforeIndexShardClosed"); } } From a62b9a70adad87d7492c526f4daf868cb05018d9 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 12 Feb 2015 15:59:54 +0100 Subject: [PATCH 22/56] fix compile error after upstream changes --- .../java/org/elasticsearch/index/engine/ShadowEngine.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index bbec033981b17..2a8e5a02daa42 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -132,12 +132,12 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { } @Override - public void forceMerge(boolean flush, boolean waitForMerge) { - forceMerge(flush, waitForMerge, 1, false, false); + public void forceMerge(boolean flush) { + forceMerge(flush, 1, false, false); } @Override - public void forceMerge(boolean flush, boolean waitForMerge, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { + public void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { logger.debug("cowardly refusing to FORCE_MERGE, since the since the primary will do it"); } From a7eb53c1e8b8fbfd9281b43ae39eacbe3cd1a0a6 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Thu, 12 Feb 2015 17:38:59 +0100 Subject: [PATCH 23/56] Simplify shared filesystem recovery by using a dedicated recovery handler that skip most phases and enforces shard closing on the soruce before the target opens it's engine --- .../org/elasticsearch/index/store/Store.java | 3 +- .../indices/recovery/RecoverySource.java | 13 +--- .../indices/recovery/RecoveryTarget.java | 11 +-- .../recovery/SharedFSRecoveryHandler.java | 72 +++++++++++++++++++ .../index/IndexWithShadowReplicasTests.java | 3 +- .../elasticsearch/index/store/StoreTest.java | 4 +- 6 files changed, 81 insertions(+), 25 deletions(-) create mode 100644 src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index 66b33e7b8d5bf..b604e2f0f1b37 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -547,11 +547,10 @@ private static final void failIfCorrupted(Directory directory, ShardId shardId) * * @param reason the reason for this cleanup operation logged for each deleted file * @param sourceMetaData the metadata used for cleanup. all files in this metadata should be kept around. - * @param indexSettings the settings for the index that is being cleaned up * @throws IOException if an IOException occurs * @throws ElasticsearchIllegalStateException if the latest snapshot in this store differs from the given one after the cleanup. */ - public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData, @IndexSettings Settings indexSettings) throws IOException { + public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData) throws IOException { failIfCorrupted(); metadataLock.writeLock().lock(); try { diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 9496c3e5505d4..de578213a9ebe 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -107,15 +107,6 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { break; } } - if (shard.routingEntry().primary() && - targetShardRouting != null && - targetShardRouting.primary() && // must be primary-to-primary relocation - IndexMetaData.usesSharedFilesystem(shard.indexSettings())) { - // nocommit - is this really how we want to handle relocation? - logger.info("aborting recovery of shadow primary to shadow primary"); - shard.engine().failEngine("attempted to relocate primary shard for shadow index", - new ElasticsearchException("aborting recovery of shadow primary to shadow primary")); - } if (targetShardRouting == null) { logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), request.targetNode()); throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); @@ -128,7 +119,9 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated()); final ShardRecoveryHandler handler; - if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { + if (IndexMetaData.usesSharedFilesystem(shard.indexSettings())) { + handler = new SharedFSRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + } else if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { logger.trace("{} taking snapshot before file sync recovery", shard.shardId()); Releasables.close(shard.snapshotIndex()); // we have to take a snapshot here and close it right away so we can catchup from the latest docs committed handler = new FileSyncRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index d73e487991d0d..35bb1a85113cb 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -172,13 +172,6 @@ protected void retryRecovery(final RecoveryStatus recoveryStatus, final String r private void doRecovery(final RecoveryStatus recoveryStatus) { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; - if (IndexMetaData.usesSharedFilesystem(recoveryStatus.indexShard().indexSettings())) { - // NOCOMMIT - this is a super shortcut we need to check if statistics are all on etc. - recoveryStatus.indexShard().performRecoveryPrepareForTranslog(); - recoveryStatus.indexShard().performRecoveryFinalization(false, recoveryStatus.state()); - onGoingRecoveries.markRecoveryAsDone(recoveryStatus.recoveryId()); - return; - } logger.trace("collecting local files for {}", recoveryStatus); final Map existingFiles; try { @@ -419,9 +412,7 @@ public void messageReceived(RecoveryCleanFilesRequest request, TransportChannel recoveryStatus.legacyChecksums().write(store); Store.MetadataSnapshot sourceMetaData = request.sourceMetaSnapshot(); try { - IndexMetaData indexMeta = clusterService.state().getMetaData().index(request.shardId().getIndex()); - assert IndexMetaData.usesSharedFilesystem(indexMeta.settings()) == false : "[" + indexMeta.getIndex() +"] index uses shared FS - can't recover / clean files"; - store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData, indexMeta.settings()); + store.cleanupAndVerify("recovery CleanFilesRequestHandler", sourceMetaData); } catch (Exception ex) { throw new RecoveryFailedException(recoveryStatus.state(), "failed to clean after recovery", ex); } diff --git a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java new file mode 100644 index 0000000000000..3351893c8e073 --- /dev/null +++ b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java @@ -0,0 +1,72 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.indices.recovery; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.cluster.ClusterService; +import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.transport.TransportService; + +import java.io.IOException; + +/** + * A recovery handler that skips phase 1 as well as sending the snapshot. During phase 3 the shard is marked + * as relocated an closed to ensure that the engine is closed and the target can acquire the IW write lock. + */ +public class SharedFSRecoveryHandler extends ShardRecoveryHandler { + + private final IndexShard shard; + private final StartRecoveryRequest request; + + public SharedFSRecoveryHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ClusterService clusterService, IndicesService indicesService, MappingUpdatedAction mappingUpdatedAction, ESLogger logger) { + super(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); + this.shard = shard; + this.request = request; + } + + @Override + public void phase1(SnapshotIndexCommit snapshot) throws ElasticsearchException { + logger.trace("{} recovery [phase2] to {}: skipping phase 1 for shared filesystem", request.shardId(), request.targetNode()); + } + + @Override + protected int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { + logger.trace("{} recovery [phase3] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode()); + return 0; + } + + @Override + public void phase3(Translog.Snapshot snapshot) throws ElasticsearchException { + if (request.markAsRelocated()) { + try { + shard.relocated("to " + request.targetNode()); + shard.close("relocated on shared filesystem"); + } catch (IOException e) { + + } + } + super.phase3(snapshot); + } +} diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 4c50659036b3a..2bc4e67abfbc1 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -35,6 +35,7 @@ import org.junit.Test; import java.nio.file.Path; +import java.util.Arrays; import java.util.List; import static com.google.common.collect.Lists.newArrayList; @@ -317,7 +318,7 @@ public void testSimpleNonSharedFS() throws Exception { for (int i = 0; i < builders.length; i++) { builders[i] = client().prepareIndex(IDX, "doc", Integer.toString(i)).setSource("foo", "foobar"); } - indexRandom(false, builders); + indexRandom(false, true, false, Arrays.asList(builders)); // don't flush here // only refresh no flush refresh(); ClusterState state = client().admin().cluster().prepareState().get().getState(); diff --git a/src/test/java/org/elasticsearch/index/store/StoreTest.java b/src/test/java/org/elasticsearch/index/store/StoreTest.java index 656a29a3e6a5d..88272ca9d14bb 100644 --- a/src/test/java/org/elasticsearch/index/store/StoreTest.java +++ b/src/test/java/org/elasticsearch/index/store/StoreTest.java @@ -967,7 +967,7 @@ public void testCleanupFromSnapshot() throws IOException { checksums.write(store); // write one checksum file here - we expect it to survive all the cleanups if (randomBoolean()) { - store.cleanupAndVerify("test", firstMeta, ImmutableSettings.EMPTY); + store.cleanupAndVerify("test", firstMeta); String[] strings = store.directory().listAll(); int numChecksums = 0; int numNotFound = 0; @@ -983,7 +983,7 @@ public void testCleanupFromSnapshot() throws IOException { assertTrue("at least one file must not be in here since we have two commits?", numNotFound > 0); assertEquals("we wrote one checksum but it's gone now? - checksums are supposed to be kept", numChecksums, 1); } else { - store.cleanupAndVerify("test", secondMeta, ImmutableSettings.EMPTY); + store.cleanupAndVerify("test", secondMeta); String[] strings = store.directory().listAll(); int numChecksums = 0; int numNotFound = 0; From d8d59dbf6d0525cd823d97268d035820e5727ac9 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 12 Feb 2015 10:34:32 -0700 Subject: [PATCH 24/56] Refactor more shared methods into the abstract Engine --- .../elasticsearch/index/engine/Engine.java | 30 +++++++++++++++++ .../index/engine/InternalEngine.java | 30 +++++------------ .../index/engine/ShadowEngine.java | 32 ++----------------- 3 files changed, 40 insertions(+), 52 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 86dd4e2c0c4be..4677a411c3cbb 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -29,6 +29,7 @@ import org.apache.lucene.util.Accountables; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchIllegalStateException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Preconditions; import org.elasticsearch.common.bytes.BytesReference; @@ -36,6 +37,7 @@ import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.VersionType; @@ -417,6 +419,34 @@ public final boolean refreshNeeded() { /** fail engine due to some error. the engine will also be closed. */ public abstract void failEngine(String reason, Throwable failure); + /** Check whether the engine should be failed */ + protected boolean maybeFailEngine(String source, Throwable t) { + if (Lucene.isCorruptionException(t)) { + if (engineConfig.isFailEngineOnCorruption()) { + failEngine("corrupt file detected source: [" + source + "]", t); + return true; + } else { + logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, + EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, engineConfig.isFailEngineOnCorruption()); + } + } else if (ExceptionsHelper.isOOM(t)) { + failEngine("out of memory", t); + return true; + } + return false; + } + + /** Wrap a Throwable in an {@code EngineClosedException} if the engine is already closed */ + protected Throwable wrapIfClosed(Throwable t) { + if (isClosed.get()) { + if (t != failedEngine && failedEngine != null) { + t.addSuppressed(failedEngine); + } + return new EngineClosedException(shardId, t); + } + return t; + } + public static interface FailedEngineListener { void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable t); } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 19e369b488b8e..3879eae7f2937 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.math.MathUtils; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; @@ -826,18 +825,15 @@ public void recover(RecoveryHandler recoveryHandler) throws EngineException { } } - private boolean maybeFailEngine(String source, Throwable t) { - if (Lucene.isCorruptionException(t)) { - if (engineConfig.isFailEngineOnCorruption()) { - failEngine("corrupt file detected source: [" + source + "]", t); - return true; - } else { - logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, engineConfig.isFailEngineOnCorruption()); - } - } else if (ExceptionsHelper.isOOM(t)) { - failEngine("out of memory", t); + @Override + protected boolean maybeFailEngine(String source, Throwable t) { + boolean shouldFail = super.maybeFailEngine(source, t); + if (shouldFail) { return true; - } else if (t instanceof AlreadyClosedException) { + } + + // Check for AlreadyClosedException + if (t instanceof AlreadyClosedException) { // if we are already closed due to some tragic exception // we need to fail the engine. it might have already been failed before // but we are double-checking it's failed and closed @@ -854,16 +850,6 @@ private boolean maybeFailEngine(String source, Throwable t) { return false; } - private Throwable wrapIfClosed(Throwable t) { - if (isClosed.get()) { - if (t != failedEngine && failedEngine != null) { - t.addSuppressed(failedEngine); - } - return new EngineClosedException(shardId, t); - } - return t; - } - @Override public SegmentsStats segmentsStats() { ensureOpen(); diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 2a8e5a02daa42..9620eb62f47e9 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -213,34 +213,6 @@ public SnapshotIndexCommit snapshotIndex() throws EngineException { } } - // TODO refactor into abstract helper - private boolean maybeFailEngine(Throwable t, String source) { - if (Lucene.isCorruptionException(t)) { - if (engineConfig.isFailEngineOnCorruption()) { - failEngine("corrupt file detected source: [" + source + "]", t); - return true; - } else { - logger.warn("corrupt file detected source: [{}] but [{}] is set to [{}]", t, source, - EngineConfig.INDEX_FAIL_ON_CORRUPTION_SETTING, engineConfig.isFailEngineOnCorruption()); - } - } else if (ExceptionsHelper.isOOM(t)) { - failEngine("out of memory", t); - return true; - } - return false; - } - - // TODO refactor into abstract helper - private Throwable wrapIfClosed(Throwable t) { - if (closedOrFailed) { - if (t != failedEngine && failedEngine != null) { - t.addSuppressed(failedEngine); - } - return new EngineClosedException(shardId, t); - } - return t; - } - @Override public void recover(RecoveryHandler recoveryHandler) throws EngineException { // take a write lock here so it won't happen while a flush is in progress @@ -256,7 +228,7 @@ public void recover(RecoveryHandler recoveryHandler) throws EngineException { try { phase1Snapshot = deletionPolicy.snapshot(); } catch (Throwable e) { - maybeFailEngine(e, "recovery"); + maybeFailEngine("recovery", e); Releasables.closeWhileHandlingException(onGoingRecoveries); throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e); } @@ -266,7 +238,7 @@ public void recover(RecoveryHandler recoveryHandler) throws EngineException { recoveryHandler.phase1(phase1Snapshot); success = true; } catch (Throwable e) { - maybeFailEngine(e, "recovery phase 1"); + maybeFailEngine("recovery phase 1", e); Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot); throw new RecoveryEngineException(shardId, 1, "Execution failed", wrapIfClosed(e)); } finally { From 28a9d1842722acba7ea69e0fa65200444532a30c Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 12 Feb 2015 14:08:25 -0700 Subject: [PATCH 25/56] Remove nocommit, document canDeleteIndexContents --- .../java/org/elasticsearch/indices/IndicesService.java | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/indices/IndicesService.java b/src/main/java/org/elasticsearch/indices/IndicesService.java index 14108fb5efb63..786e5d4172fd0 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -533,7 +533,14 @@ public void deleteShardStore(String reason, ShardId shardId, IndexMetaData metaD logger.trace("{} deleting shard reason [{}]", shardId, reason); } - //NOCOMMIT document this + /** + * This method returns true if the current node is allowed to delete the + * given index. If the index uses a shared filesystem this method always + * returns false. + * @param index {@code Index} to check whether deletion is allowed + * @param indexSettings {@code Settings} for the given index + * @return true if the index can be deleted on this node + */ public boolean canDeleteIndexContents(Index index, Settings indexSettings) { final Tuple indexServiceInjectorTuple = this.indices.get(index); if (IndexMetaData.usesSharedFilesystem(indexSettings) == false) { From 4f71c8d9f706a0c1c39aa3a370efb1604559d928 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 12 Feb 2015 14:17:22 -0700 Subject: [PATCH 26/56] Add documentation to ShadowEngine --- .../index/engine/ShadowEngine.java | 30 ++++++++++++++----- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 9620eb62f47e9..8fcb87ab462b5 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -42,7 +42,22 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; /** - * TODO: document me! + * ShadowEngine is a specialized engine that only allows read-only operations + * on the underlying Lucene index. An {@code IndexReader} is opened instead of + * an {@code IndexWriter}. All methods that would usually perform write + * operations are no-ops, this means: + * + * - No operations are written to or read from the translog + * - Create, Index, and Delete do nothing + * - Flush does not fsync any files, or make any on-disk changes + * + * In order for new segments to become visible, the ShadowEngine may perform + * stage1 of the traditional recovery process (copying segment files) from a + * regular primary (which uses {@link org.elasticsearch.index.engine.InternalEngine}) + * + * Notice that since this Engine does not deal with the translog, any + * {@link #get(Get get)} request goes directly to the searcher, meaning it is + * non-realtime. */ public class ShadowEngine extends Engine { @@ -88,25 +103,25 @@ public ShadowEngine(EngineConfig engineConfig) { @Override public void create(Create create) throws EngineException { // no-op - logger.debug("cowardly refusing to CREATE"); + logger.trace("skipping CREATE on shadow engine"); } @Override public void index(Index index) throws EngineException { // no-op - logger.debug("cowardly refusing to INDEX"); + logger.trace("skipping INDEX on shadow engine"); } @Override public void delete(Delete delete) throws EngineException { // no-op - logger.debug("cowardly refusing to DELETE"); + logger.trace("skipping DELETE on shadow engine"); } @Override public void delete(DeleteByQuery delete) throws EngineException { // no-op - logger.debug("cowardly refusing to DELETE-BY-QUERY"); + logger.trace("skipping DELETE-BY-QUERY on shadow engine"); } @Override @@ -116,7 +131,7 @@ public void flush() throws EngineException { @Override public void flush(boolean force, boolean waitIfOngoing) throws EngineException { - logger.debug("cowardly refusing to FLUSH"); + logger.trace("skipping FLUSH on shadow engine"); // reread the last committed segment infos refresh("flush"); try { @@ -138,7 +153,8 @@ public void forceMerge(boolean flush) { @Override public void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { - logger.debug("cowardly refusing to FORCE_MERGE, since the since the primary will do it"); + // no-op + logger.trace("skipping FORCE-MERGE on shadow engine"); } @Override From ea4e3e58dc6959a92c06d5990276268d586735f3 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 12 Feb 2015 14:26:30 -0700 Subject: [PATCH 27/56] Add documentation to ShadowIndexShard, remove nocommit --- .../index/shard/ShadowIndexShard.java | 21 +++++++++++++++---- 1 file changed, 17 insertions(+), 4 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index f75804632312e..3f65c36e9d1be 100644 --- a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -28,7 +28,6 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.aliases.IndexAliasesService; @@ -38,8 +37,6 @@ import org.elasticsearch.index.cache.query.ShardQueryCache; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; -import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.ShadowEngine; import org.elasticsearch.index.fielddata.IndexFieldDataService; @@ -72,7 +69,11 @@ import java.util.concurrent.CountDownLatch; /** - */ //nocommit document this! + * ShadowIndexShard extends {@link IndexShard} to add file synchronization + * from the primary when a flush happens. It also ensures that a replica being + * promoted to a primary causes the shard to fail, kicking off a re-allocation + * of the primary shard. + */ public class ShadowIndexShard extends IndexShard { private final RecoveryTarget recoveryTarget; private final ClusterService clusterService; @@ -84,6 +85,12 @@ public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, this.clusterService = clusterService; } + /** + * Flush the shard. In a regular {@link org.elasticsearch.index.shard.IndexShard} + * this would usually flush the engine, however, with a shadow replica we + * also may need to sync newly created segments (that were committed on + * the primary) to the shadow replica + */ public void flush(FlushRequest request) throws ElasticsearchException { if (state() == IndexShardState.STARTED) { syncFilesFromPrimary(); @@ -127,6 +134,12 @@ public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, bo } } + /** + * In addition to the regular accounting done in + * {@link IndexShard#routingEntry(org.elasticsearch.cluster.routing.ShardRouting)}, + * if this shadow replica needs to be promoted to a primary, the shard is + * failed in order to allow a new primary to be re-allocated. + */ @Override public IndexShard routingEntry(ShardRouting newRouting) { ShardRouting shardRouting = this.routingEntry(); From aea96920d995dacef294e48e719ba18f1ecf5860 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Feb 2015 09:56:41 +0100 Subject: [PATCH 28/56] revert unneeded changes on Store --- .../org/elasticsearch/index/store/Store.java | 29 +++++++------------ 1 file changed, 10 insertions(+), 19 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/store/Store.java b/src/main/java/org/elasticsearch/index/store/Store.java index b604e2f0f1b37..ecb5ef818e8d1 100644 --- a/src/main/java/org/elasticsearch/index/store/Store.java +++ b/src/main/java/org/elasticsearch/index/store/Store.java @@ -28,7 +28,6 @@ import org.apache.lucene.util.*; import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; @@ -540,10 +539,7 @@ private static final void failIfCorrupted(Directory directory, ShardId shardId) /** * This method deletes every file in this store that is not contained in the given source meta data or is a * legacy checksum file. After the delete it pulls the latest metadata snapshot from the store and compares it - * to the given snapshot. If the snapshots are inconsistent an illegal state exception is thrown. - * - * If the store is part of a shadow replica, extra files are not cleaned up because they could be in use - * by a shared filesystem. + * to the given snapshot. If the snapshots are inconsistent an illegal state exception is thrown * * @param reason the reason for this cleanup operation logged for each deleted file * @param sourceMetaData the metadata used for cleanup. all files in this metadata should be kept around. @@ -555,18 +551,17 @@ public void cleanupAndVerify(String reason, MetadataSnapshot sourceMetaData) thr metadataLock.writeLock().lock(); try { final StoreDirectory dir = directory; - for (String existingFile : dir.listAll()) { - // don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum) - // we also don't want to deleted IndexWriter's write.lock - // files, since it could be a shared filesystem - if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile) && !Store.isEngineLock(existingFile)) { - try { - dir.deleteFile(reason, existingFile); - } catch (Exception e) { - // ignore, we don't really care, will get deleted later on - } + for (String existingFile : dir.listAll()) { + // don't delete snapshot file, or the checksums file (note, this is extra protection since the Store won't delete checksum) + if (!sourceMetaData.contains(existingFile) && !Store.isChecksum(existingFile)) { + try { + dir.deleteFile(reason, existingFile); + dir.deleteFile(existingFile); + } catch (Exception e) { + // ignore, we don't really care, will get deleted later on } } + } final Store.MetadataSnapshot metadataOrEmpty = getMetadata(); verifyAfterCleanup(sourceMetaData, metadataOrEmpty); } finally { @@ -1088,10 +1083,6 @@ public static final boolean isChecksum(String name) { return name.startsWith(CHECKSUMS_PREFIX) || name.endsWith(".cks"); // bwcomapt - .cks used to be a previous checksum file } - public static final boolean isEngineLock(String name) { - return name.equals("write.lock"); - } - /** * Produces a string representation of the given digest value. */ From 213292e0679d8ae1492ea11861178236f4abd8ea Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Fri, 13 Feb 2015 13:58:05 +0100 Subject: [PATCH 29/56] add one more nocommit --- .../elasticsearch/indices/recovery/SharedFSRecoveryHandler.java | 1 + .../org/elasticsearch/index/IndexWithShadowReplicasTests.java | 1 - 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java index 3351893c8e073..784d29ee25d4a 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java @@ -63,6 +63,7 @@ public void phase3(Translog.Snapshot snapshot) throws ElasticsearchException { try { shard.relocated("to " + request.targetNode()); shard.close("relocated on shared filesystem"); + // nocommit we need to figure out a better way how to handle failures in phase3 we might need to restart the engine we recovering from? } catch (IOException e) { } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 2bc4e67abfbc1..88d23c1c36a10 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -49,7 +49,6 @@ * Tests for indices that use shadow replicas and a shared filesystem */ @ElasticsearchIntegrationTest.ClusterScope(scope = ElasticsearchIntegrationTest.Scope.TEST, numDataNodes = 0) -@TestLogging("_root:DEBUG,env:TRACE") //nocommit public class IndexWithShadowReplicasTests extends ElasticsearchIntegrationTest { @Test From e5bc047d7c872ae960d397b1ae7b4b78d6a1ea10 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Fri, 13 Feb 2015 11:38:09 -0700 Subject: [PATCH 30/56] Don't replicate document request when using shadow replicas --- ...nsportShardReplicationOperationAction.java | 19 ++++++++++++++++--- .../index/IndexWithShadowReplicasTests.java | 16 +++++++++------- 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java index 43c2818dc9248..77b6b3ac7b5a6 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java @@ -45,11 +45,11 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.DocumentAlreadyExistsException; import org.elasticsearch.index.engine.VersionConflictEngineException; -import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.rest.RestStatus; @@ -631,9 +631,22 @@ void performOnReplica(final ReplicationState state, final ShardRouting shard, fi } final ReplicaOperationRequest shardRequest = new ReplicaOperationRequest(shardIt.shardId(), state.replicaRequest()); + + // If the replicas use shadow replicas, there is no reason to + // perform the action on the replica, so skip it and + // immediately return + if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings())) { + // nocommit - this doesn't replicate mappings changes, so can fail if mappings are not predefined + // It was successful on the replica, although we never + // actually executed + state.onReplicaSuccess(); + return; + } + if (!nodeId.equals(observer.observedState().nodes().localNodeId())) { final DiscoveryNode node = observer.observedState().nodes().get(nodeId); - transportService.sendRequest(node, transportReplicaAction, shardRequest, transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { + transportService.sendRequest(node, transportReplicaAction, shardRequest, + transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { @Override public void handleResponse(TransportResponse.Empty vResponse) { state.onReplicaSuccess(); diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 88d23c1c36a10..024482e66edf0 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -25,13 +25,15 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.*; +import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.cluster.routing.RoutingNode; +import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; import java.nio.file.Path; @@ -70,7 +72,7 @@ public void testIndexWithFewDocuments() throws Exception { .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .build(); - prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); ensureGreen(IDX); // So basically, the primary should fail and the replica will need to @@ -181,7 +183,7 @@ public void testIndexWithShadowReplicasCleansUp() throws Exception { .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .build(); - prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); ensureGreen(IDX); client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); @@ -225,13 +227,13 @@ public void testShadowReplicaNaturalRelocation() throws Exception { .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) .build(); - prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); ensureGreen(IDX); int docCount = randomIntBetween(10, 100); List builders = newArrayList(); for (int i = 0; i < docCount; i++) { - builders.add(client().prepareIndex(IDX, "doc", i + "").setSource("body", "foo")); + builders.add(client().prepareIndex(IDX, "doc", i + "").setSource("foo", "bar")); } indexRandom(true, true, true, builders); flushAndRefresh(IDX); @@ -277,7 +279,7 @@ public void testSimpleNonSharedFS() throws Exception { .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, false) .build(); - prepareCreate(IDX).setSettings(idxSettings).get(); + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); ensureGreen(IDX); IndexRequestBuilder[] builders = new IndexRequestBuilder[between(10, 20)]; From 2ae80f9689346f8fd346a0d3775a6341874d8bef Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Fri, 13 Feb 2015 16:25:34 -0700 Subject: [PATCH 31/56] throw UnsupportedOperationException on write operations in ShadowEngine --- .../elasticsearch/index/engine/ShadowEngine.java | 12 ++++-------- .../index/engine/ShadowEngineTests.java | 15 ++++++++++++--- 2 files changed, 16 insertions(+), 11 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 8fcb87ab462b5..5e2dfeb1b24df 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -102,26 +102,22 @@ public ShadowEngine(EngineConfig engineConfig) { @Override public void create(Create create) throws EngineException { - // no-op - logger.trace("skipping CREATE on shadow engine"); + throw new UnsupportedOperationException("create operation not allowed on shadow engine"); } @Override public void index(Index index) throws EngineException { - // no-op - logger.trace("skipping INDEX on shadow engine"); + throw new UnsupportedOperationException("index operation not allowed on shadow engine"); } @Override public void delete(Delete delete) throws EngineException { - // no-op - logger.trace("skipping DELETE on shadow engine"); + throw new UnsupportedOperationException("delete operation not allowed on shadow engine"); } @Override public void delete(DeleteByQuery delete) throws EngineException { - // no-op - logger.trace("skipping DELETE-BY-QUERY on shadow engine"); + throw new UnsupportedOperationException("delete-by-query operation not allowed on shadow engine"); } @Override diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 50bc4eb968d5e..3c3896a68f1f6 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -475,7 +475,10 @@ public void testShadowEngineIgnoresWriteOperations() throws Exception { ParseContext.Document document = testDocumentWithTextField(); document.add(new Field(SourceFieldMapper.NAME, B_1.toBytes(), SourceFieldMapper.Defaults.FIELD_TYPE)); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); - replicaEngine.create(new Engine.Create(null, newUid("1"), doc)); + try { + replicaEngine.create(new Engine.Create(null, newUid("1"), doc)); + fail("should have thrown an exception"); + } catch (UnsupportedOperationException e) {} replicaEngine.refresh("test"); // its not there... @@ -491,7 +494,10 @@ public void testShadowEngineIgnoresWriteOperations() throws Exception { document = testDocument(); document.add(new TextField("value", "test1", Field.Store.YES)); doc = testParsedDocument("1", "1", "test", null, -1, -1, document, B_1, false); - replicaEngine.index(new Engine.Index(null, newUid("1"), doc)); + try { + replicaEngine.index(new Engine.Index(null, newUid("1"), doc)); + fail("should have thrown an exception"); + } catch (UnsupportedOperationException e) {} replicaEngine.refresh("test"); // its still not there... @@ -524,7 +530,10 @@ public void testShadowEngineIgnoresWriteOperations() throws Exception { getResult.release(); // try to delete it on the replica - replicaEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + try { + replicaEngine.delete(new Engine.Delete("test", "1", newUid("1"))); + fail("should have thrown an exception"); + } catch (UnsupportedOperationException e) {} replicaEngine.flush(); replicaEngine.refresh("test"); primaryEngine.refresh("test"); From 28f069b6d99a65e285ac8c821e6a332a1d8eb315 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 16 Feb 2015 16:43:46 +0100 Subject: [PATCH 32/56] fix primary relocation --- .../recovery/SharedFSRecoveryHandler.java | 23 +++---- .../index/IndexWithShadowReplicasTests.java | 65 +++++++++++++++++++ .../test/store/MockFSDirectoryService.java | 2 +- 3 files changed, 76 insertions(+), 14 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java index 784d29ee25d4a..fbadb276eae56 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/SharedFSRecoveryHandler.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.common.logging.ESLogger; @@ -48,26 +49,22 @@ public SharedFSRecoveryHandler(IndexShard shard, StartRecoveryRequest request, R @Override public void phase1(SnapshotIndexCommit snapshot) throws ElasticsearchException { + if (request.recoveryType() == RecoveryState.Type.RELOCATION && shard.routingEntry().primary()) { + // here we simply fail the primary shard since we can't move them (have 2 writers open at the same time) + // by failing the shard we play safe and just go through the entire reallocation procedure of the primary + // it would be ideal to make sure we flushed the translog here but that is not possible in the current design. + ElasticsearchIllegalStateException exception = new ElasticsearchIllegalStateException("Can't relocate primary - failing"); + shard.failShard("primary_relocation", exception); + throw exception; + } logger.trace("{} recovery [phase2] to {}: skipping phase 1 for shared filesystem", request.shardId(), request.targetNode()); } + @Override protected int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { logger.trace("{} recovery [phase3] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode()); return 0; } - @Override - public void phase3(Translog.Snapshot snapshot) throws ElasticsearchException { - if (request.markAsRelocated()) { - try { - shard.relocated("to " + request.targetNode()); - shard.close("relocated on shared filesystem"); - // nocommit we need to figure out a better way how to handle failures in phase3 we might need to restart the engine we recovering from? - } catch (IOException e) { - - } - } - super.phase3(snapshot); - } } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 024482e66edf0..8cf8fc9fa67cb 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.SearchHit; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; import java.nio.file.Path; @@ -163,6 +164,70 @@ public void testReplicaToPrimaryPromotion() throws Exception { assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); } + @Test + public void testPrimaryRelocation() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + .build(); + + String node1 = internalCluster().startNode(nodeSettings); + Path dataPath = newTempDirPath(); + String IDX = "test"; + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); + ensureYellow(IDX); + client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertTrue(gResp1.isExists()); + assertTrue(gResp2.isExists()); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + + // Node1 has the primary, now node2 has the replica + String node2 = internalCluster().startNode(nodeSettings); + ensureGreen(IDX); + client().admin().cluster().prepareHealth().setWaitForNodes("2").get(); + flushAndRefresh(IDX); + + // now prevent primary from being allocated on node 1 move to node_3 + String node3 = internalCluster().startNode(nodeSettings); + Settings build = ImmutableSettings.builder().put("index.routing.allocation.exclude._name", node1).build(); + client().admin().indices().prepareUpdateSettings(IDX).setSettings(build).execute().actionGet(); + + ensureGreen(IDX); + logger.info("--> performing query"); + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); + assertHitCount(resp, 2); + + gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); + gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + assertTrue(gResp1.isExists()); + assertTrue(gResp2.toString(), gResp2.isExists()); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + + client().prepareIndex(IDX, "doc", "3").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "4").setSource("foo", "bar").get(); + gResp1 = client().prepareGet(IDX, "doc", "3").setPreference("_primary").setFields("foo").get(); + gResp2 = client().prepareGet(IDX, "doc", "4").setPreference("_primary").setFields("foo").get(); + assertTrue(gResp1.isExists()); + assertTrue(gResp2.isExists()); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + } + @Test public void testIndexWithShadowReplicasCleansUp() throws Exception { Settings nodeSettings = ImmutableSettings.builder() diff --git a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java index 651d4787f16c3..205ed928e30ed 100644 --- a/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/src/test/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -86,12 +86,12 @@ public void beforeIndexShardClosed(ShardId sid, @Nullable IndexShard indexShard, if (indexShard != null && shardId.equals(sid)) { logger.info("Shard state before potentially flushing is {}", indexShard.state()); if (validCheckIndexStates.contains(indexShard.state()) && indexShard.engine() instanceof InternalEngine) { - canRun = true; // When the the internal engine closes we do a rollback, which removes uncommitted segments // By doing a commit flush we perform a Lucene commit, but don't clear the translog, // so that even in tests where don't flush we can check the integrity of the Lucene index Releasables.close(indexShard.engine().snapshotIndex()); // Keep translog for tests that rely on replaying it logger.info("flush finished in beforeIndexShardClosed"); + canRun = true; } } } From 82b9f0449108cd4741568d9b4495bf6c10a5b019 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 16 Feb 2015 16:45:27 +0100 Subject: [PATCH 33/56] reduce the changes compared to master --- .../elasticsearch/index/shard/IndexShard.java | 29 +++++-------------- 1 file changed, 7 insertions(+), 22 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 1468ddb2b57c6..90cf5f1ceb77c 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -34,10 +34,6 @@ import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.common.Booleans; @@ -48,7 +44,6 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.metrics.MeanMetric; import org.elasticsearch.common.settings.Settings; @@ -58,6 +53,7 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.aliases.IndexAliasesService; +import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.filter.FilterCacheStats; @@ -92,9 +88,9 @@ import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; -import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreStats; +import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.suggest.stats.ShardSuggestService; import org.elasticsearch.index.suggest.stats.SuggestStats; import org.elasticsearch.index.termvectors.ShardTermVectorsService; @@ -105,10 +101,7 @@ import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.InternalIndicesLifecycle; -import org.elasticsearch.indices.cluster.IndicesClusterStateService; -import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; -import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.search.suggest.completion.Completion090PostingsFormat; import org.elasticsearch.search.suggest.completion.CompletionStats; import org.elasticsearch.threadpool.ThreadPool; @@ -118,7 +111,6 @@ import java.nio.channels.ClosedByInterruptException; import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -183,18 +175,11 @@ public class IndexShard extends AbstractIndexShardComponent { private final MapperAnalyzer mapperAnalyzer; @Inject - public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, - IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, - ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, - IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, - ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, - ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, - PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, - CodecService codecService, ShardTermVectorsService termVectorsService, - IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, - ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, - @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, - SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { + public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, + ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, + ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, + ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, + @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, AnalysisService analysisService, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { super(shardId, indexSettings); Preconditions.checkNotNull(store, "Store must be provided to the index shard"); Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard"); From abb696563a9e418d3f842a790fcb832f91150be2 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Mon, 16 Feb 2015 17:05:02 +0100 Subject: [PATCH 34/56] remove nocommit and simplify delete logic --- .../org/elasticsearch/index/IndexService.java | 22 +++++++------------ .../elasticsearch/index/shard/IndexShard.java | 2 +- .../elasticsearch/indices/IndicesService.java | 21 ++++++------------ 3 files changed, 16 insertions(+), 29 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/IndexService.java b/src/main/java/org/elasticsearch/index/IndexService.java index d20f198b6c1c6..164e4aac9d2c9 100644 --- a/src/main/java/org/elasticsearch/index/IndexService.java +++ b/src/main/java/org/elasticsearch/index/IndexService.java @@ -304,7 +304,8 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) throws indicesLifecycle.beforeIndexShardCreated(shardId, indexSettings); logger.debug("creating shard_id {}", shardId); - + // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary. + final boolean ownsShard = IndexMetaData.usesSharedFilesystem(indexSettings) == false || (primary && IndexMetaData.usesSharedFilesystem(indexSettings)); ModulesBuilder modules = new ModulesBuilder(); modules.add(new ShardsPluginsModule(indexSettings, pluginsService)); modules.add(new IndexShardModule(shardId, primary, indexSettings)); @@ -312,7 +313,7 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) throws modules.add(new ShardSearchModule()); modules.add(new ShardGetModule()); modules.add(new StoreModule(indexSettings, injector.getInstance(IndexStore.class), lock, - new StoreCloseListener(shardId))); + new StoreCloseListener(shardId, ownsShard))); modules.add(new DeletionPolicyModule(indexSettings)); modules.add(new MergePolicyModule(indexSettings)); modules.add(new MergeSchedulerModule(indexSettings)); @@ -403,12 +404,6 @@ private void closeShardInjector(String reason, ShardId sId, Injector shardInject // call this before we close the store, so we can release resources for it indicesLifecycle.afterIndexShardClosed(sId, indexShard, indexSettings); - if (indexShard.routingEntry().primary()) { - StoreCloseListener instance = (StoreCloseListener) shardInjector.getInstance(Store.OnClose.class); - // we are the primary - we own the shard from a writing perspective - // NOCOMMIT can we make this even nicer? - instance.setOwnsShard(true); - } } finally { try { shardInjector.getInstance(Store.class).close(); @@ -439,7 +434,9 @@ private void closeInjectorResource(ShardId shardId, Injector shardInjector, Clas private void onShardClose(ShardLock lock, boolean ownsShard) { if (deleted.get()) { // we remove that shards content if this index has been deleted try { - indicesServices.deleteShardStore("delete index", lock, indexSettings, ownsShard); + if (ownsShard) { + indicesServices.deleteShardStore("delete index", lock, indexSettings); + } } catch (IOException e) { logger.warn("{} failed to delete shard content", e, lock.getShardId()); } @@ -448,13 +445,10 @@ private void onShardClose(ShardLock lock, boolean ownsShard) { private class StoreCloseListener implements Store.OnClose { private final ShardId shardId; - private volatile boolean ownsShard = false; + private final boolean ownsShard; - public StoreCloseListener(ShardId shardId) { + public StoreCloseListener(ShardId shardId, boolean ownsShard) { this.shardId = shardId; - } - - public void setOwnsShard(boolean ownsShard) { this.ownsShard = ownsShard; } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 90cf5f1ceb77c..f6fd9b71b29a1 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -179,7 +179,7 @@ public IndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexS ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, - @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, AnalysisService analysisService, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { + @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory) { super(shardId, indexSettings); Preconditions.checkNotNull(store, "Store must be provided to the index shard"); Preconditions.checkNotNull(deletionPolicy, "Snapshot deletion policy must be provided to the index shard"); diff --git a/src/main/java/org/elasticsearch/indices/IndicesService.java b/src/main/java/org/elasticsearch/indices/IndicesService.java index 786e5d4172fd0..69e3192d392ab 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -502,17 +502,10 @@ private void deleteIndexStore(String reason, Index index, Settings indexSettings * @param indexSettings the shards index settings. * @throws IOException if an IOException occurs */ - public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings, boolean ownsShard) throws IOException { + public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException { ShardId shardId = lock.getShardId(); - if (canDeleteShardContent(shardId, indexSettings, ownsShard) == false) { - if (canDeleteShardContent(shardId, indexSettings, true)) { - throw new ElasticsearchIllegalStateException("Can't delete shard " + shardId + " ownsShard: " + ownsShard); - } - logger.trace("{} skip deleting shard shard ownsShard: {} ", shardId, ownsShard); - } else { - logger.trace("{} deleting shard reason [{}]", shardId, reason); - nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings); - } + logger.trace("{} deleting shard reason [{}]", shardId, reason); + nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings); } /** @@ -526,7 +519,7 @@ public void deleteShardStore(String reason, ShardLock lock, Settings indexSettin */ public void deleteShardStore(String reason, ShardId shardId, IndexMetaData metaData) throws IOException { final Settings indexSettings = buildIndexSettings(metaData); - if (canDeleteShardContent(shardId, indexSettings, false) == false) { + if (canDeleteShardContent(shardId, indexSettings) == false) { throw new ElasticsearchIllegalStateException("Can't delete shard " + shardId); } nodeEnv.deleteShardDirectorySafe(shardId, indexSettings); @@ -571,12 +564,12 @@ public boolean canDeleteShardContent(ShardId shardId, IndexMetaData metaData) { // The plan was to make it harder to miss-use and ask for metadata instead of simple settings assert shardId.getIndex().equals(metaData.getIndex()); final Settings indexSettings = buildIndexSettings(metaData); - return canDeleteShardContent(shardId, indexSettings, false); + return canDeleteShardContent(shardId, indexSettings); } - private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings, boolean ownsShard) { + private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings) { final Tuple indexServiceInjectorTuple = this.indices.get(shardId.getIndex()); - if (IndexMetaData.usesSharedFilesystem(indexSettings) == false || ownsShard) { + if (IndexMetaData.usesSharedFilesystem(indexSettings) == false) { if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) { final IndexService indexService = indexServiceInjectorTuple.v1(); return indexService.hasShard(shardId.id()) == false; From d77414c5e7b2cde830a8e3f70fe463ccc904d4d0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 10:27:56 +0100 Subject: [PATCH 35/56] remove nocommits in IndexMetaData --- .../cluster/metadata/IndexMetaData.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index da265fb435afe..f53853eacafb5 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -787,12 +787,20 @@ public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws } } - // NOCOMMIT find a good place for this and document it + /** + * Returns true iff the given settings indicate that the index associated + * with these settings allocates it's shards on a shared filesystem. Otherwise false. The default + * setting for this is the returned value from {@link #isIndexUsingShadowReplicas(org.elasticsearch.common.settings.Settings)}. + */ public static boolean usesSharedFilesystem(Settings settings) { return settings.getAsBoolean(SETTING_SHARED_FILESYSTEM, isIndexUsingShadowReplicas(settings)); } - // NOCOMMIT find a good place for this and document it + /** + * Returns true iff the given settings indicate that the index associated + * with these settings uses shadow replicas. Otherwise false. The default + * setting for this is false. + */ public static boolean isIndexUsingShadowReplicas(Settings settings) { return settings.getAsBoolean(SETTING_SHADOW_REPLICAS, false); } From 48a700d23cff117b8e4851d4008364f92b8272a0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 10:50:59 +0100 Subject: [PATCH 36/56] add test for failing shadow engine / remove nocommit --- .../elasticsearch/index/engine/Engine.java | 59 ++++++++++++++- .../index/engine/InternalEngine.java | 59 +-------------- .../index/engine/ShadowEngine.java | 72 ++++--------------- .../index/store/DirectoryUtils.java | 7 +- .../index/engine/ShadowEngineTests.java | 40 +++++++++-- .../index/store/DirectoryUtilsTest.java | 8 +-- .../test/engine/MockInternalEngine.java | 3 +- 7 files changed, 116 insertions(+), 132 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 987ef8c2402c3..244367b605f7c 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -59,6 +59,7 @@ import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; /** * @@ -72,6 +73,10 @@ public abstract class Engine implements Closeable { protected final AtomicBoolean isClosed = new AtomicBoolean(false); protected final FailedEngineListener failedEngineListener; protected final SnapshotDeletionPolicy deletionPolicy; + protected final ReentrantLock failEngineLock = new ReentrantLock(); + protected final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); + protected final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); + protected final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); protected volatile Throwable failedEngine = null; @@ -415,8 +420,45 @@ public final boolean refreshNeeded() { public abstract void recover(RecoveryHandler recoveryHandler) throws EngineException; - /** fail engine due to some error. the engine will also be closed. */ - public abstract void failEngine(String reason, Throwable failure); + public void failEngine(String reason, Throwable failure) { + assert failure != null; + if (failEngineLock.tryLock()) { + store.incRef(); + try { + try { + // we just go and close this engine - no way to recover + closeNoLock("engine failed on: [" + reason + "]"); + // we first mark the store as corrupted before we notify any listeners + // this must happen first otherwise we might try to reallocate so quickly + // on the same node that we don't see the corrupted marker file when + // the shard is initializing + if (Lucene.isCorruptionException(failure)) { + try { + store.markStoreCorrupted(ExceptionsHelper.unwrapCorruption(failure)); + } catch (IOException e) { + logger.warn("Couldn't marks store corrupted", e); + } + } + } finally { + if (failedEngine != null) { + logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); + return; + } + logger.warn("failed engine [{}]", failure, reason); + // we must set a failure exception, generate one if not supplied + failedEngine = failure; + failedEngineListener.onFailedEngine(shardId, reason, failure); + } + } catch (Throwable t) { + // don't bubble up these exceptions up + logger.warn("failEngine threw exception", t); + } finally { + store.decRef(); + } + } else { + logger.debug("tried to fail engine but could not acquire lock - engine should be failed by now [{}]", reason, failure); + } + } /** Check whether the engine should be failed */ protected boolean maybeFailEngine(String source, Throwable t) { @@ -963,4 +1005,17 @@ public void release() { } protected abstract SearcherManager getSearcherManager(); + + protected abstract void closeNoLock(String reason) throws ElasticsearchException; + + @Override + public void close() throws IOException { + if (isClosed.get() == false) { // don't acquire the write lock if we are already closed + logger.debug("close now acquiring writeLock"); + try (ReleasableLock _ = writeLock.acquire()) { + logger.debug("close acquired writeLock"); + closeNoLock("api"); + } + } + } } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index 3879eae7f2937..c8b925fc97b4a 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -80,10 +80,6 @@ public class InternalEngine extends Engine { private final MergePolicyProvider mergePolicyProvider; private final MergeSchedulerProvider mergeScheduler; - private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); - private final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); - private final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); - private final IndexWriter indexWriter; private final SearcherFactory searcherFactory; @@ -101,7 +97,6 @@ public class InternalEngine extends Engine { private final LiveVersionMap versionMap; private final Object[] dirtyLocks; - private final ReentrantLock failEngineLock = new ReentrantLock(); private final AtomicLong translogIdGenerator = new AtomicLong(); private final AtomicBoolean versionMapRefreshPending = new AtomicBoolean(); @@ -892,23 +887,12 @@ public List segments(boolean verbose) { } } - @Override - public void close() throws ElasticsearchException { - if (isClosed.get() == false) { // don't acquire the write lock if we are already closed - logger.trace("close now acquire writeLock"); - try (ReleasableLock _ = writeLock.acquire()) { - logger.trace("close now acquired writeLock"); - closeNoLock("api"); - } - } - } - /** * Closes the engine without acquiring the write lock. This should only be * called while the write lock is hold or in a disaster condition ie. if the engine * is failed. */ - private void closeNoLock(String reason) throws ElasticsearchException { + protected final void closeNoLock(String reason) throws ElasticsearchException { if (isClosed.compareAndSet(false, true)) { assert rwl.isWriteLockedByCurrentThread() || failEngineLock.isHeldByCurrentThread() : "Either the write lock must be held or the engine must be currently be failing itself"; try { @@ -938,47 +922,6 @@ private void closeNoLock(String reason) throws ElasticsearchException { } } - @Override - public void failEngine(String reason, Throwable failure) { - assert failure != null; - if (failEngineLock.tryLock()) { - store.incRef(); - try { - try { - // we just go and close this engine - no way to recover - closeNoLock("engine failed on: [" + reason + "]"); - // we first mark the store as corrupted before we notify any listeners - // this must happen first otherwise we might try to reallocate so quickly - // on the same node that we don't see the corrupted marker file when - // the shard is initializing - if (Lucene.isCorruptionException(failure)) { - try { - store.markStoreCorrupted(ExceptionsHelper.unwrapCorruption(failure)); - } catch (IOException e) { - logger.warn("Couldn't marks store corrupted", e); - } - } - } finally { - if (failedEngine != null) { - logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); - return; - } - logger.warn("failed engine [{}]", failure, reason); - // we must set a failure exception, generate one if not supplied - failedEngine = failure; - failedEngineListener.onFailedEngine(shardId, reason, failure); - } - } catch (Throwable t) { - // don't bubble up these exceptions up - logger.warn("failEngine threw exception", t); - } finally { - store.decRef(); - } - } else { - logger.debug("tried to fail engine but could not acquire lock - engine should be failed by now [{}]", reason, failure); - } - } - @Override protected SearcherManager getSearcherManager() { return searcherManager; diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 5e2dfeb1b24df..4b304f5f6bbe0 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -27,6 +27,7 @@ import org.apache.lucene.search.SearcherManager; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.util.IOUtils; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lucene.Lucene; @@ -61,13 +62,8 @@ */ public class ShadowEngine extends Engine { - private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); - private final ReleasableLock readLock = new ReleasableLock(rwl.readLock()); - private final ReleasableLock writeLock = new ReleasableLock(rwl.writeLock()); - private final Lock failReleasableLock = new ReentrantLock(); private final RecoveryCounter onGoingRecoveries; - private volatile boolean closedOrFailed = false; private volatile SearcherManager searcherManager; private SegmentInfos lastCommittedSegmentInfos; @@ -230,9 +226,7 @@ public void recover(RecoveryHandler recoveryHandler) throws EngineException { // take a write lock here so it won't happen while a flush is in progress // this means that next commits will not be allowed once the lock is released try (ReleasableLock _ = writeLock.acquire()) { - if (closedOrFailed) { - throw new EngineClosedException(shardId, failedEngine); - } + ensureOpen(); onGoingRecoveries.startRecovery(); } @@ -261,64 +255,22 @@ public void recover(RecoveryHandler recoveryHandler) throws EngineException { // engine, there is no phase2 and phase3 of recovery } - @Override - public void failEngine(String reason, Throwable failure) { - // Note, there is no IndexWriter, so nothing to rollback here - assert failure != null; - if (failReleasableLock.tryLock()) { - try { - try { - // we first mark the store as corrupted before we notify any listeners - // this must happen first otherwise we might try to reallocate so quickly - // on the same node that we don't see the corrupted marker file when - // the shard is initializing - if (Lucene.isCorruptionException(failure)) { - try { - store.markStoreCorrupted(ExceptionsHelper.unwrapCorruption(failure)); - } catch (IOException e) { - logger.warn("Couldn't marks store corrupted", e); - } - } - } finally { - if (failedEngine != null) { - logger.debug("tried to fail engine but engine is already failed. ignoring. [{}]", reason, failure); - return; - } - logger.warn("failed engine [{}]", failure, reason); - // we must set a failure exception, generate one if not supplied - failedEngine = failure; - failedEngineListener.onFailedEngine(shardId, reason, failure); - } - } catch (Throwable t) { - // don't bubble up these exceptions up - logger.warn("failEngine threw exception", t); - } finally { - closedOrFailed = true; - } - } else { - logger.debug("tried to fail engine but could not acquire lock - engine should be failed by now [{}]", reason, failure); - } - } - @Override protected SearcherManager getSearcherManager() { return searcherManager; } + @Override - public void close() throws IOException { - logger.debug("shadow replica close now acquiring writeLock"); - try (ReleasableLock _ = writeLock.acquire()) { - logger.debug("shadow replica close acquired writeLock"); - if (isClosed.compareAndSet(false, true)) { - try { - logger.debug("shadow replica close searcher manager refCount: {}", store.refCount()); - IOUtils.close(searcherManager); - } catch (Throwable t) { - logger.warn("shadow replica failed to close searcher manager", t); - } finally { - store.decRef(); - } + protected void closeNoLock(String reason) throws ElasticsearchException { + if (isClosed.compareAndSet(false, true)) { + try { + logger.debug("shadow replica close searcher manager refCount: {}", store.refCount()); + IOUtils.close(searcherManager); + } catch (Throwable t) { + logger.warn("shadow replica failed to close searcher manager", t); + } finally { + store.decRef(); } } } diff --git a/src/main/java/org/elasticsearch/index/store/DirectoryUtils.java b/src/main/java/org/elasticsearch/index/store/DirectoryUtils.java index d06872a4dd202..ffa69c76ec78a 100644 --- a/src/main/java/org/elasticsearch/index/store/DirectoryUtils.java +++ b/src/main/java/org/elasticsearch/index/store/DirectoryUtils.java @@ -31,10 +31,13 @@ public final class DirectoryUtils { private DirectoryUtils() {} // no instance - static final Directory getLeafDirectory(FilterDirectory dir) { + static final Directory getLeafDirectory(FilterDirectory dir, Class targetClass) { Directory current = dir.getDelegate(); while (true) { if ((current instanceof FilterDirectory)) { + if (targetClass != null && targetClass.isAssignableFrom(current.getClass())) { + break; + } current = ((FilterDirectory) current).getDelegate(); } else { break; @@ -59,7 +62,7 @@ public static T getLeaf(Directory dir, Class targetClas public static T getLeaf(Directory dir, Class targetClass, T defaultValue) { Directory d = dir; if (dir instanceof FilterDirectory) { - d = getLeafDirectory((FilterDirectory) dir); + d = getLeafDirectory((FilterDirectory) dir, targetClass); } if (d instanceof FileSwitchDirectory) { T leaf = getLeaf(((FileSwitchDirectory) d).getPrimaryDir(), targetClass); diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 3c3896a68f1f6..0e8f59bc1f747 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -24,12 +24,10 @@ import org.apache.lucene.document.Field; import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.TextField; -import org.apache.lucene.index.IndexDeletionPolicy; -import org.apache.lucene.index.IndexWriterConfig; -import org.apache.lucene.index.LiveIndexWriterConfig; -import org.apache.lucene.index.Term; +import org.apache.lucene.index.*; import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.MockDirectoryWrapper; import org.elasticsearch.Version; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -42,6 +40,7 @@ import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.deletionpolicy.KeepOnlyLastDeletionPolicy; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit; import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.slowlog.ShardSlowLogIndexingService; import org.elasticsearch.index.mapper.ParseContext; @@ -56,6 +55,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.store.DirectoryService; +import org.elasticsearch.index.store.DirectoryUtils; import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.distributor.LeastUsedDistributor; import org.elasticsearch.index.translog.Translog; @@ -825,7 +825,37 @@ public void testSearchResultRelease() throws Exception { @Test public void testFailEngineOnCorruption() { - // nocommit - figure out how to implement me for shadow replica + ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); + primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); + primaryEngine.flush(); + final boolean failEngine = replicaEngine.config().isFailEngineOnCorruption(); + MockDirectoryWrapper leaf = DirectoryUtils.getLeaf(replicaEngine.config().getStore().directory(), MockDirectoryWrapper.class); + leaf.setRandomIOExceptionRate(1.0); + leaf.setRandomIOExceptionRateOnOpen(1.0); + try { + replicaEngine.refresh("foo"); + fail("exception expected"); + } catch (Exception ex) { + + } + try { + Engine.Searcher searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); + searchResult.close(); + + ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); + primaryEngine.create(new Engine.Create(null, newUid("2"), doc2)); + primaryEngine.refresh("foo"); + + searchResult = replicaEngine.acquireSearcher("test"); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 2)); + MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(2)); + searchResult.close(); + assertThat(failEngine, is(false)); + } catch (EngineClosedException ex) { + assertThat(failEngine, is(true)); + } } @Test diff --git a/src/test/java/org/elasticsearch/index/store/DirectoryUtilsTest.java b/src/test/java/org/elasticsearch/index/store/DirectoryUtilsTest.java index 7f0302356395c..7a9ee113a15d9 100644 --- a/src/test/java/org/elasticsearch/index/store/DirectoryUtilsTest.java +++ b/src/test/java/org/elasticsearch/index/store/DirectoryUtilsTest.java @@ -43,7 +43,7 @@ public void testGetLeave() throws IOException { BaseDirectoryWrapper dir = newFSDirectory(file); FSDirectory directory = DirectoryUtils.getLeaf(new FilterDirectory(dir) {}, FSDirectory.class, null); assertThat(directory, notNullValue()); - assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir))); + assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir, null))); dir.close(); } @@ -51,7 +51,7 @@ public void testGetLeave() throws IOException { BaseDirectoryWrapper dir = newFSDirectory(file); FSDirectory directory = DirectoryUtils.getLeaf(dir, FSDirectory.class, null); assertThat(directory, notNullValue()); - assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir))); + assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir, null))); dir.close(); } @@ -60,7 +60,7 @@ public void testGetLeave() throws IOException { BaseDirectoryWrapper dir = newFSDirectory(file); FSDirectory directory = DirectoryUtils.getLeaf(new FileSwitchDirectory(stringSet, dir, dir, random().nextBoolean()), FSDirectory.class, null); assertThat(directory, notNullValue()); - assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir))); + assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir, null))); dir.close(); } @@ -69,7 +69,7 @@ public void testGetLeave() throws IOException { BaseDirectoryWrapper dir = newFSDirectory(file); FSDirectory directory = DirectoryUtils.getLeaf(new FilterDirectory(new FileSwitchDirectory(stringSet, dir, dir, random().nextBoolean())) {}, FSDirectory.class, null); assertThat(directory, notNullValue()); - assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir))); + assertThat(directory, sameInstance(DirectoryUtils.getLeafDirectory(dir, null))); dir.close(); } diff --git a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java index a710a2e3933fe..e9e1037dff7f5 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java +++ b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java @@ -33,6 +33,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; +import java.io.IOException; import java.lang.reflect.Constructor; import java.util.Map; import java.util.Random; @@ -76,7 +77,7 @@ public MockInternalEngine(EngineConfig config) throws EngineException { } @Override - public void close() { + public void close() throws IOException { try { super.close(); } finally { From 11886b7653dabc23655ec76d112f291301f98f4a Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 11:26:48 +0100 Subject: [PATCH 37/56] Back out non-shared FS code. this will go in in a second iteration --- .../index/shard/ShadowIndexShard.java | 62 ------------- .../cluster/IndicesClusterStateService.java | 32 ++++--- .../recovery/FileSyncRecoveryHandler.java | 56 ------------ .../indices/recovery/RecoverySource.java | 6 +- .../indices/recovery/RecoveryState.java | 3 +- .../indices/recovery/RecoveryTarget.java | 11 --- .../recovery/ShardRecoveryHandler.java | 4 - .../index/IndexWithShadowReplicasTests.java | 88 ------------------- 8 files changed, 17 insertions(+), 245 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java diff --git a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index 3f65c36e9d1be..809b27be6c1f0 100644 --- a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -75,63 +75,10 @@ * of the primary shard. */ public class ShadowIndexShard extends IndexShard { - private final RecoveryTarget recoveryTarget; - private final ClusterService clusterService; @Inject public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, RecoveryTarget recoveryTarget, ClusterService clusterService) { super(shardId, indexSettings, indexSettingsService, indicesLifecycle, store, mergeScheduler, translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, indexingService, getService, searchService, shardWarmerService, shardFilterCache, shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService, termVectorsService, indexFieldDataService, indexService, shardSuggestService, shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService, mergePolicyProvider, factory); - this.recoveryTarget = recoveryTarget; - this.clusterService = clusterService; - } - - /** - * Flush the shard. In a regular {@link org.elasticsearch.index.shard.IndexShard} - * this would usually flush the engine, however, with a shadow replica we - * also may need to sync newly created segments (that were committed on - * the primary) to the shadow replica - */ - public void flush(FlushRequest request) throws ElasticsearchException { - if (state() == IndexShardState.STARTED) { - syncFilesFromPrimary(); - } - super.flush(request); - } - - private void syncFilesFromPrimary() { - final ShardRouting shardRouting = routingEntry(); - if (IndexMetaData.usesSharedFilesystem(indexSettings()) == false && shardRouting.primary() == false) { - // nocommit - we are running a full recovery here I wonder if we should do this only do this if request.waitIfOngoing() == true? Or if we need a new parameter? - // I also wonder if we want to have an infrastructure for this instead that communicates with the primary etc? - ClusterState state = clusterService.state(); - final CountDownLatch latch = new CountDownLatch(1); - DiscoveryNode sourceNode = IndicesClusterStateService.findSourceNodeForPeerRecovery(state.routingTable(), state.nodes(), shardRouting, logger); - if (sourceNode != null) { - assert engine() instanceof ShadowEngine; - recoveryTarget.startFileSync(this, sourceNode, new RecoveryTarget.RecoveryListener() { - @Override - public void onRecoveryDone(RecoveryState state) { - latch.countDown(); - logger.info("shadow replica catchup done {}", state); - // nocommit - } - - @Override - public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, boolean sendShardFailure) { - latch.countDown(); - logger.warn(" failed to catch up shadow replica can't find source node", e); - //nocommit - } - }); - try { - latch.await(); - } catch (InterruptedException e) { - // ignore - } - } else { - logger.warn(" failed to catch up shadow replica can't find source node", shardId); - } - } } /** @@ -153,13 +100,4 @@ public IndexShard routingEntry(ShardRouting newRouting) { } return this; } - - @Override - public void performRecoveryFinalization(boolean withFlush, RecoveryState recoveryState) throws ElasticsearchException { - if (recoveryState.getType() == RecoveryState.Type.FILE_SYNC) { - logger.debug("skipping recovery finalization file sync runs on a started engine"); - } else { - super.performRecoveryFinalization(withFlush, recoveryState); - } - } } diff --git a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index e2b36279c0852..b8972a6b82e57 100644 --- a/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -44,12 +44,10 @@ import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.compress.CompressedString; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.env.NodeEnvironment; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexShardAlreadyExistsException; import org.elasticsearch.index.IndexShardMissingException; import org.elasticsearch.index.aliases.IndexAlias; @@ -59,20 +57,18 @@ import org.elasticsearch.index.gateway.IndexShardGatewayService; import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.settings.IndexSettingsService; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryFailedException; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.threadpool.ThreadPool; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; +import java.util.*; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicLong; @@ -99,6 +95,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent failedShards = ConcurrentCollections.newConcurrentMap(); + private final NodeEnvironment nodeEnvironment; static class FailedShard { public final long version; @@ -114,13 +111,15 @@ static class FailedShard { private final FailedEngineHandler failedEngineHandler = new FailedEngineHandler(); private final boolean sendRefreshMapping; + private final AtomicLong recoveryIdGenerator = new AtomicLong(); @Inject public IndicesClusterStateService(Settings settings, IndicesService indicesService, ClusterService clusterService, ThreadPool threadPool, RecoveryTarget recoveryTarget, ShardStateAction shardStateAction, NodeIndexDeletedAction nodeIndexDeletedAction, - NodeMappingRefreshAction nodeMappingRefreshAction) { + NodeMappingRefreshAction nodeMappingRefreshAction, + NodeEnvironment nodeEnvironment) { super(settings); this.indicesService = indicesService; this.clusterService = clusterService; @@ -131,6 +130,7 @@ public IndicesClusterStateService(Settings settings, IndicesService indicesServi this.nodeMappingRefreshAction = nodeMappingRefreshAction; this.sendRefreshMapping = componentSettings.getAsBoolean("send_refresh_mapping", true); + this.nodeEnvironment = nodeEnvironment; } @Override @@ -553,7 +553,7 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela continue; } - final IndexShard indexShard = indexService.shard(shardId); + IndexShard indexShard = indexService.shard(shardId); if (indexShard != null) { ShardRouting currentRoutingEntry = indexShard.routingEntry(); // if the current and global routing are initializing, but are still not the same, its a different "shard" being allocated @@ -570,10 +570,9 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela RecoveryState recoveryState = recoveryTarget.recoveryState(indexShard); if (recoveryState != null && recoveryState.getStage() != RecoveryState.Stage.DONE) { // we have an ongoing recovery, find the source based on current routing and compare them - DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting, logger); + DiscoveryNode sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting); if (!recoveryState.getSourceNode().equals(sourceNode)) { - logger.debug("[{}][{}] removing shard (recovery source changed), current [{}], global [{}])", - shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting); + logger.debug("[{}][{}] removing shard (recovery source changed), current [{}], global [{}])", shardRouting.index(), shardRouting.id(), currentRoutingEntry, shardRouting); // closing the shard will also cancel any ongoing recovery. indexService.removeShard(shardRouting.id(), "removing shard (recovery source node changed)"); shardHasBeenRemoved = true; @@ -586,9 +585,9 @@ private void applyNewOrUpdatedShards(final ClusterChangedEvent event) throws Ela indexService.shardInjectorSafe(shardId).getInstance(IndexShardGatewayService.class).routingStateChanged(); } } + if (shardRouting.initializing()) { - applyInitializingShard(routingTable, nodes, indexMetaData, - routingTable.index(shardRouting.index()).shard(shardRouting.id()), shardRouting); + applyInitializingShard(routingTable, nodes, indexMetaData, routingTable.index(shardRouting.index()).shard(shardRouting.id()), shardRouting); } } } @@ -666,7 +665,7 @@ private void applyInitializingShard(final RoutingTable routingTable, final Disco // if we're in peer recovery, try to find out the source node now so in case it fails, we will not create the index shard DiscoveryNode sourceNode = null; if (isPeerRecovery(shardRouting)) { - sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting, logger); + sourceNode = findSourceNodeForPeerRecovery(routingTable, nodes, shardRouting); if (sourceNode == null) { logger.trace("ignoring initializing shard {} - no source node can be found.", shardRouting.shardId()); return; @@ -764,8 +763,7 @@ public void onRecoveryFailed(IndexShardGatewayRecoveryException e) { * routing to *require* peer recovery, use {@link #isPeerRecovery(org.elasticsearch.cluster.routing.ShardRouting)} to * check if its needed or not. */ - public static DiscoveryNode findSourceNodeForPeerRecovery(RoutingTable routingTable, DiscoveryNodes nodes, ShardRouting shardRouting, ESLogger logger) { - //nocommit factor this out somewhere useful + private DiscoveryNode findSourceNodeForPeerRecovery(RoutingTable routingTable, DiscoveryNodes nodes, ShardRouting shardRouting) { DiscoveryNode sourceNode = null; if (!shardRouting.primary()) { IndexShardRoutingTable shardRoutingTable = routingTable.index(shardRouting.index()).shard(shardRouting.id()); diff --git a/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java deleted file mode 100644 index d87885e453279..0000000000000 --- a/src/main/java/org/elasticsearch/indices/recovery/FileSyncRecoveryHandler.java +++ /dev/null @@ -1,56 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.indices.recovery; - -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.action.index.MappingUpdatedAction; -import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.translog.Translog; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.transport.TransportService; - -/** - * A recovery handler that skips phase 2 and 3 of the recovery - */ -public class FileSyncRecoveryHandler extends ShardRecoveryHandler { - - private IndexShard shard; - private final StartRecoveryRequest request; - - public FileSyncRecoveryHandler(IndexShard shard, StartRecoveryRequest request, RecoverySettings recoverySettings, TransportService transportService, ClusterService clusterService, IndicesService indicesService, MappingUpdatedAction mappingUpdatedAction, ESLogger logger) { - super(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); - this.shard = shard; - this.request = request; - - } - - @Override - public void phase2(Translog.Snapshot snapshot) throws ElasticsearchException { - logger.trace("{} recovery [phase2] to {}: skipping translog for file sync", request.shardId(), request.targetNode()); - } - - @Override - protected int sendSnapshot(Translog.Snapshot snapshot) throws ElasticsearchException { - logger.trace("{} recovery [phase3] to {}: skipping transaction log operations for file sync", shard.shardId(), request.targetNode()); - return 0; - } -} diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index de578213a9ebe..4b8245e7b5d7a 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -111,7 +111,7 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), request.targetNode()); throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); } - if (request.recoveryType() != RecoveryState.Type.FILE_SYNC && !targetShardRouting.initializing()) { + if (!targetShardRouting.initializing()) { logger.debug("delaying recovery of {} as it is not listed as initializing on the target node {}. known shards state is [{}]", request.shardId(), request.targetNode(), targetShardRouting.state()); throw new DelayRecoveryException("source node has the state of the target shard to be [" + targetShardRouting.state() + "], expecting to be [initializing]"); @@ -121,10 +121,6 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { final ShardRecoveryHandler handler; if (IndexMetaData.usesSharedFilesystem(shard.indexSettings())) { handler = new SharedFSRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); - } else if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { - logger.trace("{} taking snapshot before file sync recovery", shard.shardId()); - Releasables.close(shard.snapshotIndex()); // we have to take a snapshot here and close it right away so we can catchup from the latest docs committed - handler = new FileSyncRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); } else { handler = new ShardRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); } diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java index 1ffa83e7572e7..e60541819aa30 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryState.java @@ -83,8 +83,7 @@ public static enum Type { GATEWAY((byte) 0), SNAPSHOT((byte) 1), REPLICA((byte) 2), - RELOCATION((byte) 3), - FILE_SYNC((byte) 4); + RELOCATION((byte) 3); private static final Type[] TYPES = new Type[Type.values().length]; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 2aba08f96053c..bcb9df043a35d 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -141,17 +141,6 @@ public void startRecovery(final IndexShard indexShard, final RecoveryState.Type startRecoveryInternal(indexShard, recoveryType, sourceNode, listener); } - /** - * Runs a full recovery but skips phase 2 and 3 and only syncs the files that have been committed. - */ - public void startFileSync(final IndexShard indexShard, final DiscoveryNode sourceNode, final RecoveryListener listener) { - if (indexShard.state() != IndexShardState.STARTED) { - throw new ElasticsearchIllegalStateException("Shard is not started can't sync files -state: " + indexShard.state()); - } - logger.trace("{} starting file sync with {}", indexShard.shardId(), sourceNode); - startRecoveryInternal(indexShard, RecoveryState.Type.FILE_SYNC, sourceNode, listener); - } - private void startRecoveryInternal(IndexShard indexShard, RecoveryState.Type recoveryType, DiscoveryNode sourceNode, RecoveryListener listener) { // create a new recovery status, and process... RecoveryState recoveryState = new RecoveryState(indexShard.shardId()); diff --git a/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java b/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java index ed5ba05281fa0..a67dd3199f6d8 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java +++ b/src/main/java/org/elasticsearch/indices/recovery/ShardRecoveryHandler.java @@ -416,10 +416,6 @@ public int compare(StoreFileMetaData o1, StoreFileMetaData o2) { */ @Override public void phase2(Translog.Snapshot snapshot) throws ElasticsearchException { - if (request.recoveryType() == RecoveryState.Type.FILE_SYNC) { - logger.trace("{} recovery [phase2] to {}: skipping translog for file sync", request.shardId(), request.targetNode()); - return; - } if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 8cf8fc9fa67cb..4c3865e9643c3 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -329,92 +329,4 @@ public void run() { assertPathHasBeenCleared(dataPath); } - - - public void testSimpleNonSharedFS() throws Exception { - internalCluster().startNodesAsync(3).get(); - final String IDX = "test"; - final Path dataPath = newTempDirPath(); - - Settings idxSettings = ImmutableSettings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(1, 2)) - .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) - .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) - .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, false) - .build(); - - prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string").get(); - ensureGreen(IDX); - - IndexRequestBuilder[] builders = new IndexRequestBuilder[between(10, 20)]; - for (int i = 0; i < builders.length; i++) { - builders[i] = client().prepareIndex(IDX, "doc", Integer.toString(i)).setSource("foo", "bar"); - } - indexRandom(false, builders); - flush(IDX); - logger.info("--> performing query"); - for (int i = 0; i < 10; i++) { - SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); - assertHitCount(resp, builders.length); - } - - logger.info("--> restarting all nodes"); - if (randomBoolean()) { - logger.info("--> rolling restart"); - internalCluster().rollingRestart(); - } else { - logger.info("--> full restart"); - internalCluster().fullRestart(); - } - - client().admin().cluster().prepareHealth().setWaitForNodes("3").get(); - ensureGreen(IDX); - - logger.info("--> performing query"); - for (int i = 0; i < 10; i++) { - SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); - assertHitCount(resp, builders.length); - for (SearchHit hit : resp.getHits().getHits()) { - assertEquals(hit.sourceAsMap().get("foo"), "bar"); - } - } - - // we reindex and check if the new updated docs are all available on the primary - for (int i = 0; i < builders.length; i++) { - builders[i] = client().prepareIndex(IDX, "doc", Integer.toString(i)).setSource("foo", "foobar"); - } - indexRandom(false, true, false, Arrays.asList(builders)); // don't flush here - // only refresh no flush - refresh(); - ClusterState state = client().admin().cluster().prepareState().get().getState(); - GroupShardsIterator shardIterators = state.getRoutingNodes().getRoutingTable().activePrimaryShardsGrouped(new String[]{IDX}, false); - ShardRouting primary = shardIterators.iterator().next().nextOrNull(); - for (int i = 0; i < 10; i++) { - SearchResponse resp = client().prepareSearch(IDX).setExplain(true).setQuery(matchAllQuery()).get(); - assertHitCount(resp, builders.length); - for (SearchHit hit : resp.getHits().getHits()) { - if (hit.shard().getNodeId().equals(primary.currentNodeId())) { - // this comes from the primary so we have the latest - assertEquals(hit.sourceAsMap().get("foo"), "foobar"); - } else { - // this comes from the replica not caught up yet - assertEquals(hit.sourceAsMap().get("foo"), "bar"); - } - } - } - - flush(IDX); - logger.info("--> performing query"); - for (int i = 0; i < 10; i++) { - SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).get(); - assertHitCount(resp, builders.length); - for (SearchHit hit : resp.getHits().getHits()) { - assertEquals(hit.sourceAsMap().get("foo"), "foobar"); - } - } - - logger.info("--> deleting index"); - assertAcked(client().admin().indices().prepareDelete(IDX)); - } } From 744f228c192602a6737051571e040731d413ba8b Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 11:28:12 +0100 Subject: [PATCH 38/56] revert changes to IndexShardGateway - these are leftovers from previous iterations --- .../index/gateway/IndexShardGateway.java | 34 +++++-------------- 1 file changed, 9 insertions(+), 25 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java index 35751d2047a0f..49da62e51121b 100644 --- a/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java +++ b/src/main/java/org/elasticsearch/index/gateway/IndexShardGateway.java @@ -29,19 +29,20 @@ import org.elasticsearch.cluster.action.index.MappingUpdatedAction; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; -import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.gateway.IndexShardGateway; +import org.elasticsearch.index.gateway.IndexShardGatewayRecoveryException; +import org.elasticsearch.index.IndexService; import org.elasticsearch.index.settings.IndexSettings; import org.elasticsearch.index.shard.AbstractIndexShardComponent; -import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardState; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.translog.*; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.rest.RestStatus; @@ -99,17 +100,7 @@ public IndexShardGateway(ShardId shardId, @IndexSettings Settings indexSettings, } } - public void recover(boolean indexShouldExist, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException { - recover(this.indexShard, this.logger, this.mappingUpdatedAction, this.cancellableThreads, - indexShouldExist, recoveryState, this.waitForMappingUpdatePostRecovery); - } - - public static void recover(IndexShard indexShard, final ESLogger logger, - final MappingUpdatedAction mappingUpdatedAction, - final CancellableThreads cancellableThreads, - boolean indexShouldExists, RecoveryState recoveryState, - final TimeValue waitForMappingUpdatePostRecovery) - throws IndexShardGatewayRecoveryException { + public void recover(boolean indexShouldExists, RecoveryState recoveryState) throws IndexShardGatewayRecoveryException { recoveryState.getIndex().startTime(System.currentTimeMillis()); recoveryState.setStage(RecoveryState.Stage.INDEX); long version = -1; @@ -130,8 +121,7 @@ public static void recover(IndexShard indexShard, final ESLogger logger, files += " (failure=" + ExceptionsHelper.detailedMessage(e1) + ")"; } if (indexShouldExists) { - throw new IndexShardGatewayRecoveryException(indexShard.shardId(), - "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e); + throw new IndexShardGatewayRecoveryException(shardId(), "shard allocated for local recovery (post api), should exist, but doesn't, current files: " + files, e); } } if (si != null) { @@ -157,8 +147,7 @@ public static void recover(IndexShard indexShard, final ESLogger logger, } } } catch (Throwable e) { - throw new IndexShardGatewayRecoveryException(indexShard.shardId(), - "failed to fetch index version after copying it over", e); + throw new IndexShardGatewayRecoveryException(shardId(), "failed to fetch index version after copying it over", e); } recoveryState.getIndex().updateVersion(version); recoveryState.getIndex().time(System.currentTimeMillis() - recoveryState.getIndex().startTime()); @@ -293,8 +282,7 @@ public static void recover(IndexShard indexShard, final ESLogger logger, } } catch (Throwable e) { IOUtils.closeWhileHandlingException(indexShard.translog()); - throw new IndexShardGatewayRecoveryException(indexShard.shardId(), - "failed to recover shard", e); + throw new IndexShardGatewayRecoveryException(shardId, "failed to recover shard", e); } finally { IOUtils.closeWhileHandlingException(in); } @@ -310,11 +298,7 @@ public static void recover(IndexShard indexShard, final ESLogger logger, } for (final String type : typesToUpdate) { final CountDownLatch latch = new CountDownLatch(1); - IndexService indexService = indexShard.indexService(); - mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), - indexService.mapperService().documentMapper(type), - indexService.indexUUID(), - new MappingUpdatedAction.MappingUpdateListener() { + mappingUpdatedAction.updateMappingOnMaster(indexService.index().name(), indexService.mapperService().documentMapper(type), indexService.indexUUID(), new MappingUpdatedAction.MappingUpdateListener() { @Override public void onMappingUpdate() { latch.countDown(); From 45cd34a12a442080477da3ef14ab2fe7947ea97e Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 11:32:57 +0100 Subject: [PATCH 39/56] fix tests --- .../index/engine/ShadowEngineTests.java | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index 0e8f59bc1f747..b4be7436d0897 100644 --- a/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -828,7 +828,6 @@ public void testFailEngineOnCorruption() { ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, false); primaryEngine.create(new Engine.Create(null, newUid("1"), doc)); primaryEngine.flush(); - final boolean failEngine = replicaEngine.config().isFailEngineOnCorruption(); MockDirectoryWrapper leaf = DirectoryUtils.getLeaf(replicaEngine.config().getStore().directory(), MockDirectoryWrapper.class); leaf.setRandomIOExceptionRate(1.0); leaf.setRandomIOExceptionRateOnOpen(1.0); @@ -843,18 +842,9 @@ public void testFailEngineOnCorruption() { MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(1)); MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 1)); searchResult.close(); - - ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, false); - primaryEngine.create(new Engine.Create(null, newUid("2"), doc2)); - primaryEngine.refresh("foo"); - - searchResult = replicaEngine.acquireSearcher("test"); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(new TermQuery(new Term("value", "test")), 2)); - MatcherAssert.assertThat(searchResult, EngineSearcherTotalHitsMatcher.engineSearcherTotalHits(2)); - searchResult.close(); - assertThat(failEngine, is(false)); + fail("exception expected"); } catch (EngineClosedException ex) { - assertThat(failEngine, is(true)); + // all is well } } From c5ece6d16d423fbdd36f5d789bd8daa5724d77b0 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 16:13:12 +0100 Subject: [PATCH 40/56] simplify shadow engine --- .../elasticsearch/index/engine/Engine.java | 29 ++++++- .../index/engine/InternalEngine.java | 27 +----- .../index/engine/ShadowEngine.java | 83 ++----------------- 3 files changed, 38 insertions(+), 101 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/engine/Engine.java b/src/main/java/org/elasticsearch/index/engine/Engine.java index 00c256d0bb49f..0a758b4af6736 100644 --- a/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -283,7 +283,30 @@ protected void ensureOpen() { /** * Global stats on segments. */ - public abstract SegmentsStats segmentsStats(); + public final SegmentsStats segmentsStats() { + ensureOpen(); + try (final Searcher searcher = acquireSearcher("segments_stats")) { + SegmentsStats stats = new SegmentsStats(); + for (LeafReaderContext reader : searcher.reader().leaves()) { + final SegmentReader segmentReader = segmentReader(reader.reader()); + stats.add(1, segmentReader.ramBytesUsed()); + stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader())); + stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader())); + stats.addTermVectorsMemoryInBytes(guardedRamBytesUsed(segmentReader.getTermVectorsReader())); + stats.addNormsMemoryInBytes(guardedRamBytesUsed(segmentReader.getNormsReader())); + stats.addDocValuesMemoryInBytes(guardedRamBytesUsed(segmentReader.getDocValuesReader())); + } + writerSegmentStats(stats); + return stats; + } + } + + protected void writerSegmentStats(SegmentsStats stats) { + // by default we don't have a writer here... subclasses can override this + stats.addVersionMapMemoryInBytes(0); + stats.addIndexWriterMemoryInBytes(0); + stats.addIndexWriterMaxMemoryInBytes(0); + } protected Segment[] getSegmentInfo(SegmentInfos lastCommittedSegmentInfos, boolean verbose) { ensureOpen(); @@ -405,7 +428,9 @@ public final boolean refreshNeeded() { /** * Optimizes to 1 segment */ - abstract void forceMerge(boolean flush); + public void forceMerge(boolean flush) { + forceMerge(flush, 1, false, false); + } /** * Triggers a forced merge on this engine diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index c8b925fc97b4a..ed7879f064a8b 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -706,11 +706,6 @@ private void waitForMerges(boolean flushAfter, boolean upgrade) { } } - @Override - public void forceMerge(boolean flush) { - forceMerge(flush, 1, false, false); - } - @Override public void forceMerge(final boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, final boolean upgrade) throws EngineException { if (optimizeMutex.compareAndSet(false, true)) { @@ -846,24 +841,10 @@ protected boolean maybeFailEngine(String source, Throwable t) { } @Override - public SegmentsStats segmentsStats() { - ensureOpen(); - try (final Searcher searcher = acquireSearcher("segments_stats")) { - SegmentsStats stats = new SegmentsStats(); - for (LeafReaderContext reader : searcher.reader().leaves()) { - final SegmentReader segmentReader = segmentReader(reader.reader()); - stats.add(1, segmentReader.ramBytesUsed()); - stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader())); - stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader())); - stats.addTermVectorsMemoryInBytes(guardedRamBytesUsed(segmentReader.getTermVectorsReader())); - stats.addNormsMemoryInBytes(guardedRamBytesUsed(segmentReader.getNormsReader())); - stats.addDocValuesMemoryInBytes(guardedRamBytesUsed(segmentReader.getDocValuesReader())); - } - stats.addVersionMapMemoryInBytes(versionMap.ramBytesUsed()); - stats.addIndexWriterMemoryInBytes(indexWriter.ramBytesUsed()); - stats.addIndexWriterMaxMemoryInBytes((long) (indexWriter.getConfig().getRAMBufferSizeMB() * 1024 * 1024)); - return stats; - } + protected final void writerSegmentStats(SegmentsStats stats) { + stats.addVersionMapMemoryInBytes(versionMap.ramBytesUsed()); + stats.addIndexWriterMemoryInBytes(indexWriter.ramBytesUsed()); + stats.addIndexWriterMaxMemoryInBytes((long) (indexWriter.getConfig().getRAMBufferSizeMB() * 1024 * 1024)); } @Override diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index 4b304f5f6bbe0..0fd4accf166e2 100644 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -62,8 +62,6 @@ */ public class ShadowEngine extends Engine { - private final RecoveryCounter onGoingRecoveries; - private volatile SearcherManager searcherManager; private SegmentInfos lastCommittedSegmentInfos; @@ -71,7 +69,6 @@ public class ShadowEngine extends Engine { public ShadowEngine(EngineConfig engineConfig) { super(engineConfig); SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); - this.onGoingRecoveries = new RecoveryCounter(store); try { DirectoryReader reader = null; store.incRef(); @@ -98,22 +95,22 @@ public ShadowEngine(EngineConfig engineConfig) { @Override public void create(Create create) throws EngineException { - throw new UnsupportedOperationException("create operation not allowed on shadow engine"); + throw new UnsupportedOperationException(shardId + " create operation not allowed on shadow engine"); } @Override public void index(Index index) throws EngineException { - throw new UnsupportedOperationException("index operation not allowed on shadow engine"); + throw new UnsupportedOperationException(shardId + " index operation not allowed on shadow engine"); } @Override public void delete(Delete delete) throws EngineException { - throw new UnsupportedOperationException("delete operation not allowed on shadow engine"); + throw new UnsupportedOperationException(shardId + " delete operation not allowed on shadow engine"); } @Override public void delete(DeleteByQuery delete) throws EngineException { - throw new UnsupportedOperationException("delete-by-query operation not allowed on shadow engine"); + throw new UnsupportedOperationException(shardId + " delete-by-query operation not allowed on shadow engine"); } @Override @@ -126,7 +123,7 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { logger.trace("skipping FLUSH on shadow engine"); // reread the last committed segment infos refresh("flush"); - try { + try (ReleasableLock _ = readLock.acquire()) { lastCommittedSegmentInfos = store.readLastCommittedSegmentsInfo(); } catch (Throwable e) { if (isClosed.get() == false) { @@ -138,11 +135,6 @@ public void flush(boolean force, boolean waitIfOngoing) throws EngineException { } } - @Override - public void forceMerge(boolean flush) { - forceMerge(flush, 1, false, false); - } - @Override public void forceMerge(boolean flush, int maxNumSegments, boolean onlyExpungeDeletes, boolean upgrade) throws EngineException { // no-op @@ -155,29 +147,6 @@ public GetResult get(Get get) throws EngineException { return getFromSearcher(get); } - @Override - public SegmentsStats segmentsStats() { - ensureOpen(); - try (final Searcher searcher = acquireSearcher("segments_stats")) { - SegmentsStats stats = new SegmentsStats(); - for (LeafReaderContext reader : searcher.reader().leaves()) { - final SegmentReader segmentReader = segmentReader(reader.reader()); - stats.add(1, segmentReader.ramBytesUsed()); - stats.addTermsMemoryInBytes(guardedRamBytesUsed(segmentReader.getPostingsReader())); - stats.addStoredFieldsMemoryInBytes(guardedRamBytesUsed(segmentReader.getFieldsReader())); - stats.addTermVectorsMemoryInBytes(guardedRamBytesUsed(segmentReader.getTermVectorsReader())); - stats.addNormsMemoryInBytes(guardedRamBytesUsed(segmentReader.getNormsReader())); - stats.addDocValuesMemoryInBytes(guardedRamBytesUsed(segmentReader.getDocValuesReader())); - } - // No version map for shadow engine - stats.addVersionMapMemoryInBytes(0); - // Since there is no IndexWriter, these are 0 - stats.addIndexWriterMemoryInBytes(0); - stats.addIndexWriterMaxMemoryInBytes(0); - return stats; - } - } - @Override public List segments(boolean verbose) { try (ReleasableLock _ = readLock.acquire()) { @@ -210,49 +179,12 @@ public void refresh(String source) throws EngineException { @Override public SnapshotIndexCommit snapshotIndex() throws EngineException { - // we have to flush outside of the readlock otherwise we might have a problem upgrading - // the to a write lock when we fail the engine in this operation - flush(false, true); - try (ReleasableLock _ = readLock.acquire()) { - ensureOpen(); - return deletionPolicy.snapshot(); - } catch (IOException e) { - throw new SnapshotFailedEngineException(shardId, e); - } + throw new UnsupportedOperationException("Can not take snapshot from a shadow engine"); } @Override public void recover(RecoveryHandler recoveryHandler) throws EngineException { - // take a write lock here so it won't happen while a flush is in progress - // this means that next commits will not be allowed once the lock is released - try (ReleasableLock _ = writeLock.acquire()) { - ensureOpen(); - onGoingRecoveries.startRecovery(); - } - - SnapshotIndexCommit phase1Snapshot; - try { - phase1Snapshot = deletionPolicy.snapshot(); - } catch (Throwable e) { - maybeFailEngine("recovery", e); - Releasables.closeWhileHandlingException(onGoingRecoveries); - throw new RecoveryEngineException(shardId, 1, "Snapshot failed", e); - } - - boolean success = false; - try { - recoveryHandler.phase1(phase1Snapshot); - success = true; - } catch (Throwable e) { - maybeFailEngine("recovery phase 1", e); - Releasables.closeWhileHandlingException(onGoingRecoveries, phase1Snapshot); - throw new RecoveryEngineException(shardId, 1, "Execution failed", wrapIfClosed(e)); - } finally { - Releasables.close(success, onGoingRecoveries, writeLock, phase1Snapshot); - } - - // Since operations cannot be replayed from a translog on a shadow - // engine, there is no phase2 and phase3 of recovery + throw new UnsupportedOperationException("Can not recover from a shadow engine"); } @Override @@ -260,7 +192,6 @@ protected SearcherManager getSearcherManager() { return searcherManager; } - @Override protected void closeNoLock(String reason) throws ElasticsearchException { if (isClosed.compareAndSet(false, true)) { From eb699c19f04965952ae45e2caf107124837c4654 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 17 Feb 2015 16:15:39 +0100 Subject: [PATCH 41/56] remove last nocommit --- .../TransportShardReplicationOperationAction.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java index 77b6b3ac7b5a6..bd7dbc7a1f02d 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java @@ -636,9 +636,9 @@ void performOnReplica(final ReplicationState state, final ShardRouting shard, fi // perform the action on the replica, so skip it and // immediately return if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings())) { - // nocommit - this doesn't replicate mappings changes, so can fail if mappings are not predefined - // It was successful on the replica, although we never - // actually executed + // this doesn't replicate mappings changes, so can fail if mappings are not predefined + // It was successful on the replica, although we never actually executed - in the future we will + // ack mapping updates with the master and that will sync with replicas. For now this is just fine. state.onReplicaSuccess(); return; } From 4473e630460e2f0ca2a2e2478f3712f39a64c919 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 17 Feb 2015 09:00:39 -0700 Subject: [PATCH 42/56] Add asciidoc documentation for shadow replicas --- docs/reference/indices.asciidoc | 6 ++ .../indices/shadow-replicas.asciidoc | 89 +++++++++++++++++++ 2 files changed, 95 insertions(+) create mode 100644 docs/reference/indices/shadow-replicas.asciidoc diff --git a/docs/reference/indices.asciidoc b/docs/reference/indices.asciidoc index 11a73784919e5..616d24ffbbbbe 100644 --- a/docs/reference/indices.asciidoc +++ b/docs/reference/indices.asciidoc @@ -41,6 +41,11 @@ and warmers. * <> * <> +[float] +[[shadow-replicas]] +== Replica configurations +* <> + [float] [[monitoring]] == Monitoring: @@ -107,3 +112,4 @@ include::indices/optimize.asciidoc[] include::indices/upgrade.asciidoc[] +include::indices/shadow-replicas.asciidoc[] diff --git a/docs/reference/indices/shadow-replicas.asciidoc b/docs/reference/indices/shadow-replicas.asciidoc new file mode 100644 index 0000000000000..baa152f86f428 --- /dev/null +++ b/docs/reference/indices/shadow-replicas.asciidoc @@ -0,0 +1,89 @@ +[[indices-shadow-replicas]] +== Shadow replica indices + +If you would like to use a shared filesystem, you can use the shadow replicas +settings to choose where on disk the data for an index should be kept, as well +as how Elasticsearch should replay operations on all the replica shards of an +index. + +In order to fully utilize the `index.data_path` and `index.shadow_replicas` +settings, you need to enable using it in elasticsearch.yml: + +[source,yaml] +-------------------------------------------------- +node.enable_custom_paths: true +-------------------------------------------------- + +You can then create an index with a custom data path, where each node will use +this path for the data: + +[source,js] +-------------------------------------------------- +curl -XPUT 'localhost:9200/my_index/_settings' -d ' +{ + "index" : { + "number_of_shards" : 1 + "number_of_replicas" : 4, + "data_path": "/var/data/my_index", + "shadow_replicas": true + } +}' +-------------------------------------------------- + +[WARNING] +======================== +In the above example, the "/var/data/my_index" path is a shared filesystem that +must be available on every node in the Elasticsearch cluster. You must also +ensure that the Elasticsearch process has the correct permissions to read from +and write to the directory used in the `index.data_path` setting. +======================== + +An index that has been created with the `index.shadow_replicas` setting set to +"true" will not replicate document operations to any of the replica shards, +instead, it will only continually refresh. Once segments are available on the +filesystem where the shadow replica resides (after an Elasticsearch "flush"), a +regular refresh (governed by the `index.refresh_interval`) can be used to make +the new data searchable. + +In order to ensure the data is being synchronized in a fast enough manner, you +may need to tune the flush threshold for the index to a desired number. A flush +is needed to fsync segment files to disk, so they will be visible to all other +replica nodes. Users should test what flush threshold levels they are +comfortable with, as increased flushing can impact indexing performance. + +The Elasticsearch cluster will still detect the loss of a primary shard, and +transform the replica into a primary in this situation. This transformation will +take slightly longer, since no `IndexWriter` is maintained for each shadow +replica. + +Below is the list of settings that can be changed using the update +settings API: + +`index.data_path` (string):: + Path to use for the index's data. Note that by default Elasticsearch will + append the node ordinal by default to the path to ensure multiple instances + of Elasticsearch on the same machine do not share a data directory. + +`index.shadow_replicas`:: + Boolean value indicating this index should use shadow replicas. Defaults to + `false`. + +`index.shared_filesystem`:: + Boolean value indicating this index uses a shared filesystem. Defaults to + the `true` if `index.shadow_replicas` is set to true, `false` otherwise. + +=== Node level settings related to shadow replicas + +These are non-dynamic settings that need to be configured in `elasticsearch.yml` + +`node.add_id_to_custom_path`:: + Boolean setting indicating whether Elasticsearch should append the node's + ordinal to the custom data path. For example, if this is enabled and a path + of "/tmp/foo" is used, the first locally-running node will use "/tmp/foo/0", + the second will use "/tmp/foo/1", the third "/tmp/foo/2", etc. Defaults to + `true`. + +`node.enable_custom_paths`:: + Boolean value that must be set to `true` in order to use the + `index.data_path` setting. Defaults to `false`. + From a1b8b8cf0db49d1bd1aeb84e51491f7f0de43b59 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Tue, 17 Feb 2015 14:31:50 -0700 Subject: [PATCH 43/56] Remove unused import and fix index creation example in docs --- docs/reference/indices/shadow-replicas.asciidoc | 4 ++-- .../elasticsearch/indices/recovery/RecoveriesCollection.java | 1 - 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/reference/indices/shadow-replicas.asciidoc b/docs/reference/indices/shadow-replicas.asciidoc index baa152f86f428..b4a8be649973a 100644 --- a/docs/reference/indices/shadow-replicas.asciidoc +++ b/docs/reference/indices/shadow-replicas.asciidoc @@ -19,10 +19,10 @@ this path for the data: [source,js] -------------------------------------------------- -curl -XPUT 'localhost:9200/my_index/_settings' -d ' +curl -XPUT 'localhost:9200/my_index' -d ' { "index" : { - "number_of_shards" : 1 + "number_of_shards" : 1, "number_of_replicas" : 4, "data_path": "/var/data/my_index", "shadow_replicas": true diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java index b1312cee52365..cac4a57b979cf 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveriesCollection.java @@ -20,7 +20,6 @@ package org.elasticsearch.indices.recovery; import org.elasticsearch.ElasticsearchTimeoutException; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.ESLogger; From b64fef1d2c5e167713e869b22d388ff479252173 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 08:25:19 -0700 Subject: [PATCH 44/56] Add warning that predefined mappings should be used --- docs/reference/indices/shadow-replicas.asciidoc | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/docs/reference/indices/shadow-replicas.asciidoc b/docs/reference/indices/shadow-replicas.asciidoc index b4a8be649973a..240abfb1bcac9 100644 --- a/docs/reference/indices/shadow-replicas.asciidoc +++ b/docs/reference/indices/shadow-replicas.asciidoc @@ -1,6 +1,8 @@ [[indices-shadow-replicas]] == Shadow replica indices +experimental[] + If you would like to use a shared filesystem, you can use the shadow replicas settings to choose where on disk the data for an index should be kept, as well as how Elasticsearch should replay operations on all the replica shards of an @@ -17,6 +19,15 @@ node.enable_custom_paths: true You can then create an index with a custom data path, where each node will use this path for the data: +[WARNING] +======================== +Because shadow replicas do not index the document on replica shards, it's +possible for the replica's known mapping to be behind the index's known mapping +if the latest cluster state has not yet been processed on the node containing +the replica. Because of this, it is highly recommended to use pre-defined +mappings when using shadow replicas. +======================== + [source,js] -------------------------------------------------- curl -XPUT 'localhost:9200/my_index' -d ' From 23001af834d66278ac84d9a72c37b5d1f3a10a7b Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 08:35:25 -0700 Subject: [PATCH 45/56] Remove ShadowEngineFactory, add .newReadOnlyEngine method in EngineFactory --- .../index/engine/EngineFactory.java | 4 ++- .../index/engine/InternalEngineFactory.java | 7 ++++- .../index/engine/ShadowEngineFactory.java | 30 ------------------- .../elasticsearch/index/shard/IndexShard.java | 12 ++++++-- .../index/shard/IndexShardModule.java | 11 +------ .../test/engine/MockEngineFactory.java | 8 ++++- 6 files changed, 26 insertions(+), 46 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java diff --git a/src/main/java/org/elasticsearch/index/engine/EngineFactory.java b/src/main/java/org/elasticsearch/index/engine/EngineFactory.java index 89425dc46826d..ab5cfc5e6a916 100644 --- a/src/main/java/org/elasticsearch/index/engine/EngineFactory.java +++ b/src/main/java/org/elasticsearch/index/engine/EngineFactory.java @@ -23,5 +23,7 @@ */ public interface EngineFactory { - public Engine newEngine(EngineConfig config); + public Engine newReadWriteEngine(EngineConfig config); + + public Engine newReadOnlyEngine(EngineConfig config); } diff --git a/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java b/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java index 0af65d08671ab..3857e9391c24e 100644 --- a/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java +++ b/src/main/java/org/elasticsearch/index/engine/InternalEngineFactory.java @@ -20,7 +20,12 @@ public class InternalEngineFactory implements EngineFactory { @Override - public Engine newEngine(EngineConfig config) { + public Engine newReadWriteEngine(EngineConfig config) { return new InternalEngine(config); } + + @Override + public Engine newReadOnlyEngine(EngineConfig config) { + return new ShadowEngine(config); + } } diff --git a/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java b/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java deleted file mode 100644 index 8606eac78de32..0000000000000 --- a/src/main/java/org/elasticsearch/index/engine/ShadowEngineFactory.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.index.engine; - -/** - * TODO: document me! - */ -public class ShadowEngineFactory implements EngineFactory { - @Override - public Engine newEngine(EngineConfig config) { - return new ShadowEngine(config); - } -} diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index f6fd9b71b29a1..ea9ca64b84a3a 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -34,6 +34,7 @@ import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.optimize.OptimizeRequest; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.common.Booleans; @@ -53,7 +54,6 @@ import org.elasticsearch.index.IndexService; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.aliases.IndexAliasesService; -import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.filter.FilterCacheStats; @@ -88,9 +88,9 @@ import org.elasticsearch.index.settings.IndexSettingsService; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreStats; -import org.elasticsearch.index.store.Store.MetadataSnapshot; import org.elasticsearch.index.suggest.stats.ShardSuggestService; import org.elasticsearch.index.suggest.stats.SuggestStats; import org.elasticsearch.index.termvectors.ShardTermVectorsService; @@ -1147,7 +1147,13 @@ private void createNewEngine() { throw new EngineClosedException(shardId); } assert this.currentEngineReference.get() == null; - this.currentEngineReference.set(engineFactory.newEngine(config)); + // Use the read-only engine for shadow replicas + if (IndexMetaData.isIndexUsingShadowReplicas(this.indexSettings) && + this.shardRouting.primary() == false) { + this.currentEngineReference.set(engineFactory.newReadOnlyEngine(config)); + } else { + this.currentEngineReference.set(engineFactory.newReadWriteEngine(config)); + } } } } diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index dde9088b7e498..98608200e2758 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory; -import org.elasticsearch.index.engine.ShadowEngineFactory; import org.elasticsearch.index.warmer.ShardIndexWarmerService; /** @@ -35,9 +34,7 @@ public class IndexShardModule extends AbstractModule { public static final String ENGINE_FACTORY = "index.engine.factory"; - public static final String SHADOW_ENGINE_FACTORY = "index.shadow_engine.factory"; private static final Class DEFAULT_ENGINE_FACTORY_CLASS = InternalEngineFactory.class; - private static final Class SHADOW_ENGINE_FACTORY_CLASS = ShadowEngineFactory.class; private static final String ENGINE_PREFIX = "org.elasticsearch.index.engine."; private static final String ENGINE_SUFFIX = "EngineFactory"; @@ -66,13 +63,7 @@ protected void configure() { bind(IndexShard.class).asEagerSingleton(); } - Class engineFactory = DEFAULT_ENGINE_FACTORY_CLASS; - String factorySetting = ENGINE_FACTORY; - if (useShadowEngine()) { - engineFactory = SHADOW_ENGINE_FACTORY_CLASS; - factorySetting = SHADOW_ENGINE_FACTORY; - } - bind(EngineFactory.class).to(settings.getAsClass(factorySetting, engineFactory, ENGINE_PREFIX, ENGINE_SUFFIX)); + bind(EngineFactory.class).to(settings.getAsClass(ENGINE_FACTORY, DEFAULT_ENGINE_FACTORY_CLASS, ENGINE_PREFIX, ENGINE_SUFFIX)); bind(ShardIndexWarmerService.class).asEagerSingleton(); } diff --git a/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java b/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java index 57d0d8eca3742..51dab2041a180 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java +++ b/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java @@ -21,13 +21,19 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineFactory; +import org.elasticsearch.index.engine.ShadowEngine; /** * */ public final class MockEngineFactory implements EngineFactory { @Override - public Engine newEngine(EngineConfig config) { + public Engine newReadWriteEngine(EngineConfig config) { return new MockInternalEngine(config); } + + @Override + public Engine newReadOnlyEngine(EngineConfig config) { + return new ShadowEngine(config); + } } From d90d6980c9b737bd8c0f4339613a5373b1645e95 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 08:35:44 -0700 Subject: [PATCH 46/56] Rename `ownsShard` to `canDeleteShardContent` --- src/main/java/org/elasticsearch/index/IndexService.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/IndexService.java b/src/main/java/org/elasticsearch/index/IndexService.java index 164e4aac9d2c9..22e0c8e14ef9e 100644 --- a/src/main/java/org/elasticsearch/index/IndexService.java +++ b/src/main/java/org/elasticsearch/index/IndexService.java @@ -305,7 +305,8 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) throws logger.debug("creating shard_id {}", shardId); // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary. - final boolean ownsShard = IndexMetaData.usesSharedFilesystem(indexSettings) == false || (primary && IndexMetaData.usesSharedFilesystem(indexSettings)); + final boolean canDeleteShardContent = IndexMetaData.usesSharedFilesystem(indexSettings) == false || + (primary && IndexMetaData.usesSharedFilesystem(indexSettings)); ModulesBuilder modules = new ModulesBuilder(); modules.add(new ShardsPluginsModule(indexSettings, pluginsService)); modules.add(new IndexShardModule(shardId, primary, indexSettings)); @@ -313,7 +314,7 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) throws modules.add(new ShardSearchModule()); modules.add(new ShardGetModule()); modules.add(new StoreModule(indexSettings, injector.getInstance(IndexStore.class), lock, - new StoreCloseListener(shardId, ownsShard))); + new StoreCloseListener(shardId, canDeleteShardContent))); modules.add(new DeletionPolicyModule(indexSettings)); modules.add(new MergePolicyModule(indexSettings)); modules.add(new MergeSchedulerModule(indexSettings)); From 7346f9f382f83a21cd2445b3386fe67472bc3184 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 08:37:14 -0700 Subject: [PATCH 47/56] Revert changes to RecoveryTarget.java --- .../elasticsearch/indices/recovery/RecoveryTarget.java | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index bcb9df043a35d..6f8efa5ebfc0d 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -25,10 +25,8 @@ import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexOutput; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ElasticsearchIllegalStateException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.StopWatch; @@ -138,10 +136,6 @@ public void startRecovery(final IndexShard indexShard, final RecoveryState.Type logger.debug("{} ignore recovery. already in recovering process, {}", indexShard.shardId(), e.getMessage()); return; } - startRecoveryInternal(indexShard, recoveryType, sourceNode, listener); - } - - private void startRecoveryInternal(IndexShard indexShard, RecoveryState.Type recoveryType, DiscoveryNode sourceNode, RecoveryListener listener) { // create a new recovery status, and process... RecoveryState recoveryState = new RecoveryState(indexShard.shardId()); recoveryState.setType(recoveryType); @@ -150,6 +144,7 @@ private void startRecoveryInternal(IndexShard indexShard, RecoveryState.Type rec recoveryState.setPrimary(indexShard.routingEntry().primary()); final long recoveryId = onGoingRecoveries.startRecovery(indexShard, sourceNode, recoveryState, listener, recoverySettings.activityTimeout()); threadPool.generic().execute(new RecoveryRunner(recoveryId)); + } protected void retryRecovery(final RecoveryStatus recoveryStatus, final String reason, TimeValue retryAfter, final StartRecoveryRequest currentRequest) { @@ -164,6 +159,7 @@ protected void retryRecovery(final RecoveryStatus recoveryStatus, final String r private void doRecovery(final RecoveryStatus recoveryStatus) { assert recoveryStatus.sourceNode() != null : "can't do a recovery without a source node"; + logger.trace("collecting local files for {}", recoveryStatus); final Map existingFiles; try { From 60a4d5374af5262bd415f4ef40f635278ed12a03 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 09:18:22 -0700 Subject: [PATCH 48/56] Add a test for shadow replicas that uses field data --- .../index/IndexWithShadowReplicasTests.java | 47 ++++++++++++++++--- 1 file changed, 40 insertions(+), 7 deletions(-) diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 4c3865e9643c3..5655e9e61fb98 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -23,28 +23,23 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; -import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.settings.ImmutableSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ElasticsearchIntegrationTest; import org.elasticsearch.test.InternalTestCluster; -import org.elasticsearch.test.junit.annotations.TestLogging; import org.junit.Test; import java.nio.file.Path; -import java.util.Arrays; import java.util.List; import static com.google.common.collect.Lists.newArrayList; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.*; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -329,4 +324,42 @@ public void run() { assertPathHasBeenCleared(dataPath); } + + @Test + public void testShadowReplicasUsingFieldData() throws Exception { + Settings nodeSettings = ImmutableSettings.builder() + .put("node.add_id_to_custom_path", false) + .put("node.enable_custom_paths", true) + .build(); + + internalCluster().startNodesAsync(3, nodeSettings).get(); + Path dataPath = newTempDirPath(); + String IDX = "test"; + + Settings idxSettings = ImmutableSettings.builder() + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 2) + .put(IndexMetaData.SETTING_DATA_PATH, dataPath.toAbsolutePath().toString()) + .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) + .put(IndexMetaData.SETTING_SHARED_FILESYSTEM, true) + .build(); + + prepareCreate(IDX).setSettings(idxSettings).addMapping("doc", "foo", "type=string,index=not_analyzed").get(); + ensureGreen(IDX); + + client().prepareIndex(IDX, "doc", "1").setSource("foo", "foo").get(); + client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + client().prepareIndex(IDX, "doc", "3").setSource("foo", "baz").get(); + client().prepareIndex(IDX, "doc", "4").setSource("foo", "eggplant").get(); + flushAndRefresh(IDX); + + SearchResponse resp = client().prepareSearch(IDX).setQuery(matchAllQuery()).addFieldDataField("foo").addSort("foo", SortOrder.ASC).get(); + assertHitCount(resp, 4); + assertOrderedSearchHits(resp, "2", "3", "4", "1"); + SearchHit[] hits = resp.getHits().hits(); + assertThat(hits[0].field("foo").getValue().toString(), equalTo("bar")); + assertThat(hits[1].field("foo").getValue().toString(), equalTo("baz")); + assertThat(hits[2].field("foo").getValue().toString(), equalTo("eggplant")); + assertThat(hits[3].field("foo").getValue().toString(), equalTo("foo")); + } } From c8e8db473830fce1bdca3c4df80a685e782383bc Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 09:45:50 -0700 Subject: [PATCH 49/56] Clarify comment about pre-defined mappings --- .../TransportShardReplicationOperationAction.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java index bd7dbc7a1f02d..5aa1b7adf82e5 100644 --- a/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java +++ b/src/main/java/org/elasticsearch/action/support/replication/TransportShardReplicationOperationAction.java @@ -636,9 +636,10 @@ void performOnReplica(final ReplicationState state, final ShardRouting shard, fi // perform the action on the replica, so skip it and // immediately return if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings())) { - // this doesn't replicate mappings changes, so can fail if mappings are not predefined - // It was successful on the replica, although we never actually executed - in the future we will - // ack mapping updates with the master and that will sync with replicas. For now this is just fine. + // this delays mapping updates on replicas because they have + // to wait until they get the new mapping through the cluster + // state, which is why we recommend pre-defined mappings for + // indices using shadow replicas state.onReplicaSuccess(); return; } From 73c62df4fc7da8a5ed557620a83910d89b313aa1 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 09:58:02 -0700 Subject: [PATCH 50/56] Add MockShadowEngine and hook it up to be used --- .../test/engine/MockEngineFactory.java | 3 +- .../test/engine/MockInternalEngine.java | 9 +- .../test/engine/MockShadowEngine.java | 119 ++++++++++++++++++ 3 files changed, 126 insertions(+), 5 deletions(-) create mode 100644 src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java diff --git a/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java b/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java index 51dab2041a180..82b69b2bf035a 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java +++ b/src/test/java/org/elasticsearch/test/engine/MockEngineFactory.java @@ -21,7 +21,6 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineFactory; -import org.elasticsearch.index.engine.ShadowEngine; /** * @@ -34,6 +33,6 @@ public Engine newReadWriteEngine(EngineConfig config) { @Override public Engine newReadOnlyEngine(EngineConfig config) { - return new ShadowEngine(config); + return new MockShadowEngine(config); } } diff --git a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java index e9e1037dff7f5..d2ebebe1ffedd 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java +++ b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java @@ -26,6 +26,7 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SearcherManager; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineException; @@ -105,7 +106,7 @@ protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherMa // pass the original searcher to the super.newSearcher() method to make sure this is the searcher that will // be released later on. If we wrap an index reader here must not pass the wrapped version to the manager // on release otherwise the reader will be closed too early. - good news, stuff will fail all over the place if we don't get this right here - return new AssertingSearcher(assertingIndexSearcher, super.newSearcher(source, searcher, manager), shardId); + return new AssertingSearcher(assertingIndexSearcher, super.newSearcher(source, searcher, manager), shardId, logger); } private DirectoryReader wrapReader(DirectoryReader reader) { @@ -132,19 +133,21 @@ private DirectoryReader wrapReader(DirectoryReader reader) { return reader; } - public final class AssertingSearcher extends Searcher { + public static final class AssertingSearcher extends Searcher { private final Searcher wrappedSearcher; private final ShardId shardId; private final IndexSearcher indexSearcher; private RuntimeException firstReleaseStack; private final Object lock = new Object(); private final int initialRefCount; + private final ESLogger logger; - public AssertingSearcher(IndexSearcher indexSearcher, Searcher wrappedSearcher, ShardId shardId) { + public AssertingSearcher(IndexSearcher indexSearcher, Searcher wrappedSearcher, ShardId shardId, ESLogger logger) { super(wrappedSearcher.source(), indexSearcher); // we only use the given index searcher here instead of the IS of the wrapped searcher. the IS might be a wrapped searcher // with a wrapped reader. this.wrappedSearcher = wrappedSearcher; + this.logger = logger; this.shardId = shardId; initialRefCount = wrappedSearcher.reader().getRefCount(); this.indexSearcher = indexSearcher; diff --git a/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java b/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java new file mode 100644 index 0000000000000..9e1ff9c103025 --- /dev/null +++ b/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java @@ -0,0 +1,119 @@ +/* + * 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.test.engine; + +import org.apache.lucene.index.AssertingDirectoryReader; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.AssertingIndexSearcher; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SearcherManager; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.EngineException; +import org.elasticsearch.index.engine.ShadowEngine; +import org.elasticsearch.test.ElasticsearchIntegrationTest; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +public class MockShadowEngine extends ShadowEngine { + + private final MockInternalEngine.MockContext mockContext; + public static final ConcurrentMap INFLIGHT_ENGINE_SEARCHERS = new ConcurrentHashMap<>(); + + public MockShadowEngine(EngineConfig config) { + super(config); + Settings indexSettings = config.getIndexSettings(); + final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.SETTING_INDEX_SEED, 0l); + Random random = new Random(seed); + final double ratio = indexSettings.getAsDouble(MockInternalEngine.WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow + Class wrapper = indexSettings.getAsClass(MockInternalEngine.READER_WRAPPER_TYPE, AssertingDirectoryReader.class); + boolean wrapReader = random.nextDouble() < ratio; + logger.trace("Using [{}] for shard [{}] seed: [{}] wrapReader: [{}]", this.getClass().getName(), shardId, seed, wrapReader); + mockContext = new MockInternalEngine.MockContext(random, wrapReader, wrapper, indexSettings); + } + + + @Override + public void close() throws IOException { + try { + super.close(); + } finally { + if (logger.isTraceEnabled()) { + // log debug if we have pending searchers + for (Map.Entry entry : INFLIGHT_ENGINE_SEARCHERS.entrySet()) { + logger.trace("Unreleased Searchers instance for shard [{}]", entry.getValue(), entry.getKey().shardId()); + } + } + } + } + + @Override + protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherManager manager) throws EngineException { + + IndexReader reader = searcher.getIndexReader(); + IndexReader wrappedReader = reader; + if (reader instanceof DirectoryReader && mockContext.wrapReader) { + wrappedReader = wrapReader((DirectoryReader) reader); + } + // this executes basic query checks and asserts that weights are normalized only once etc. + final AssertingIndexSearcher assertingIndexSearcher = new AssertingIndexSearcher(mockContext.random, wrappedReader); + assertingIndexSearcher.setSimilarity(searcher.getSimilarity()); + // pass the original searcher to the super.newSearcher() method to make + // sure this is the searcher that will be released later on. If we wrap + // an index reader here must not pass the wrapped version to the manager + // on release otherwise the reader will be closed too early. - good + // news, stuff will fail all over the place if we don't get this + // right here + return new MockInternalEngine.AssertingSearcher(assertingIndexSearcher, + super.newSearcher(source, searcher, manager), shardId, logger); + } + + private DirectoryReader wrapReader(DirectoryReader reader) { + try { + Constructor[] constructors = mockContext.wrapper.getConstructors(); + Constructor nonRandom = null; + for (Constructor constructor : constructors) { + Class[] parameterTypes = constructor.getParameterTypes(); + if (parameterTypes.length > 0 && parameterTypes[0] == DirectoryReader.class) { + if (parameterTypes.length == 1) { + nonRandom = constructor; + } else if (parameterTypes.length == 2 && parameterTypes[1] == Settings.class) { + + return (DirectoryReader) constructor.newInstance(reader, mockContext.indexSettings); + } + } + } + if (nonRandom != null) { + return (DirectoryReader) nonRandom.newInstance(reader); + } + } catch (Exception e) { + throw new ElasticsearchException("Can not wrap reader", e); + } + return reader; + } + +} From 1a0d45629457578a60ae5bccbeba05acf5d79ddd Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 09:59:31 -0700 Subject: [PATCH 51/56] Rename usesSharedFilesystem -> isOnSharedFilesystem --- .../elasticsearch/cluster/metadata/IndexMetaData.java | 10 ++++++---- .../java/org/elasticsearch/index/IndexService.java | 4 ++-- .../java/org/elasticsearch/indices/IndicesService.java | 4 ++-- .../elasticsearch/indices/recovery/RecoverySource.java | 6 +----- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index f53853eacafb5..253b27ddc96df 100644 --- a/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -788,11 +788,13 @@ public static void writeTo(IndexMetaData indexMetaData, StreamOutput out) throws } /** - * Returns true iff the given settings indicate that the index associated - * with these settings allocates it's shards on a shared filesystem. Otherwise false. The default - * setting for this is the returned value from {@link #isIndexUsingShadowReplicas(org.elasticsearch.common.settings.Settings)}. + * Returns true iff the given settings indicate that the index + * associated with these settings allocates it's shards on a shared + * filesystem. Otherwise false. The default setting for this + * is the returned value from + * {@link #isIndexUsingShadowReplicas(org.elasticsearch.common.settings.Settings)}. */ - public static boolean usesSharedFilesystem(Settings settings) { + public static boolean isOnSharedFilesystem(Settings settings) { return settings.getAsBoolean(SETTING_SHARED_FILESYSTEM, isIndexUsingShadowReplicas(settings)); } diff --git a/src/main/java/org/elasticsearch/index/IndexService.java b/src/main/java/org/elasticsearch/index/IndexService.java index 22e0c8e14ef9e..74202a6bad1d0 100644 --- a/src/main/java/org/elasticsearch/index/IndexService.java +++ b/src/main/java/org/elasticsearch/index/IndexService.java @@ -305,8 +305,8 @@ public synchronized IndexShard createShard(int sShardId, boolean primary) throws logger.debug("creating shard_id {}", shardId); // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary. - final boolean canDeleteShardContent = IndexMetaData.usesSharedFilesystem(indexSettings) == false || - (primary && IndexMetaData.usesSharedFilesystem(indexSettings)); + final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false || + (primary && IndexMetaData.isOnSharedFilesystem(indexSettings)); ModulesBuilder modules = new ModulesBuilder(); modules.add(new ShardsPluginsModule(indexSettings, pluginsService)); modules.add(new IndexShardModule(shardId, primary, indexSettings)); diff --git a/src/main/java/org/elasticsearch/indices/IndicesService.java b/src/main/java/org/elasticsearch/indices/IndicesService.java index 69e3192d392ab..f0061984b5171 100644 --- a/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -536,7 +536,7 @@ public void deleteShardStore(String reason, ShardId shardId, IndexMetaData metaD */ public boolean canDeleteIndexContents(Index index, Settings indexSettings) { final Tuple indexServiceInjectorTuple = this.indices.get(index); - if (IndexMetaData.usesSharedFilesystem(indexSettings) == false) { + if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) { if (indexServiceInjectorTuple == null && nodeEnv.hasNodeFile()) { return true; } @@ -569,7 +569,7 @@ public boolean canDeleteShardContent(ShardId shardId, IndexMetaData metaData) { private boolean canDeleteShardContent(ShardId shardId, @IndexSettings Settings indexSettings) { final Tuple indexServiceInjectorTuple = this.indices.get(shardId.getIndex()); - if (IndexMetaData.usesSharedFilesystem(indexSettings) == false) { + if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) { if (indexServiceInjectorTuple != null && nodeEnv.hasNodeFile()) { final IndexService indexService = indexServiceInjectorTuple.v1(); return indexService.hasShard(shardId.id()) == false; diff --git a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java index 4b8245e7b5d7a..9ab36b9915e98 100644 --- a/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java +++ b/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java @@ -19,7 +19,6 @@ package org.elasticsearch.indices.recovery; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; @@ -29,12 +28,9 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; -import org.elasticsearch.common.lease.Releasable; -import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.settings.IndexSettings; -import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesLifecycle; @@ -119,7 +115,7 @@ private RecoveryResponse recover(final StartRecoveryRequest request) { logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated()); final ShardRecoveryHandler handler; - if (IndexMetaData.usesSharedFilesystem(shard.indexSettings())) { + if (IndexMetaData.isOnSharedFilesystem(shard.indexSettings())) { handler = new SharedFSRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); } else { handler = new ShardRecoveryHandler(shard, request, recoverySettings, transportService, clusterService, indicesService, mappingUpdatedAction, logger); From 62b0c28df8c23cc0b8205b33f7595c68ff940e2b Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 11:43:17 -0700 Subject: [PATCH 52/56] Use IndexMetaData.isIndexUsingShadowReplicas helper --- .../java/org/elasticsearch/index/shard/IndexShardModule.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java index 98608200e2758..2ba09533eae3d 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShardModule.java @@ -51,7 +51,7 @@ public IndexShardModule(ShardId shardId, boolean primary, Settings settings) { /** Return true if a shadow engine should be used */ protected boolean useShadowEngine() { - return primary == false && settings.getAsBoolean(IndexMetaData.SETTING_SHADOW_REPLICAS, false); + return primary == false && IndexMetaData.isIndexUsingShadowReplicas(settings); } @Override From 67a797a9235d4aa376ff4af16f3944d907df4577 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 13:14:01 -0700 Subject: [PATCH 53/56] Factor out AssertingSearcher so it can be used by mock Engines --- .../test/engine/AssertingSearcher.java | 105 ++++++++++++++++++ .../test/engine/MockInternalEngine.java | 79 +------------ .../test/engine/MockShadowEngine.java | 9 +- .../hamcrest/ElasticsearchAssertions.java | 16 ++- 4 files changed, 127 insertions(+), 82 deletions(-) create mode 100644 src/test/java/org/elasticsearch/test/engine/AssertingSearcher.java diff --git a/src/test/java/org/elasticsearch/test/engine/AssertingSearcher.java b/src/test/java/org/elasticsearch/test/engine/AssertingSearcher.java new file mode 100644 index 0000000000000..b3d893c4362c2 --- /dev/null +++ b/src/test/java/org/elasticsearch/test/engine/AssertingSearcher.java @@ -0,0 +1,105 @@ +/* + * 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.test.engine; + +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.search.IndexSearcher; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.logging.ESLogger; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.shard.ShardId; + +import java.util.Map; + +/** + * A searcher that asserts the IndexReader's refcount on close + */ +public class AssertingSearcher extends Engine.Searcher { + private final Engine.Searcher wrappedSearcher; + private final ShardId shardId; + private final IndexSearcher indexSearcher; + private RuntimeException firstReleaseStack; + private final Object lock = new Object(); + private final int initialRefCount; + private final ESLogger logger; + private final Map inFlightSearchers; + + public AssertingSearcher(IndexSearcher indexSearcher, Engine.Searcher wrappedSearcher, + ShardId shardId, Map inFlightSearchers, + ESLogger logger) { + super(wrappedSearcher.source(), indexSearcher); + // we only use the given index searcher here instead of the IS of the wrapped searcher. the IS might be a wrapped searcher + // with a wrapped reader. + this.wrappedSearcher = wrappedSearcher; + this.logger = logger; + this.shardId = shardId; + initialRefCount = wrappedSearcher.reader().getRefCount(); + this.indexSearcher = indexSearcher; + assert initialRefCount > 0 : "IndexReader#getRefCount() was [" + initialRefCount + "] expected a value > [0] - reader is already closed"; + this.inFlightSearchers = inFlightSearchers; + this.inFlightSearchers.put(this, new RuntimeException("Unreleased Searcher, source [" + wrappedSearcher.source() + "]")); + } + + @Override + public String source() { + return wrappedSearcher.source(); + } + + @Override + public void close() throws ElasticsearchException { + RuntimeException remove = inFlightSearchers.remove(this); + synchronized (lock) { + // make sure we only get this once and store the stack of the first caller! + if (remove == null) { + assert firstReleaseStack != null; + AssertionError error = new AssertionError("Released Searcher more than once, source [" + wrappedSearcher.source() + "]"); + error.initCause(firstReleaseStack); + throw error; + } else { + assert firstReleaseStack == null; + firstReleaseStack = new RuntimeException("Searcher Released first here, source [" + wrappedSearcher.source() + "]"); + } + } + final int refCount = wrappedSearcher.reader().getRefCount(); + // this assert seems to be paranoid but given LUCENE-5362 we better add some assertions here to make sure we catch any potential + // problems. + assert refCount > 0 : "IndexReader#getRefCount() was [" + refCount + "] expected a value > [0] - reader is already closed. Initial refCount was: [" + initialRefCount + "]"; + try { + wrappedSearcher.close(); + } catch (RuntimeException ex) { + logger.debug("Failed to release searcher", ex); + throw ex; + } + } + + @Override + public IndexReader reader() { + return indexSearcher.getIndexReader(); + } + + @Override + public IndexSearcher searcher() { + return indexSearcher; + } + + public ShardId shardId() { + return shardId; + } +} diff --git a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java index d2ebebe1ffedd..0a657ab0ee935 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java +++ b/src/test/java/org/elasticsearch/test/engine/MockInternalEngine.java @@ -26,12 +26,10 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.SearcherManager; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.InternalEngine; -import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ElasticsearchIntegrationTest; import java.io.IOException; @@ -84,8 +82,9 @@ public void close() throws IOException { } finally { if (logger.isTraceEnabled()) { // log debug if we have pending searchers - for (Map.Entry entry : INFLIGHT_ENGINE_SEARCHERS.entrySet()) { - logger.trace("Unreleased Searchers instance for shard [{}]", entry.getValue(), entry.getKey().shardId); + for (Map.Entry entry : INFLIGHT_ENGINE_SEARCHERS.entrySet()) { + logger.trace("Unreleased Searchers instance for shard [{}]", + entry.getValue(), entry.getKey().shardId()); } } } @@ -106,7 +105,8 @@ protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherMa // pass the original searcher to the super.newSearcher() method to make sure this is the searcher that will // be released later on. If we wrap an index reader here must not pass the wrapped version to the manager // on release otherwise the reader will be closed too early. - good news, stuff will fail all over the place if we don't get this right here - return new AssertingSearcher(assertingIndexSearcher, super.newSearcher(source, searcher, manager), shardId, logger); + return new AssertingSearcher(assertingIndexSearcher, + super.newSearcher(source, searcher, manager), shardId, INFLIGHT_ENGINE_SEARCHERS, logger); } private DirectoryReader wrapReader(DirectoryReader reader) { @@ -133,75 +133,6 @@ private DirectoryReader wrapReader(DirectoryReader reader) { return reader; } - public static final class AssertingSearcher extends Searcher { - private final Searcher wrappedSearcher; - private final ShardId shardId; - private final IndexSearcher indexSearcher; - private RuntimeException firstReleaseStack; - private final Object lock = new Object(); - private final int initialRefCount; - private final ESLogger logger; - - public AssertingSearcher(IndexSearcher indexSearcher, Searcher wrappedSearcher, ShardId shardId, ESLogger logger) { - super(wrappedSearcher.source(), indexSearcher); - // we only use the given index searcher here instead of the IS of the wrapped searcher. the IS might be a wrapped searcher - // with a wrapped reader. - this.wrappedSearcher = wrappedSearcher; - this.logger = logger; - this.shardId = shardId; - initialRefCount = wrappedSearcher.reader().getRefCount(); - this.indexSearcher = indexSearcher; - assert initialRefCount > 0 : "IndexReader#getRefCount() was [" + initialRefCount + "] expected a value > [0] - reader is already closed"; - INFLIGHT_ENGINE_SEARCHERS.put(this, new RuntimeException("Unreleased Searcher, source [" + wrappedSearcher.source() + "]")); - } - - @Override - public String source() { - return wrappedSearcher.source(); - } - - @Override - public void close() throws ElasticsearchException { - RuntimeException remove = INFLIGHT_ENGINE_SEARCHERS.remove(this); - synchronized (lock) { - // make sure we only get this once and store the stack of the first caller! - if (remove == null) { - assert firstReleaseStack != null; - AssertionError error = new AssertionError("Released Searcher more than once, source [" + wrappedSearcher.source() + "]"); - error.initCause(firstReleaseStack); - throw error; - } else { - assert firstReleaseStack == null; - firstReleaseStack = new RuntimeException("Searcher Released first here, source [" + wrappedSearcher.source() + "]"); - } - } - final int refCount = wrappedSearcher.reader().getRefCount(); - // this assert seems to be paranoid but given LUCENE-5362 we better add some assertions here to make sure we catch any potential - // problems. - assert refCount > 0 : "IndexReader#getRefCount() was [" + refCount + "] expected a value > [0] - reader is already closed. Initial refCount was: [" + initialRefCount + "]"; - try { - wrappedSearcher.close(); - } catch (RuntimeException ex) { - logger.debug("Failed to release searcher", ex); - throw ex; - } - } - - @Override - public IndexReader reader() { - return indexSearcher.getIndexReader(); - } - - @Override - public IndexSearcher searcher() { - return indexSearcher; - } - - public ShardId shardId() { - return shardId; - } - } - public static abstract class DirectoryReaderWrapper extends FilterDirectoryReader { protected final SubReaderWrapper subReaderWrapper; diff --git a/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java b/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java index 9e1ff9c103025..f6597378ba8ff 100644 --- a/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java +++ b/src/test/java/org/elasticsearch/test/engine/MockShadowEngine.java @@ -42,7 +42,7 @@ public class MockShadowEngine extends ShadowEngine { private final MockInternalEngine.MockContext mockContext; - public static final ConcurrentMap INFLIGHT_ENGINE_SEARCHERS = new ConcurrentHashMap<>(); + public static final ConcurrentMap INFLIGHT_ENGINE_SEARCHERS = new ConcurrentHashMap<>(); public MockShadowEngine(EngineConfig config) { super(config); @@ -64,7 +64,7 @@ public void close() throws IOException { } finally { if (logger.isTraceEnabled()) { // log debug if we have pending searchers - for (Map.Entry entry : INFLIGHT_ENGINE_SEARCHERS.entrySet()) { + for (Map.Entry entry : INFLIGHT_ENGINE_SEARCHERS.entrySet()) { logger.trace("Unreleased Searchers instance for shard [{}]", entry.getValue(), entry.getKey().shardId()); } } @@ -88,8 +88,9 @@ protected Searcher newSearcher(String source, IndexSearcher searcher, SearcherMa // on release otherwise the reader will be closed too early. - good // news, stuff will fail all over the place if we don't get this // right here - return new MockInternalEngine.AssertingSearcher(assertingIndexSearcher, - super.newSearcher(source, searcher, manager), shardId, logger); + return new AssertingSearcher(assertingIndexSearcher, + super.newSearcher(source, searcher, manager), shardId, + INFLIGHT_ENGINE_SEARCHERS, logger); } private DirectoryReader wrapReader(DirectoryReader reader) { diff --git a/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index 05f9e73b7c93a..d7abf2a6205c8 100644 --- a/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/src/test/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -23,7 +23,6 @@ import com.google.common.base.Predicates; import com.google.common.collect.FluentIterable; import com.google.common.collect.Iterables; - import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; import org.elasticsearch.ElasticsearchException; @@ -65,7 +64,9 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.suggest.Suggest; +import org.elasticsearch.test.engine.AssertingSearcher; import org.elasticsearch.test.engine.MockInternalEngine; +import org.elasticsearch.test.engine.MockShadowEngine; import org.elasticsearch.test.store.MockDirectoryHelper; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -643,20 +644,26 @@ public static void assertAllSearchersClosed() { try { if (awaitBusy(new Predicate() { public boolean apply(Object o) { - return MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty(); + return MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty() && + MockShadowEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty(); } }, 5, TimeUnit.SECONDS)) { return; } } catch (InterruptedException ex) { - if (MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty()) { + if (MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty() && + MockShadowEngine.INFLIGHT_ENGINE_SEARCHERS.isEmpty()) { return; } } try { RuntimeException ex = null; StringBuilder builder = new StringBuilder("Unclosed Searchers instance for shards: ["); - for (Map.Entry entry : MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.entrySet()) { + for (Map.Entry entry : MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.entrySet()) { + ex = entry.getValue(); + builder.append(entry.getKey().shardId()).append(","); + } + for (Map.Entry entry : MockShadowEngine.INFLIGHT_ENGINE_SEARCHERS.entrySet()) { ex = entry.getValue(); builder.append(entry.getKey().shardId()).append(","); } @@ -664,6 +671,7 @@ public boolean apply(Object o) { throw new RuntimeException(builder.toString(), ex); } finally { MockInternalEngine.INFLIGHT_ENGINE_SEARCHERS.clear(); + MockShadowEngine.INFLIGHT_ENGINE_SEARCHERS.clear(); } } From edd49434af5de7e55928f27a1c9ed0fddb1fb133 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 14:32:06 -0700 Subject: [PATCH 54/56] Move engine creation into protected createNewEngine method --- .../elasticsearch/index/shard/IndexShard.java | 18 +++---- .../index/shard/ShadowIndexShard.java | 52 +++++++++++++------ 2 files changed, 43 insertions(+), 27 deletions(-) diff --git a/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ea9ca64b84a3a..8b405b903c76b 100644 --- a/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -130,7 +130,6 @@ public class IndexShard extends AbstractIndexShardComponent { private final InternalIndicesLifecycle indicesLifecycle; private final Store store; private final MergeSchedulerProvider mergeScheduler; - private final AtomicReference currentEngineReference = new AtomicReference<>(); private final Translog translog; private final IndexAliasesService indexAliasesService; private final ShardIndexingService indexingService; @@ -151,16 +150,17 @@ public class IndexShard extends AbstractIndexShardComponent { private final Object mutex = new Object(); private final String checkIndexOnStartup; - private final EngineConfig config; - private final EngineFactory engineFactory; private long checkIndexTook = 0; - private volatile IndexShardState state; private TimeValue refreshInterval; private volatile ScheduledFuture refreshScheduledFuture; private volatile ScheduledFuture mergeScheduleFuture; protected volatile ShardRouting shardRouting; + protected volatile IndexShardState state; + protected final AtomicReference currentEngineReference = new AtomicReference<>(); + protected final EngineConfig config; + protected final EngineFactory engineFactory; @Nullable private RecoveryState recoveryState; @@ -1141,19 +1141,13 @@ public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable f } } - private void createNewEngine() { + protected void createNewEngine() { synchronized (mutex) { if (state == IndexShardState.CLOSED) { throw new EngineClosedException(shardId); } assert this.currentEngineReference.get() == null; - // Use the read-only engine for shadow replicas - if (IndexMetaData.isIndexUsingShadowReplicas(this.indexSettings) && - this.shardRouting.primary() == false) { - this.currentEngineReference.set(engineFactory.newReadOnlyEngine(config)); - } else { - this.currentEngineReference.set(engineFactory.newReadWriteEngine(config)); - } + this.currentEngineReference.set(engineFactory.newReadWriteEngine(config)); } } } diff --git a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index 809b27be6c1f0..45f5ed3ac49da 100644 --- a/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -18,13 +18,7 @@ */ package org.elasticsearch.index.shard; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchIllegalStateException; -import org.elasticsearch.action.admin.indices.flush.FlushRequest; -import org.elasticsearch.cluster.ClusterService; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; @@ -37,8 +31,8 @@ import org.elasticsearch.index.cache.query.ShardQueryCache; import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.deletionpolicy.SnapshotDeletionPolicy; +import org.elasticsearch.index.engine.EngineClosedException; import org.elasticsearch.index.engine.EngineFactory; -import org.elasticsearch.index.engine.ShadowEngine; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.ShardFieldData; import org.elasticsearch.index.get.ShardGetService; @@ -60,14 +54,8 @@ import org.elasticsearch.index.warmer.ShardIndexWarmerService; import org.elasticsearch.indices.IndicesLifecycle; import org.elasticsearch.indices.IndicesWarmer; -import org.elasticsearch.indices.cluster.IndicesClusterStateService; -import org.elasticsearch.indices.recovery.RecoveryFailedException; -import org.elasticsearch.indices.recovery.RecoveryState; -import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.threadpool.ThreadPool; -import java.util.concurrent.CountDownLatch; - /** * ShadowIndexShard extends {@link IndexShard} to add file synchronization * from the primary when a flush happens. It also ensures that a replica being @@ -76,9 +64,30 @@ */ public class ShadowIndexShard extends IndexShard { + private final Object mutex = new Object(); + @Inject - public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog, ThreadPool threadPool, MapperService mapperService, IndexQueryParserService queryParserService, IndexCache indexCache, IndexAliasesService indexAliasesService, ShardIndexingService indexingService, ShardGetService getService, ShardSearchService searchService, ShardIndexWarmerService shardWarmerService, ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService, ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory, RecoveryTarget recoveryTarget, ClusterService clusterService) { - super(shardId, indexSettings, indexSettingsService, indicesLifecycle, store, mergeScheduler, translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, indexingService, getService, searchService, shardWarmerService, shardFilterCache, shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService, termVectorsService, indexFieldDataService, indexService, shardSuggestService, shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService, mergePolicyProvider, factory); + public ShadowIndexShard(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService, + IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, + Translog translog, ThreadPool threadPool, MapperService mapperService, + IndexQueryParserService queryParserService, IndexCache indexCache, + IndexAliasesService indexAliasesService, ShardIndexingService indexingService, + ShardGetService getService, ShardSearchService searchService, + ShardIndexWarmerService shardWarmerService, ShardFilterCache shardFilterCache, + ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, + ShardPercolateService shardPercolateService, CodecService codecService, + ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, + IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, + ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer, + SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, + MergePolicyProvider mergePolicyProvider, EngineFactory factory) { + super(shardId, indexSettings, indexSettingsService, indicesLifecycle, store, mergeScheduler, + translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService, + indexingService, getService, searchService, shardWarmerService, shardFilterCache, + shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService, + termVectorsService, indexFieldDataService, indexService, shardSuggestService, + shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService, + mergePolicyProvider, factory); } /** @@ -100,4 +109,17 @@ public IndexShard routingEntry(ShardRouting newRouting) { } return this; } + + @Override + protected void createNewEngine() { + synchronized (mutex) { + if (state == IndexShardState.CLOSED) { + throw new EngineClosedException(shardId); + } + assert this.currentEngineReference.get() == null; + assert this.shardRouting.primary() == false; + // Use the read-only engine for shadow replicas + this.currentEngineReference.set(engineFactory.newReadOnlyEngine(config)); + } + } } From 325acbe4585179190a959ba3101ee63b99f1931a Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 14:32:41 -0700 Subject: [PATCH 55/56] Use ?preference=_primary automatically for realtime GET operations --- docs/reference/indices/shadow-replicas.asciidoc | 5 +++++ .../action/get/TransportGetAction.java | 13 +++++++++++-- .../index/IndexWithShadowReplicasTests.java | 14 +++++++++++--- 3 files changed, 27 insertions(+), 5 deletions(-) diff --git a/docs/reference/indices/shadow-replicas.asciidoc b/docs/reference/indices/shadow-replicas.asciidoc index 240abfb1bcac9..1f4db1c90cf20 100644 --- a/docs/reference/indices/shadow-replicas.asciidoc +++ b/docs/reference/indices/shadow-replicas.asciidoc @@ -56,6 +56,11 @@ filesystem where the shadow replica resides (after an Elasticsearch "flush"), a regular refresh (governed by the `index.refresh_interval`) can be used to make the new data searchable. +NOTE: Since documents are only indexed on the primary shard, realtime GET +requests could fail to return a document if executed on the replica shard, +therefore, GET API requests automatically have the `?preference=_primary` flag +set if there is no preference flag already set. + In order to ensure the data is being synchronized in a fast enough manner, you may need to tune the flush threshold for the index to a desired number. A flush is needed to fsync segment files to disk, so they will be visible to all other diff --git a/src/main/java/org/elasticsearch/action/get/TransportGetAction.java b/src/main/java/org/elasticsearch/action/get/TransportGetAction.java index 0da7d9a89c4f0..247d8589de5e6 100644 --- a/src/main/java/org/elasticsearch/action/get/TransportGetAction.java +++ b/src/main/java/org/elasticsearch/action/get/TransportGetAction.java @@ -25,13 +25,14 @@ import org.elasticsearch.action.support.single.shard.TransportShardSingleOperationAction; import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.IndexService; -import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -74,6 +75,14 @@ protected void resolveRequest(ClusterState state, InternalRequest request) { if (request.request().realtime == null) { request.request().realtime = this.realtime; } + IndexMetaData indexMeta = state.getMetaData().index(request.concreteIndex()); + if (request.request().realtime && // if the realtime flag is set + request.request().preference() == null && // the preference flag is not already set + indexMeta != null && // and we have the index + IndexMetaData.isIndexUsingShadowReplicas(indexMeta.settings())) { // and the index uses shadow replicas + // set the preference for the request to use "_primary" automatically + request.request().preference("_primary"); + } // update the routing (request#index here is possibly an alias) request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index())); // Fail fast on the node that received the request. diff --git a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java index 5655e9e61fb98..3850ac0f5d2ee 100644 --- a/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java +++ b/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasTests.java @@ -75,14 +75,22 @@ public void testIndexWithFewDocuments() throws Exception { // replay the translog, this is what this tests client().prepareIndex(IDX, "doc", "1").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "2").setSource("foo", "bar").get(); + + // Check that we can get doc 1 and 2, because we are doing realtime + // gets and getting from the primary + GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setRealtime(true).setFields("foo").get(); + GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setRealtime(true).setFields("foo").get(); + assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); + assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); + flushAndRefresh(IDX); client().prepareIndex(IDX, "doc", "3").setSource("foo", "bar").get(); client().prepareIndex(IDX, "doc", "4").setSource("foo", "bar").get(); refresh(); - // Check that we can get doc 1 and 2 - GetResponse gResp1 = client().prepareGet(IDX, "doc", "1").setFields("foo").get(); - GetResponse gResp2 = client().prepareGet(IDX, "doc", "2").setFields("foo").get(); + // Check that we can get doc 1 and 2 without realtime + gResp1 = client().prepareGet(IDX, "doc", "1").setRealtime(false).setFields("foo").get(); + gResp2 = client().prepareGet(IDX, "doc", "2").setRealtime(false).setFields("foo").get(); assertThat(gResp1.getField("foo").getValue().toString(), equalTo("bar")); assertThat(gResp2.getField("foo").getValue().toString(), equalTo("bar")); From 20835037c98e7d2fac4206c372717a05a27c4790 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 18 Feb 2015 15:27:17 -0700 Subject: [PATCH 56/56] Use Enum for "_primary" preference --- .../java/org/elasticsearch/action/get/TransportGetAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/main/java/org/elasticsearch/action/get/TransportGetAction.java b/src/main/java/org/elasticsearch/action/get/TransportGetAction.java index 247d8589de5e6..39626a18b076a 100644 --- a/src/main/java/org/elasticsearch/action/get/TransportGetAction.java +++ b/src/main/java/org/elasticsearch/action/get/TransportGetAction.java @@ -26,6 +26,7 @@ import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.routing.Preference; import org.elasticsearch.cluster.routing.ShardIterator; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; @@ -81,7 +82,7 @@ protected void resolveRequest(ClusterState state, InternalRequest request) { indexMeta != null && // and we have the index IndexMetaData.isIndexUsingShadowReplicas(indexMeta.settings())) { // and the index uses shadow replicas // set the preference for the request to use "_primary" automatically - request.request().preference("_primary"); + request.request().preference(Preference.PRIMARY.type()); } // update the routing (request#index here is possibly an alias) request.request().routing(state.metaData().resolveIndexRouting(request.request().routing(), request.request().index()));