summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorBoaz Leskes <b.leskes@gmail.com>2015-11-12 18:13:22 +0100
committerBoaz Leskes <b.leskes@gmail.com>2015-11-13 14:30:23 +0100
commitac0da91bf7b1a2387e35eece610ff4844c7f1ddb (patch)
tree1f9b3f9e7165d4aa674553aaa4fe2c1c7b33670d
parent9673ddc5d06c793d0029931192637bcba2d501d7 (diff)
Extend usage of IndexSetting class
I decided to leave external listeners (used by plugins) alone, for now. Closes #14731
-rw-r--r--core/src/main/java/org/elasticsearch/cluster/action/index/NodeIndexDeletedAction.java5
-rw-r--r--core/src/main/java/org/elasticsearch/env/NodeEnvironment.java73
-rw-r--r--core/src/main/java/org/elasticsearch/index/IndexService.java18
-rw-r--r--core/src/main/java/org/elasticsearch/index/IndexSettings.java28
-rw-r--r--core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java7
-rw-r--r--core/src/main/java/org/elasticsearch/index/engine/Engine.java3
-rw-r--r--core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java22
-rw-r--r--core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java5
-rw-r--r--core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/indexing/ShardIndexingService.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java2
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java13
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/IndexShard.java34
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/IndexShardComponent.java4
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/MergeSchedulerConfig.java12
-rw-r--r--core/src/main/java/org/elasticsearch/index/shard/ShardPath.java6
-rw-r--r--core/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java7
-rw-r--r--core/src/main/java/org/elasticsearch/index/store/Store.java6
-rw-r--r--core/src/main/java/org/elasticsearch/indices/IndicesService.java51
-rw-r--r--core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java7
-rw-r--r--core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java3
-rw-r--r--core/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java2
-rw-r--r--core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java3
-rw-r--r--core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java25
-rw-r--r--core/src/test/java/org/elasticsearch/index/IndexModuleTests.java23
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java20
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java2
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java6
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java8
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java40
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java6
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java4
-rw-r--r--core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java3
-rw-r--r--core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java11
-rw-r--r--core/src/test/java/org/elasticsearch/index/codec/CodecTests.java6
-rw-r--r--core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java63
-rw-r--r--core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java41
-rw-r--r--core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java14
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java6
-rw-r--r--core/src/test/java/org/elasticsearch/index/query/TemplateQueryParserTests.java2
-rw-r--r--core/src/test/java/org/elasticsearch/index/shard/IndexSearcherWrapperTests.java6
-rw-r--r--core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java21
-rw-r--r--core/src/test/java/org/elasticsearch/index/shard/NewPathForShardTests.java11
-rw-r--r--core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java10
-rw-r--r--core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java5
-rw-r--r--core/src/test/java/org/elasticsearch/index/store/StoreTests.java47
-rw-r--r--core/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java4
-rw-r--r--core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java24
-rw-r--r--core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java25
-rw-r--r--core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java15
-rw-r--r--plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisTestUtils.java3
-rw-r--r--plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java8
-rw-r--r--plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java3
-rw-r--r--plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java3
-rw-r--r--plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java4
-rw-r--r--plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java3
-rw-r--r--plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperTestUtils.java2
-rw-r--r--test-framework/src/main/java/org/elasticsearch/test/IndexSettingsModule.java31
-rw-r--r--test-framework/src/main/java/org/elasticsearch/test/engine/MockEngineSupport.java14
-rw-r--r--test-framework/src/main/java/org/elasticsearch/test/engine/MockInternalEngine.java3
-rw-r--r--test-framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java12
62 files changed, 346 insertions, 505 deletions
diff --git a/core/src/main/java/org/elasticsearch/cluster/action/index/NodeIndexDeletedAction.java b/core/src/main/java/org/elasticsearch/cluster/action/index/NodeIndexDeletedAction.java
index 46f9b7e26a..0a5ae5ca54 100644
--- a/core/src/main/java/org/elasticsearch/cluster/action/index/NodeIndexDeletedAction.java
+++ b/core/src/main/java/org/elasticsearch/cluster/action/index/NodeIndexDeletedAction.java
@@ -30,6 +30,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.Index;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
@@ -70,7 +71,7 @@ public class NodeIndexDeletedAction extends AbstractComponent {
listeners.remove(listener);
}
- public void nodeIndexDeleted(final ClusterState clusterState, final String index, final Settings indexSettings, final String nodeId) {
+ public void nodeIndexDeleted(final ClusterState clusterState, final String index, final IndexSettings indexSettings, final String nodeId) {
final DiscoveryNodes nodes = clusterState.nodes();
transportService.sendRequest(clusterState.nodes().masterNode(),
INDEX_DELETED_ACTION_NAME, new NodeIndexDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME);
@@ -91,7 +92,7 @@ public class NodeIndexDeletedAction extends AbstractComponent {
});
}
- private void lockIndexAndAck(String index, DiscoveryNodes nodes, String nodeId, ClusterState clusterState, Settings indexSettings) throws IOException {
+ private void lockIndexAndAck(String index, DiscoveryNodes nodes, String nodeId, ClusterState clusterState, IndexSettings indexSettings) throws IOException {
try {
// we are waiting until we can lock the index / all shards on the node and then we ack the delete of the store to the
// master. If we can't acquire the locks here immediately there might be a shard of this index still holding on to the lock
diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java
index 288a21a76f..3a1b430f98 100644
--- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java
+++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java
@@ -21,12 +21,7 @@ package org.elasticsearch.env;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.store.Lock;
-import org.apache.lucene.store.LockObtainFailedException;
-import org.apache.lucene.store.NativeFSLockFactory;
-import org.apache.lucene.store.SimpleFSDirectory;
+import org.apache.lucene.store.*;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -38,6 +33,7 @@ import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.monitor.fs.FsInfo;
@@ -45,21 +41,8 @@ import org.elasticsearch.monitor.fs.FsProbe;
import java.io.Closeable;
import java.io.IOException;
-import java.nio.file.AtomicMoveNotSupportedException;
-import java.nio.file.DirectoryStream;
-import java.nio.file.FileStore;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardCopyOption;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
-import java.util.Set;
+import java.nio.file.*;
+import java.util.*;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -308,9 +291,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param shardId the id of the shard to delete to delete
* @throws IOException if an IOException occurs
*/
- public void deleteShardDirectorySafe(ShardId shardId, Settings indexSettings) throws IOException {
- // This is to ensure someone doesn't use Settings.EMPTY
- assert indexSettings != Settings.EMPTY;
+ public void deleteShardDirectorySafe(ShardId shardId, IndexSettings indexSettings) throws IOException {
final Path[] paths = availableShardPaths(shardId);
logger.trace("deleting shard {} directory, paths: [{}]", shardId, paths);
try (ShardLock lock = shardLock(shardId)) {
@@ -325,7 +306,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
*
* @throws LockObtainFailedException if any of the locks could not be acquired
*/
- public static void acquireFSLockForPaths(Settings indexSettings, Path... shardPaths) throws IOException {
+ public static void acquireFSLockForPaths(IndexSettings indexSettings, Path... shardPaths) throws IOException {
Lock[] locks = new Lock[shardPaths.length];
Directory[] dirs = new Directory[shardPaths.length];
try {
@@ -359,15 +340,14 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @throws IOException if an IOException occurs
* @throws ElasticsearchException if the write.lock is not acquirable
*/
- public void deleteShardDirectoryUnderLock(ShardLock lock, Settings indexSettings) throws IOException {
- assert indexSettings != Settings.EMPTY;
+ public void deleteShardDirectoryUnderLock(ShardLock lock, IndexSettings indexSettings) throws IOException {
final ShardId shardId = lock.getShardId();
assert isShardLocked(shardId) : "shard " + shardId + " is not locked";
final Path[] paths = availableShardPaths(shardId);
logger.trace("acquiring locks for {}, paths: [{}]", shardId, paths);
acquireFSLockForPaths(indexSettings, paths);
IOUtils.rm(paths);
- if (hasCustomDataPath(indexSettings)) {
+ if (indexSettings.hasCustomDataPath()) {
Path customLocation = resolveCustomLocation(indexSettings, shardId);
logger.trace("acquiring lock for {}, custom path: [{}]", shardId, customLocation);
acquireFSLockForPaths(indexSettings, customLocation);
@@ -397,9 +377,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index being deleted
* @throws IOException if any of the shards data directories can't be locked or deleted
*/
- public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, Settings indexSettings) throws IOException {
- // This is to ensure someone doesn't use Settings.EMPTY
- assert indexSettings != Settings.EMPTY;
+ public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, IndexSettings indexSettings) throws IOException {
final List<ShardLock> locks = lockAllForIndex(index, indexSettings, lockTimeoutMS);
try {
deleteIndexDirectoryUnderLock(index, indexSettings);
@@ -415,13 +393,11 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param index the index to delete
* @param indexSettings settings for the index being deleted
*/
- public void deleteIndexDirectoryUnderLock(Index index, Settings indexSettings) throws IOException {
- // This is to ensure someone doesn't use Settings.EMPTY
- assert indexSettings != Settings.EMPTY;
+ public void deleteIndexDirectoryUnderLock(Index index, IndexSettings indexSettings) throws IOException {
final Path[] indexPaths = indexPaths(index);
logger.trace("deleting index {} directory, paths({}): [{}]", index, indexPaths.length, indexPaths);
IOUtils.rm(indexPaths);
- if (hasCustomDataPath(indexSettings)) {
+ if (indexSettings.hasCustomDataPath()) {
Path customLocation = resolveCustomLocation(indexSettings, index.name());
logger.trace("deleting custom index {} directory [{}]", index, customLocation);
IOUtils.rm(customLocation);
@@ -438,9 +414,9 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @return the {@link ShardLock} instances for this index.
* @throws IOException if an IOException occurs.
*/
- public List<ShardLock> lockAllForIndex(Index index, Settings settings, long lockTimeoutMS) throws IOException {
- final Integer numShards = settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, null);
- if (numShards == null || numShards <= 0) {
+ public List<ShardLock> lockAllForIndex(Index index, IndexSettings settings, long lockTimeoutMS) throws IOException {
+ final int numShards = settings.getNumberOfShards();
+ if (numShards <= 0) {
throw new IllegalArgumentException("settings must contain a non-null > 0 number of shards");
}
logger.trace("locking all shards for index {} - [{}]", index, numShards);
@@ -637,8 +613,8 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* Returns all shard paths excluding custom shard path. Note: Shards are only allocated on one of the
* returned paths. The returned array may contain paths to non-existing directories.
*
- * @see #hasCustomDataPath(org.elasticsearch.common.settings.Settings)
- * @see #resolveCustomLocation(org.elasticsearch.common.settings.Settings, org.elasticsearch.index.shard.ShardId)
+ * @see IndexSettings#hasCustomDataPath()
+ * @see #resolveCustomLocation(IndexSettings, ShardId)
*
*/
public Path[] availableShardPaths(ShardId shardId) {
@@ -781,23 +757,14 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
}
/**
- * @param indexSettings settings for an index
- * @return true if the index has a custom data path
- */
- public static boolean hasCustomDataPath(Settings indexSettings) {
- return indexSettings.get(IndexMetaData.SETTING_DATA_PATH) != null;
- }
-
- /**
* Resolve the custom path for a index's shard.
* Uses the {@code IndexMetaData.SETTING_DATA_PATH} setting to determine
* the root path for the index.
*
* @param indexSettings settings for the index
*/
- private Path resolveCustomLocation(Settings indexSettings) {
- assert indexSettings != Settings.EMPTY;
- String customDataDir = indexSettings.get(IndexMetaData.SETTING_DATA_PATH);
+ private Path resolveCustomLocation(IndexSettings indexSettings) {
+ String customDataDir = indexSettings.customDataPath();
if (customDataDir != null) {
// This assert is because this should be caught by MetaDataCreateIndexService
assert sharedDataPath != null;
@@ -819,7 +786,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index
* @param indexName index to resolve the path for
*/
- private Path resolveCustomLocation(Settings indexSettings, final String indexName) {
+ private Path resolveCustomLocation(IndexSettings indexSettings, final String indexName) {
return resolveCustomLocation(indexSettings).resolve(indexName);
}
@@ -831,7 +798,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index
* @param shardId shard to resolve the path to
*/
- public Path resolveCustomLocation(Settings indexSettings, final ShardId shardId) {
+ public Path resolveCustomLocation(IndexSettings indexSettings, final ShardId shardId) {
return resolveCustomLocation(indexSettings, shardId.index().name()).resolve(Integer.toString(shardId.id()));
}
diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java
index 459d3df876..4687303e46 100644
--- a/core/src/main/java/org/elasticsearch/index/IndexService.java
+++ b/core/src/main/java/org/elasticsearch/index/IndexService.java
@@ -52,12 +52,16 @@ import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store;
-import org.elasticsearch.indices.*;
+import org.elasticsearch.indices.AliasFilterParsingException;
+import org.elasticsearch.indices.InvalidAliasNameException;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
-import java.util.*;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
@@ -343,14 +347,13 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
private void onShardClose(ShardLock lock, boolean ownsShard) {
if (deleted.get()) { // we remove that shards content if this index has been deleted
- final Settings indexSettings = this.getIndexSettings().getSettings();
try {
if (ownsShard) {
try {
- eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings);
+ eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings.getSettings());
} finally {
shardStoreDeleter.deleteShardStore("delete index", lock, indexSettings);
- eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings);
+ eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings.getSettings());
}
}
} catch (IOException e) {
@@ -533,8 +536,9 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
}
public interface ShardStoreDeleter {
- void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException;
- void addPendingDelete(ShardId shardId, Settings indexSettings);
+ void deleteShardStore(String reason, ShardLock lock, IndexSettings indexSettings) throws IOException;
+
+ void addPendingDelete(ShardId shardId, IndexSettings indexSettings);
}
final EngineFactory getEngineFactory() {
diff --git a/core/src/main/java/org/elasticsearch/index/IndexSettings.java b/core/src/main/java/org/elasticsearch/index/IndexSettings.java
index 7d17fa43a3..2b99f14c8c 100644
--- a/core/src/main/java/org/elasticsearch/index/IndexSettings.java
+++ b/core/src/main/java/org/elasticsearch/index/IndexSettings.java
@@ -25,7 +25,6 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import java.util.ArrayList;
@@ -182,7 +181,32 @@ public final class IndexSettings {
* Returns <code>true</code> if the index has a custom data path
*/
public boolean hasCustomDataPath() {
- return NodeEnvironment.hasCustomDataPath(settings);
+ return customDataPath() != null;
+ }
+
+ /**
+ * Returns the customDataPath for this index, if configured. null <code>o.w.</code>.
+ */
+ public String customDataPath() {
+ return settings.get(IndexMetaData.SETTING_DATA_PATH);
+ }
+
+ /**
+ * Returns <code>true</code> iff the given settings indicate that the index
+ * associated with these settings allocates it's shards on a shared
+ * filesystem.
+ */
+ public boolean isOnSharedFilesystem() {
+ return IndexMetaData.isOnSharedFilesystem(getSettings());
+ }
+
+ /**
+ * Returns <code>true</code> iff the given settings indicate that the index associated
+ * with these settings uses shadow replicas. Otherwise <code>false</code>. The default
+ * setting for this is <code>false</code>.
+ */
+ public boolean isIndexUsingShadowReplicas() {
+ return IndexMetaData.isOnSharedFilesystem(getSettings());
}
/**
diff --git a/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java b/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java
index 81466128c5..fa5b89505c 100644
--- a/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java
+++ b/core/src/main/java/org/elasticsearch/index/engine/ElasticsearchConcurrentMergeScheduler.java
@@ -29,6 +29,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.EsExecutors;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.merge.OnGoingMerge;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
@@ -62,11 +63,11 @@ class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
private final Set<OnGoingMerge> readOnlyOnGoingMerges = Collections.unmodifiableSet(onGoingMerges);
private final MergeSchedulerConfig config;
- public ElasticsearchConcurrentMergeScheduler(ShardId shardId, Settings indexSettings, MergeSchedulerConfig config) {
+ public ElasticsearchConcurrentMergeScheduler(ShardId shardId, IndexSettings indexSettings, MergeSchedulerConfig config) {
this.config = config;
this.shardId = shardId;
- this.indexSettings = indexSettings;
- this.logger = Loggers.getLogger(getClass(), indexSettings, shardId);
+ this.indexSettings = indexSettings.getSettings();
+ this.logger = Loggers.getLogger(getClass(), this.indexSettings, shardId);
refreshConfig();
}
diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java
index 3f30354158..0504fdfa99 100644
--- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java
+++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java
@@ -54,7 +54,6 @@ import java.io.IOException;
import java.util.*;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@@ -101,7 +100,7 @@ public abstract class Engine implements Closeable {
this.shardId = engineConfig.getShardId();
this.store = engineConfig.getStore();
this.logger = Loggers.getLogger(Engine.class, // we use the engine class directly here to make sure all subclasses have the same logger name
- engineConfig.getIndexSettings(), engineConfig.getShardId());
+ engineConfig.getIndexSettings().getSettings(), engineConfig.getShardId());
this.eventListener = engineConfig.getEventListener();
this.deletionPolicy = engineConfig.getDeletionPolicy();
}
diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java
index f4337de266..c5da8e83b3 100644
--- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java
+++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java
@@ -25,10 +25,10 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
-import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.shard.MergeSchedulerConfig;
@@ -36,7 +36,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.TranslogConfig;
-import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool;
@@ -50,7 +49,7 @@ import java.util.concurrent.TimeUnit;
public final class EngineConfig {
private final ShardId shardId;
private final TranslogRecoveryPerformer translogRecoveryPerformer;
- private final Settings indexSettings;
+ private final IndexSettings indexSettings;
private volatile ByteSizeValue indexingBufferSize;
private volatile ByteSizeValue versionMapSize;
private volatile String versionMapSizeSetting;
@@ -115,11 +114,12 @@ public final class EngineConfig {
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
*/
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService,
- Settings indexSettings, Engine.Warmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
+ IndexSettings indexSettings, Engine.Warmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) {
this.shardId = shardId;
+ final Settings settings = indexSettings.getSettings();
this.indexSettings = indexSettings;
this.threadPool = threadPool;
this.indexingService = indexingService;
@@ -132,15 +132,15 @@ public final class EngineConfig {
this.similarity = similarity;
this.codecService = codecService;
this.eventListener = eventListener;
- this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
- codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
+ this.compoundOnFlush = settings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
+ codecName = settings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME);
// We start up inactive and rely on IndexingMemoryController to give us our fair share once we start indexing:
indexingBufferSize = IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER;
- gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
- versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
+ gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
+ versionMapSizeSetting = settings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer;
- this.forceNewTranslog = indexSettings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false);
+ this.forceNewTranslog = settings.getAsBoolean(INDEX_FORCE_NEW_TRANSLOG, false);
this.queryCache = queryCache;
this.queryCachingPolicy = queryCachingPolicy;
this.translogConfig = translogConfig;
@@ -314,9 +314,9 @@ public final class EngineConfig {
}
/**
- * Returns the latest index settings directly from the index settings service.
+ * Returns the index settings for this index.
*/
- public Settings getIndexSettings() {
+ public IndexSettings getIndexSettings() {
return indexSettings;
}
diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java
index 9ab09a245e..1404b61b8e 100644
--- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java
+++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java
@@ -42,10 +42,10 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.math.MathUtils;
-import org.elasticsearch.common.settings.Settings;
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.IndexSettings;
import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.merge.MergeStats;
@@ -57,7 +57,6 @@ import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogCorruptedException;
-import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
@@ -1051,7 +1050,7 @@ public class InternalEngine extends Engine {
private final AtomicInteger numMergesInFlight = new AtomicInteger(0);
private final AtomicBoolean isThrottling = new AtomicBoolean();
- EngineMergeScheduler(ShardId shardId, Settings indexSettings, MergeSchedulerConfig config) {
+ EngineMergeScheduler(ShardId shardId, IndexSettings indexSettings, MergeSchedulerConfig config) {
super(shardId, indexSettings, config);
}
diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java
index 46677d3a55..af3e0ae82a 100644
--- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java
+++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java
@@ -68,7 +68,7 @@ public class ShadowEngine extends Engine {
public ShadowEngine(EngineConfig engineConfig) {
super(engineConfig);
SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig);
- final long nonexistentRetryTime = engineConfig.getIndexSettings()
+ final long nonexistentRetryTime = engineConfig.getIndexSettings().getSettings()
.getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT)
.getMillis();
try {
diff --git a/core/src/main/java/org/elasticsearch/index/indexing/ShardIndexingService.java b/core/src/main/java/org/elasticsearch/index/indexing/ShardIndexingService.java
index 601b94bbcb..5cf180c3a2 100644
--- a/core/src/main/java/org/elasticsearch/index/indexing/ShardIndexingService.java
+++ b/core/src/main/java/org/elasticsearch/index/indexing/ShardIndexingService.java
@@ -51,7 +51,7 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
public ShardIndexingService(ShardId shardId, IndexSettings indexSettings) {
super(shardId, indexSettings);
- this.slowLog = new IndexingSlowLog(this.indexSettings);
+ this.slowLog = new IndexingSlowLog(this.indexSettings.getSettings());
}
/**
diff --git a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java
index c7e4dbcaae..eaf562e212 100644
--- a/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java
+++ b/core/src/main/java/org/elasticsearch/index/percolator/PercolatorQueriesRegistry.java
@@ -92,7 +92,7 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
this.indexingService = indexingService;
this.queryShardContext = queryShardContext;
this.indexFieldDataService = indexFieldDataService;
- this.mapUnmappedFieldsAsString = this.indexSettings.getAsBoolean(MAP_UNMAPPED_FIELDS_AS_STRING, false);
+ this.mapUnmappedFieldsAsString = this.indexSettings.getSettings().getAsBoolean(MAP_UNMAPPED_FIELDS_AS_STRING, false);
mapperService.addTypeListener(percolateTypeListener);
}
diff --git a/core/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java b/core/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java
index d39f4d19e5..c8719a610e 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/AbstractIndexShardComponent.java
@@ -22,7 +22,6 @@ package org.elasticsearch.index.shard;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
-import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings;
/**
@@ -33,12 +32,12 @@ public abstract class AbstractIndexShardComponent implements IndexShardComponent
protected final ESLogger logger;
protected final DeprecationLogger deprecationLogger;
protected final ShardId shardId;
- protected final Settings indexSettings;
+ protected final IndexSettings indexSettings;
protected AbstractIndexShardComponent(ShardId shardId, IndexSettings indexSettings) {
this.shardId = shardId;
- this.indexSettings = indexSettings.getSettings();
- this.logger = Loggers.getLogger(getClass(), this.indexSettings, shardId);
+ this.indexSettings = indexSettings;
+ this.logger = Loggers.getLogger(getClass(), this.indexSettings.getSettings(), shardId);
this.deprecationLogger = new DeprecationLogger(logger);
}
@@ -48,12 +47,12 @@ public abstract class AbstractIndexShardComponent implements IndexShardComponent
}
@Override
- public Settings indexSettings() {
- return this.indexSettings;
+ public IndexSettings indexSettings() {
+ return indexSettings;
}
public String nodeName() {
- return indexSettings.get("name", "");
+ return indexSettings.getNodeName();
}
diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
index 9c7cf499e5..2370ace046 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShard.java
@@ -30,13 +30,11 @@ import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.ThreadInterruptedException;
import org.elasticsearch.ElasticsearchException;
-import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
@@ -105,8 +103,8 @@ import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoveryFailedException;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.percolator.PercolatorService;
-import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.search.suggest.completion.CompletionFieldStats;
+import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
@@ -205,7 +203,8 @@ public class IndexShard extends AbstractIndexShardComponent {
@Nullable EngineFactory engineFactory,
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider) {
super(shardId, indexSettings);
- this.inactiveTime = this.indexSettings.getAsTime(INDEX_SHARD_INACTIVE_TIME_SETTING, this.indexSettings.getAsTime(INDICES_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5)));
+ final Settings settings = indexSettings.getSettings();
+ this.inactiveTime = settings.getAsTime(INDEX_SHARD_INACTIVE_TIME_SETTING, settings.getAsTime(INDICES_INACTIVE_TIME_SETTING, TimeValue.timeValueMinutes(5)));
this.idxSettings = indexSettings;
this.codecService = new CodecService(mapperService, logger);
this.warmer = provider.getWarmer();
@@ -215,14 +214,14 @@ public class IndexShard extends AbstractIndexShardComponent {
this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
this.store = store;
this.indexEventListener = indexEventListener;
- this.mergeSchedulerConfig = new MergeSchedulerConfig(this.indexSettings);
+ this.mergeSchedulerConfig = new MergeSchedulerConfig(indexSettings);
this.threadPool = provider.getThreadPool();
this.mapperService = mapperService;
this.indexCache = indexCache;
this.indexingService = new ShardIndexingService(shardId, indexSettings);
this.getService = new ShardGetService(indexSettings, this, mapperService);
this.termVectorsService = provider.getTermVectorsService();
- this.searchService = new ShardSearchStats(this.indexSettings);
+ this.searchService = new ShardSearchStats(settings);
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
this.indicesQueryCache = provider.getIndicesQueryCache();
this.shardQueryCache = new ShardRequestCache(shardId, indexSettings);
@@ -230,29 +229,29 @@ public class IndexShard extends AbstractIndexShardComponent {
this.indexFieldDataService = indexFieldDataService;
this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings);
state = IndexShardState.CREATED;
- this.refreshInterval = this.indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL);
- this.flushOnClose = this.indexSettings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true);
+ this.refreshInterval = settings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL);
+ this.flushOnClose = settings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true);
this.path = path;
- this.mergePolicyConfig = new MergePolicyConfig(logger, this.indexSettings);
+ this.mergePolicyConfig = new MergePolicyConfig(logger, settings);
/* create engine config */
logger.debug("state: [CREATED]");
- this.checkIndexOnStartup = this.indexSettings.get("index.shard.check_on_startup", "false");
- this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, this.indexSettings, Translog.Durabilty.REQUEST),
+ this.checkIndexOnStartup = settings.get("index.shard.check_on_startup", "false");
+ this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, settings, Translog.Durabilty.REQUEST),
provider.getBigArrays(), threadPool);
final QueryCachingPolicy cachingPolicy;
// the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis
- if (this.indexSettings.getAsBoolean(IndexModule.QUERY_CACHE_EVERYTHING, false)) {
+ if (settings.getAsBoolean(IndexModule.QUERY_CACHE_EVERYTHING, false)) {
cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
} else {
cachingPolicy = new UsageTrackingQueryCachingPolicy();
}
this.engineConfig = newEngineConfig(translogConfig, cachingPolicy);
- this.flushThresholdOperations = this.indexSettings.getAsInt(INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, this.indexSettings.getAsInt("index.translog.flush_threshold", Integer.MAX_VALUE));
- this.flushThresholdSize = this.indexSettings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB));
- this.disableFlush = this.indexSettings.getAsBoolean(INDEX_TRANSLOG_DISABLE_FLUSH, false);
+ this.flushThresholdOperations = settings.getAsInt(INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, settings.getAsInt("index.translog.flush_threshold", Integer.MAX_VALUE));
+ this.flushThresholdSize = settings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB));
+ this.disableFlush = settings.getAsBoolean(INDEX_TRANSLOG_DISABLE_FLUSH, false);
this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId);
this.provider = provider;
this.searcherWrapper = indexSearcherWrapper;
@@ -1445,10 +1444,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
private String getIndexUUID() {
- assert indexSettings.get(IndexMetaData.SETTING_INDEX_UUID) != null
- || indexSettings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).before(Version.V_0_90_6) :
- "version: " + indexSettings.getAsVersion(IndexMetaData.SETTING_VERSION_CREATED, null) + " uuid: " + indexSettings.get(IndexMetaData.SETTING_INDEX_UUID);
- return indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
+ return indexSettings.getUUID();
}
private DocumentMapperForType docMapper(String type) {
diff --git a/core/src/main/java/org/elasticsearch/index/shard/IndexShardComponent.java b/core/src/main/java/org/elasticsearch/index/shard/IndexShardComponent.java
index 38886032a7..a1665a7d5c 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/IndexShardComponent.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/IndexShardComponent.java
@@ -19,7 +19,7 @@
package org.elasticsearch.index.shard;
-import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.index.IndexSettings;
/**
*
@@ -28,5 +28,5 @@ public interface IndexShardComponent {
ShardId shardId();
- Settings indexSettings();
+ IndexSettings indexSettings();
}
diff --git a/core/src/main/java/org/elasticsearch/index/shard/MergeSchedulerConfig.java b/core/src/main/java/org/elasticsearch/index/shard/MergeSchedulerConfig.java
index f061a95f2a..c329722a13 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/MergeSchedulerConfig.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/MergeSchedulerConfig.java
@@ -22,6 +22,7 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors;
+import org.elasticsearch.index.IndexSettings;
/**
* The merge scheduler (<code>ConcurrentMergeScheduler</code>) controls the execution of
@@ -61,11 +62,12 @@ public final class MergeSchedulerConfig {
private volatile int maxMergeCount;
private final boolean notifyOnMergeFailure;
- public MergeSchedulerConfig(Settings indexSettings) {
- maxThreadCount = indexSettings.getAsInt(MAX_THREAD_COUNT, Math.max(1, Math.min(4, EsExecutors.boundedNumberOfProcessors(indexSettings) / 2)));
- maxMergeCount = indexSettings.getAsInt(MAX_MERGE_COUNT, maxThreadCount + 5);
- this.autoThrottle = indexSettings.getAsBoolean(AUTO_THROTTLE, true);
- notifyOnMergeFailure = indexSettings.getAsBoolean(NOTIFY_ON_MERGE_FAILURE, true);
+ public MergeSchedulerConfig(IndexSettings indexSettings) {
+ final Settings settings = indexSettings.getSettings();
+ maxThreadCount = settings.getAsInt(MAX_THREAD_COUNT, Math.max(1, Math.min(4, EsExecutors.boundedNumberOfProcessors(settings) / 2)));
+ maxMergeCount = settings.getAsInt(MAX_MERGE_COUNT, maxThreadCount + 5);
+ this.autoThrottle = settings.getAsBoolean(AUTO_THROTTLE, true);
+ notifyOnMergeFailure = settings.getAsBoolean(NOTIFY_ON_MERGE_FAILURE, true);
}
/**
diff --git a/core/src/main/java/org/elasticsearch/index/shard/ShardPath.java b/core/src/main/java/org/elasticsearch/index/shard/ShardPath.java
index 327a080cd6..d940d1a93c 100644
--- a/core/src/main/java/org/elasticsearch/index/shard/ShardPath.java
+++ b/core/src/main/java/org/elasticsearch/index/shard/ShardPath.java
@@ -139,7 +139,7 @@ public final class ShardPath {
final Path dataPath;
final Path statePath = loadedPath;
if (indexSettings.hasCustomDataPath()) {
- dataPath = env.resolveCustomLocation(indexSettings.getSettings(), shardId);
+ dataPath = env.resolveCustomLocation(indexSettings, shardId);
} else {
dataPath = statePath;
}
@@ -161,7 +161,7 @@ public final class ShardPath {
if (load.indexUUID.equals(indexUUID) == false && IndexMetaData.INDEX_UUID_NA_VALUE.equals(load.indexUUID) == false) {
logger.warn("{} deleting leftover shard on path: [{}] with a different index UUID", lock.getShardId(), path);
assert Files.isDirectory(path) : path + " is not a directory";
- NodeEnvironment.acquireFSLockForPaths(indexSettings.getSettings(), paths);
+ NodeEnvironment.acquireFSLockForPaths(indexSettings, paths);
IOUtils.rm(path);
}
}
@@ -203,7 +203,7 @@ public final class ShardPath {
final Path statePath;
if (indexSettings.hasCustomDataPath()) {
- dataPath = env.resolveCustomLocation(indexSettings.getSettings(), shardId);
+ dataPath = env.resolveCustomLocation(indexSettings, shardId);
statePath = env.nodePaths()[0].resolve(shardId);
} else {
diff --git a/core/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java b/core/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java
index 5c673b2aeb..60752dd774 100644
--- a/core/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java
+++ b/core/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java
@@ -61,8 +61,9 @@ public class FsDirectoryService extends DirectoryService implements StoreRateLim
return indexStore.rateLimiting();
}
- public static LockFactory buildLockFactory(Settings indexSettings) {
- String fsLock = indexSettings.get("index.store.fs.lock", indexSettings.get("index.store.fs.fs_lock", "native"));
+ public static LockFactory buildLockFactory(IndexSettings indexSettings) {
+ final Settings settings = indexSettings.getSettings();
+ String fsLock = settings.get("index.store.fs.lock", settings.get("index.store.fs.fs_lock", "native"));
LockFactory lockFactory;
if (fsLock.equals("native")) {
lockFactory = NativeFSLockFactory.INSTANCE;
@@ -101,7 +102,7 @@ public class FsDirectoryService extends DirectoryService implements StoreRateLim
protected Directory newFSDirectory(Path location, LockFactory lockFactory) throws IOException {
- final String storeType = indexSettings.get(IndexModule.STORE_TYPE, IndexModule.Type.DEFAULT.getSettingsKey());
+ final String storeType = indexSettings.getSettings().get(IndexModule.STORE_TYPE, IndexModule.Type.DEFAULT.getSettingsKey());
if (IndexModule.Type.FS.match(storeType) || IndexModule.Type.DEFAULT.match(storeType)) {
final FSDirectory open = FSDirectory.open(location, lockFactory); // use lucene defaults
if (open instanceof MMapDirectory && Constants.WINDOWS == false) {
diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java
index 033fb6d18b..729e2b65b2 100644
--- a/core/src/main/java/org/elasticsearch/index/store/Store.java
+++ b/core/src/main/java/org/elasticsearch/index/store/Store.java
@@ -39,6 +39,7 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
+import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.SingleObjectCache;
@@ -116,10 +117,11 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
@Inject
public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException {
super(shardId, indexSettings);
- this.directory = new StoreDirectory(directoryService.newDirectory(), Loggers.getLogger("index.store.deletes", this.indexSettings, shardId));
+ final Settings settings = indexSettings.getSettings();
+ this.directory = new StoreDirectory(directoryService.newDirectory(), Loggers.getLogger("index.store.deletes", settings, shardId));
this.shardLock = shardLock;
this.onClose = onClose;
- final TimeValue refreshInterval = this.indexSettings.getAsTime(INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueSeconds(10));
+ final TimeValue refreshInterval = settings.getAsTime(INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueSeconds(10));
this.statsCache = new StoreStatsCache(refreshInterval, directory, directoryService);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java
index 0a14546969..69832ef370 100644
--- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java
+++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java
@@ -34,7 +34,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractLifecycleComponent;
-import org.elasticsearch.common.inject.*;
+import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
@@ -52,11 +52,10 @@ import org.elasticsearch.index.recovery.RecoveryStats;
import org.elasticsearch.index.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.shard.IllegalIndexShardStateException;
-import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexEventListener;
+import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.IndexStoreConfig;
-import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.PluginsService;
@@ -73,7 +72,6 @@ import java.util.function.Predicate;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
-import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList;
/**
@@ -330,8 +328,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
logger.debug("[{}] closed... (reason [{}])", index, reason);
listener.afterIndexClosed(indexService.index(), indexService.getIndexSettings().getSettings());
if (delete) {
- final Settings indexSettings = indexService.getIndexSettings().getSettings();
- listener.afterIndexDeleted(indexService.index(), indexSettings);
+ final IndexSettings indexSettings = indexService.getIndexSettings();
+ listener.afterIndexDeleted(indexService.index(), indexSettings.getSettings());
// now we are done - try to wipe data on disk if possible
deleteIndexStore(reason, indexService.index(), indexSettings, false);
}
@@ -412,12 +410,12 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
}
Index index = new Index(metaData.getIndex());
- final Settings indexSettings = buildIndexSettings(metaData);
+ final IndexSettings indexSettings = buildIndexSettings(metaData);
deleteIndexStore(reason, index, indexSettings, closed);
}
}
- private void deleteIndexStore(String reason, Index index, Settings indexSettings, boolean closed) throws IOException {
+ private void deleteIndexStore(String reason, Index index, IndexSettings indexSettings, boolean closed) throws IOException {
boolean success = false;
try {
// we are trying to delete the index store here - not a big deal if the lock can't be obtained
@@ -449,7 +447,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @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, IndexSettings indexSettings) throws IOException {
ShardId shardId = lock.getShardId();
logger.trace("{} deleting shard reason [{}]", shardId, reason);
nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings);
@@ -470,7 +468,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
public void deleteShardStore(String reason, ShardId shardId, ClusterState clusterState) throws IOException {
final IndexMetaData metaData = clusterState.getMetaData().indices().get(shardId.getIndex());
- final Settings indexSettings = buildIndexSettings(metaData);
+ final IndexSettings indexSettings = buildIndexSettings(metaData);
if (canDeleteShardContent(shardId, indexSettings) == false) {
throw new IllegalStateException("Can't delete shard " + shardId);
}
@@ -498,14 +496,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* 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
+ * @param indexSettings {@code IndexSettings} for the given index
* @return true if the index can be deleted on this node
*/
- public boolean canDeleteIndexContents(Index index, Settings indexSettings, boolean closed) {
+ public boolean canDeleteIndexContents(Index index, IndexSettings indexSettings, boolean closed) {
final IndexService indexService = this.indices.get(index.name());
// Closed indices may be deleted, even if they are on a shared
// filesystem. Since it is closed we aren't deleting it for relocation
- if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false || closed) {
+ if (indexSettings.isOnSharedFilesystem() == false || closed) {
if (indexService == null && nodeEnv.hasNodeFile()) {
return true;
}
@@ -532,17 +530,17 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
// to decide where the shard content lives. In the future we might even need more info here ie. for shadow replicas
// 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);
+ final IndexSettings indexSettings = buildIndexSettings(metaData);
return canDeleteShardContent(shardId, indexSettings);
}
- private boolean canDeleteShardContent(ShardId shardId, Settings indexSettings) {
+ private boolean canDeleteShardContent(ShardId shardId, IndexSettings indexSettings) {
final IndexService indexService = this.indices.get(shardId.getIndex());
- if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
+ if (indexSettings.isOnSharedFilesystem() == false) {
if (indexService != null && nodeEnv.hasNodeFile()) {
return indexService.hasShard(shardId.id()) == false;
} else if (nodeEnv.hasNodeFile()) {
- if (NodeEnvironment.hasCustomDataPath(indexSettings)) {
+ if (indexSettings.hasCustomDataPath()) {
return Files.exists(nodeEnv.resolveCustomLocation(indexSettings, shardId));
} else {
return FileSystemUtils.exists(nodeEnv.availableShardPaths(shardId));
@@ -554,20 +552,17 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
return false;
}
- private Settings buildIndexSettings(IndexMetaData metaData) {
+ private IndexSettings buildIndexSettings(IndexMetaData metaData) {
// play safe here and make sure that we take node level settings into account.
// we might run on nodes where we use shard FS and then in the future don't delete
// actual content.
- Settings.Builder builder = settingsBuilder();
- builder.put(settings);
- builder.put(metaData.getSettings());
- return builder.build();
+ return new IndexSettings(metaData, settings, Collections.EMPTY_LIST);
}
/**
* Adds a pending delete for the given index shard.
*/
- public void addPendingDelete(ShardId shardId, Settings settings) {
+ public void addPendingDelete(ShardId shardId, IndexSettings settings) {
if (shardId == null) {
throw new IllegalArgumentException("shardId must not be null");
}
@@ -581,7 +576,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
/**
* Adds a pending delete for the given index.
*/
- public void addPendingDelete(Index index, Settings settings) {
+ public void addPendingDelete(Index index, IndexSettings settings) {
PendingDelete pendingDelete = new PendingDelete(index, settings);
addPendingDelete(index, pendingDelete);
}
@@ -600,13 +595,13 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private static final class PendingDelete implements Comparable<PendingDelete> {
final String index;
final int shardId;
- final Settings settings;
+ final IndexSettings settings;
final boolean deleteIndex;
/**
* Creates a new pending delete of an index
*/
- public PendingDelete(ShardId shardId, Settings settings) {
+ public PendingDelete(ShardId shardId, IndexSettings settings) {
this.index = shardId.getIndex();
this.shardId = shardId.getId();
this.settings = settings;
@@ -616,7 +611,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
/**
* Creates a new pending delete of a shard
*/
- public PendingDelete(Index index, Settings settings) {
+ public PendingDelete(Index index, IndexSettings settings) {
this.index = index.getName();
this.shardId = -1;
this.settings = settings;
@@ -648,7 +643,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @param index the index to process the pending deletes for
* @param timeout the timeout used for processing pending deletes
*/
- public void processPendingDeletes(Index index, Settings indexSettings, TimeValue timeout) throws IOException {
+ public void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeout) throws IOException {
logger.debug("{} processing pending deletes", index);
final long startTimeNS = System.nanoTime();
final List<ShardLock> shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis());
diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java
index 74f11b4b92..d1532eb540 100644
--- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java
+++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java
@@ -45,6 +45,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexShardAlreadyExistsException;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper;
@@ -236,15 +237,15 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (logger.isDebugEnabled()) {
logger.debug("[{}] cleaning index, no longer part of the metadata", index);
}
- final Settings indexSettings;
final IndexService idxService = indicesService.indexService(index);
+ final IndexSettings indexSettings;
if (idxService != null) {
- indexSettings = idxService.getIndexSettings().getSettings();
+ indexSettings = idxService.getIndexSettings();
deleteIndex(index, "index no longer part of the metadata");
} else {
final IndexMetaData metaData = previousState.metaData().index(index);
assert metaData != null;
- indexSettings = metaData.getSettings();
+ indexSettings = new IndexSettings(metaData, settings, Collections.EMPTY_LIST);
indicesService.deleteClosedIndex("closed index no longer part of the metadata", metaData, event.state());
}
try {
diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java
index 8960273fc8..80c18ef3d6 100644
--- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java
+++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoverySource.java
@@ -21,7 +21,6 @@ package org.elasticsearch.indices.recovery;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.ClusterService;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
@@ -108,7 +107,7 @@ public class RecoverySource extends AbstractComponent implements IndexEventListe
logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated());
final RecoverySourceHandler handler;
- if (IndexMetaData.isOnSharedFilesystem(shard.indexSettings())) {
+ if (shard.indexSettings().isOnSharedFilesystem()) {
handler = new SharedFSRecoverySourceHandler(shard, request, recoverySettings, transportService, logger);
} else {
handler = new RecoverySourceHandler(shard, request, recoverySettings, transportService, logger);
diff --git a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java
index 6e9505f077..80f458b8f5 100644
--- a/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java
+++ b/core/src/main/java/org/elasticsearch/indices/recovery/RecoveryStatus.java
@@ -77,7 +77,7 @@ public class RecoveryStatus extends AbstractRefCounted {
super("recovery_status");
this.recoveryId = idGenerator.incrementAndGet();
this.listener = listener;
- this.logger = Loggers.getLogger(getClass(), indexShard.indexSettings(), indexShard.shardId());
+ this.logger = Loggers.getLogger(getClass(), indexShard.indexSettings().getSettings(), indexShard.shardId());
this.indexShard = indexShard;
this.sourceNode = sourceNode;
this.shardId = indexShard.shardId();
diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java
index d9a1c37abe..cb0e0fa0f7 100644
--- a/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java
+++ b/core/src/test/java/org/elasticsearch/action/admin/indices/TransportAnalyzeActionTests.java
@@ -34,7 +34,6 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
-import java.util.Collections;
import java.util.List;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@@ -58,7 +57,7 @@ public class TransportAnalyzeActionTests extends ESTestCase {
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.analyzer.custom_analyzer.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter").build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings);
environment = new Environment(settings);
registry = new AnalysisRegistry(null, environment);
analysisService = registry.build(idxSettings);
diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java
index 6a3145647d..acee455bb6 100644
--- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java
+++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java
@@ -26,8 +26,10 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.Index;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
+import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.nio.file.Files;
@@ -38,13 +40,11 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
-import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
-import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.CoreMatchers.equalTo;
@LuceneTestCase.SuppressFileSystems("ExtrasFS") // TODO: fix test to allow extras
public class NodeEnvironmentTests extends ESTestCase {
- private final Settings idxSettings = Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).build();
+ private final IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("foo", Settings.EMPTY);
public void testNodeLockSingleEnvironment() throws IOException {
NodeEnvironment env = newNodeEnvironment(Settings.builder()
@@ -100,9 +100,8 @@ public class NodeEnvironmentTests extends ESTestCase {
Files.createDirectories(path.resolve("0"));
Files.createDirectories(path.resolve("1"));
}
- Settings settings = settingsBuilder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 10)).build();
try {
- env.lockAllForIndex(new Index("foo"), settings, randomIntBetween(0, 10));
+ env.lockAllForIndex(new Index("foo"), idxSettings, randomIntBetween(0, 10));
fail("shard 0 is locked");
} catch (LockObtainFailedException ex) {
// expected
@@ -112,7 +111,7 @@ public class NodeEnvironmentTests extends ESTestCase {
// can lock again?
env.shardLock(new ShardId("foo", 0)).close();
- List<ShardLock> locks = env.lockAllForIndex(new Index("foo"), settings, randomIntBetween(0, 10));
+ List<ShardLock> locks = env.lockAllForIndex(new Index("foo"), idxSettings, randomIntBetween(0, 10));
try {
env.shardLock(new ShardId("foo", 0));
fail("shard is locked");
@@ -244,7 +243,7 @@ public class NodeEnvironmentTests extends ESTestCase {
flipFlop[i] = new AtomicInteger();
}
- Thread[] threads = new Thread[randomIntBetween(2,5)];
+ Thread[] threads = new Thread[randomIntBetween(2, 5)];
final CountDownLatch latch = new CountDownLatch(1);
final int iters = scaledRandomIntBetween(10000, 100000);
for (int i = 0; i < threads.length; i++) {
@@ -257,7 +256,7 @@ public class NodeEnvironmentTests extends ESTestCase {
fail(e.getMessage());
}
for (int i = 0; i < iters; i++) {
- int shard = randomIntBetween(0, counts.length-1);
+ int shard = randomIntBetween(0, counts.length - 1);
try {
try (ShardLock autoCloses = env.shardLock(new ShardId("foo", shard), scaledRandomIntBetween(0, 10))) {
counts[shard].value++;
@@ -293,18 +292,16 @@ public class NodeEnvironmentTests extends ESTestCase {
String[] dataPaths = tmpPaths();
NodeEnvironment env = newNodeEnvironment(dataPaths, "/tmp", Settings.EMPTY);
- Settings s1 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build();
- Settings s2 = Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, "/tmp/foo").build();
+ IndexSettings s1 = IndexSettingsModule.newIndexSettings("myindex", Settings.EMPTY);
+ IndexSettings s2 = IndexSettingsModule.newIndexSettings("myindex", Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, "/tmp/foo").build());
ShardId sid = new ShardId("myindex", 0);
Index i = new Index("myindex");
- assertFalse("no settings should mean no custom data path", NodeEnvironment.hasCustomDataPath(s1));
- assertTrue("settings with path_data should have a custom data path", NodeEnvironment.hasCustomDataPath(s2));
+ assertFalse("no settings should mean no custom data path", s1.hasCustomDataPath());
+ assertTrue("settings with path_data should have a custom data path", s2.hasCustomDataPath());
assertThat(env.availableShardPaths(sid), equalTo(env.availableShardPaths(sid)));
- assertFalse(NodeEnvironment.hasCustomDataPath(s1));
assertThat(env.resolveCustomLocation(s2, sid), equalTo(PathUtils.get("/tmp/foo/0/myindex/0")));
- assertTrue(NodeEnvironment.hasCustomDataPath(s2));
assertThat("shard paths with a custom data_path should contain only regular paths",
env.availableShardPaths(sid),
diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java
index f9ea857932..ee059924c6 100644
--- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java
+++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java
@@ -39,7 +39,6 @@ import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
-import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper;
@@ -83,10 +82,10 @@ public class IndexModuleTests extends ESTestCase {
private NodeServicesProvider nodeServicesProvider;
private IndexService.ShardStoreDeleter deleter = new IndexService.ShardStoreDeleter() {
@Override
- public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException {
+ public void deleteShardStore(String reason, ShardLock lock, IndexSettings indexSettings) throws IOException {
}
@Override
- public void addPendingDelete(ShardId shardId, Settings indexSettings) {
+ public void addPendingDelete(ShardId shardId, IndexSettings indexSettings) {
}
};
@@ -112,7 +111,7 @@ public class IndexModuleTests extends ESTestCase {
super.setUp();
index = new Index("foo");
settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build();
- indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
environment = new Environment(settings);
nodeServicesProvider = newNodeServiceProvider(settings, environment, null);
nodeEnvironment = new NodeEnvironment(settings, environment);
@@ -142,7 +141,7 @@ public class IndexModuleTests extends ESTestCase {
public void testRegisterIndexStore() throws IOException {
final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).put(IndexModule.STORE_TYPE, "foo_store").build();
- IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
module.addIndexStore("foo_store", FooStore::new);
IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
@@ -164,7 +163,7 @@ public class IndexModuleTests extends ESTestCase {
atomicBoolean.set(true);
}
};
- IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
Consumer<Settings> listener = (s) -> {};
module.addIndexSettingsListener(listener);
@@ -213,7 +212,7 @@ public class IndexModuleTests extends ESTestCase {
.put("index.similarity.my_similarity.key", "there is a key")
.put("path.home", createTempDir().toString())
.build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
module.addSimilarity("test_similarity", (string, settings) -> new SimilarityProvider() {
@Override
public String name() {
@@ -241,7 +240,7 @@ public class IndexModuleTests extends ESTestCase {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put("path.home", createTempDir().toString())
.build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
try {
module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) {
@@ -255,7 +254,7 @@ public class IndexModuleTests extends ESTestCase {
.put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null,new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
try {
module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) {
@@ -267,7 +266,7 @@ public class IndexModuleTests extends ESTestCase {
Settings indexSettings = Settings.settingsBuilder()
.put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
try {
module.registerQueryCache("index", IndexQueryCache::new);
fail("only once");
@@ -295,7 +294,7 @@ public class IndexModuleTests extends ESTestCase {
.put(IndexModule.QUERY_CACHE_TYPE, "custom")
.put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
try {
module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
@@ -313,7 +312,7 @@ public class IndexModuleTests extends ESTestCase {
Settings indexSettings = Settings.settingsBuilder()
.put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
+ IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings), null, new AnalysisRegistry(null, environment));
IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.cache().query() instanceof IndexQueryCache);
indexService.close("simon says", false);
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java
index 3e50a71f20..ef13a891a2 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java
@@ -66,7 +66,7 @@ public class AnalysisModuleTests extends ModuleTestCase {
public AnalysisService getAnalysisService(AnalysisRegistry registry, Settings settings) throws IOException {
Index index = new Index("test");
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
return registry.build(idxSettings);
}
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java
index 89ad959446..cd5138b4e0 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisServiceTests.java
@@ -50,7 +50,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testDefaultAnalyzers() throws IOException {
Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).put("path.home", createTempDir().toString()).build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
@@ -60,7 +60,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testOverrideDefaultAnalyzer() throws IOException {
Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
- AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST),
+ AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings),
Collections.singletonMap("default", analyzerProvider("default")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
@@ -72,7 +72,7 @@ public class AnalysisServiceTests extends ESTestCase {
Version version = VersionUtils.randomVersionBetween(getRandom(), Version.V_3_0_0, Version.CURRENT);
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
try {
- AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST),
+ AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings),
Collections.singletonMap("default_index", new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer())),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
fail("Expected ISE");
@@ -85,7 +85,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testBackCompatOverrideDefaultIndexAnalyzer() {
Version version = VersionUtils.randomVersionBetween(getRandom(), VersionUtils.getFirstVersion(), VersionUtils.getPreviousVersion(Version.V_3_0_0));
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
- AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST),
+ AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings),
Collections.singletonMap("default_index", analyzerProvider("default_index")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
@@ -96,7 +96,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testOverrideDefaultSearchAnalyzer() {
Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
- AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST),
+ AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings),
Collections.singletonMap("default_search", analyzerProvider("default_search")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
@@ -110,7 +110,7 @@ public class AnalysisServiceTests extends ESTestCase {
Map<String, AnalyzerProvider> analyzers = new HashMap<>();
analyzers.put("default_index", analyzerProvider("default_index"));
analyzers.put("default_search", analyzerProvider("default_search"));
- AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST),
+ AnalysisService analysisService = new AnalysisService(IndexSettingsModule.newIndexSettings(new Index("index"), settings),
analyzers, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
assertThat(analysisService.defaultSearchAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
@@ -129,7 +129,7 @@ public class AnalysisServiceTests extends ESTestCase {
.put("index.analysis.analyzer.custom_analyzer_1.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer_1.filter", "lowercase", "word_delimiter").build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
try (NamedAnalyzer custom_analyser = analysisService.analyzer("custom_analyzer")) {
assertNotNull(custom_analyser);
@@ -174,7 +174,7 @@ public class AnalysisServiceTests extends ESTestCase {
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.analyzer.custom_analyzer_1.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer_1.filter", "lowercase", "word_delimiter").build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
TokenFilterFactory word_delimiter = analysisService.tokenFilter("word_delimiter");
@@ -185,7 +185,7 @@ public class AnalysisServiceTests extends ESTestCase {
//unconfigured
IndexSettings idxSettings1 = IndexSettingsModule.newIndexSettings(new Index("index"), settingsBuilder()
- .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), Collections.EMPTY_LIST);
+ .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build());
AnalysisService analysisService1 = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings1);
assertSame(analysisService1.tokenFilter("wordDelimiter"), analysisService1.tokenFilter("word_delimiter"));
assertSame(analysisService1.tokenFilter("porterStem"), analysisService1.tokenFilter("porter_stem"));
@@ -195,7 +195,7 @@ public class AnalysisServiceTests extends ESTestCase {
Settings settings = Settings.builder().put("path.home", createTempDir().toString()).build();
Settings indexSettings = settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
AnalysisService otherAnalysisSergice = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
final int numIters = randomIntBetween(5, 20);
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java
index b6ba4b4dba..7d4164939b 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisTestsHelper.java
@@ -49,7 +49,7 @@ public class AnalysisTestsHelper {
if (settings.get(IndexMetaData.SETTING_VERSION_CREATED) == null) {
settings = Settings.builder().put(settings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
}
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
Environment environment = new Environment(settings);
return new AnalysisRegistry(new HunspellService(settings, environment, Collections.EMPTY_MAP), environment).build(idxSettings);
}
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java
index 34f5939559..dd08d47013 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/CharFilterTests.java
@@ -27,8 +27,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule;
-import java.util.Collections;
-
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
/**
@@ -44,7 +42,7 @@ public class CharFilterTests extends ESTokenStreamTestCase {
.putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "my_mapping")
.put("path.home", createTempDir().toString())
.build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter");
@@ -62,7 +60,7 @@ public class CharFilterTests extends ESTokenStreamTestCase {
.putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "html_strip")
.put("path.home", createTempDir().toString())
.build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter");
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java
index 89276a8a24..0eb916826f 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java
@@ -42,9 +42,7 @@ import java.util.Collections;
import java.util.List;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.hasItems;
-import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.*;
/**
*/
@@ -52,7 +50,7 @@ public class CompoundAnalysisTests extends ESTestCase {
public void testDefaultsCompoundAnalysis() throws Exception {
Index index = new Index("test");
Settings settings = getJsonSettings();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings),
Collections.EMPTY_MAP,Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings);
@@ -71,7 +69,7 @@ public class CompoundAnalysisTests extends ESTestCase {
private List<String> analyze(Settings settings, String analyzerName, String text) throws IOException {
Index index = new Index("test");
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings),
Collections.EMPTY_MAP, Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings);
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java b/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java
index c467f642d7..b7bdbb2327 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/NGramTokenizerFactoryTests.java
@@ -22,12 +22,7 @@ package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer;
-import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter;
-import org.apache.lucene.analysis.ngram.EdgeNGramTokenizer;
-import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenFilter;
-import org.apache.lucene.analysis.ngram.Lucene43EdgeNGramTokenizer;
-import org.apache.lucene.analysis.ngram.Lucene43NGramTokenizer;
-import org.apache.lucene.analysis.ngram.NGramTokenizer;
+import org.apache.lucene.analysis.ngram.*;
import org.apache.lucene.analysis.reverse.ReverseStringFilter;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
@@ -42,7 +37,10 @@ import java.io.IOException;
import java.io.StringReader;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Random;
import static com.carrotsearch.randomizedtesting.RandomizedTest.scaledRandomIntBetween;
import static org.hamcrest.Matchers.instanceOf;
@@ -52,7 +50,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final Index index = new Index("test");
final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build();
- IndexSettings indexProperties = IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST);
+ IndexSettings indexProperties = IndexSettingsModule.newIndexSettings(index, indexSettings);
for (String tokenChars : Arrays.asList("letters", "number", "DIRECTIONALITY_UNDEFINED")) {
final Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", tokenChars).build();
try {
@@ -64,7 +62,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
}
for (String tokenChars : Arrays.asList("letter", " digit ", "punctuation", "DIGIT", "CoNtRoL", "dash_punctuation")) {
final Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", tokenChars).build();
- indexProperties = IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST);
+ indexProperties = IndexSettingsModule.newIndexSettings(index, indexSettings);
new NGramTokenizerFactory(indexProperties, null, name, settings).create();
// no exception
@@ -76,7 +74,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build();
final Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 4).putArray("token_chars", new String[0]).build();
- Tokenizer tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
tokenizer.setReader(new StringReader("1.34"));
assertTokenStreamContents(tokenizer, new String[] {"1.", "1.3", "1.34", ".3", ".34", "34"});
}
@@ -87,12 +85,12 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build();
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit").build();
- Tokenizer tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
tokenizer.setReader(new StringReader("Åbc déf g\uD801\uDC00f "));
assertTokenStreamContents(tokenizer,
new String[] {"Åb", "Åbc", "bc", "dé", "déf", "éf", "g\uD801\uDC00", "g\uD801\uDC00f", "\uD801\uDC00f"});
settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit,punctuation,whitespace,symbol").build();
- tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ tokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
tokenizer.setReader(new StringReader(" a!$ 9"));
assertTokenStreamContents(tokenizer,
new String[] {" a", " a!", "a!", "a!$", "!$", "!$ ", "$ ", "$ 9", " 9"});
@@ -104,12 +102,12 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build();
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit").build();
- Tokenizer tokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer tokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
tokenizer.setReader(new StringReader("Åbc déf g\uD801\uDC00f "));
assertTokenStreamContents(tokenizer,
new String[] {"Åb", "Åbc", "dé", "déf", "g\uD801\uDC00", "g\uD801\uDC00f"});
settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit,punctuation,whitespace,symbol").build();
- tokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ tokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
tokenizer.setReader(new StringReader(" a!$ 9"));
assertTokenStreamContents(tokenizer,
new String[] {" a", " a!"});
@@ -130,7 +128,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
}
Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
- Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
edgeNGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) {
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class));
@@ -141,7 +139,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("side", "back").build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
- Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer edgeNGramTokenizer = new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
edgeNGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class));
}
@@ -149,7 +147,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("side", "back").build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
try {
- new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ new EdgeNGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
fail("should fail side:back is not supported anymore");
} catch (IllegalArgumentException ex) {
}
@@ -170,7 +168,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
}
Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
- Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
nGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) {
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class));
@@ -181,7 +179,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
- Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create();
+ Tokenizer nGramTokenizer = new NGramTokenizerFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create();
nGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class));
}
@@ -208,7 +206,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer tokenizer = new MockTokenizer();
tokenizer.setReader(new StringReader("foo bar"));
- TokenStream edgeNGramTokenFilter = new EdgeNGramTokenFilterFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create(tokenizer);
+ TokenStream edgeNGramTokenFilter = new EdgeNGramTokenFilterFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create(tokenizer);
if (reverse) {
assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class));
} else if (compatVersion) {
@@ -227,7 +225,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer tokenizer = new MockTokenizer();
tokenizer.setReader(new StringReader("foo bar"));
- TokenStream edgeNGramTokenFilter = new EdgeNGramTokenFilterFactory(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), null, name, settings).create(tokenizer);
+ TokenStream edgeNGramTokenFilter = new EdgeNGramTokenFilterFactory(IndexSettingsModule.newIndexSettings(index, indexSettings), null, name, settings).create(tokenizer);
if (reverse) {
assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class));
} else {
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java b/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java
index 6b11685817..4b7119df01 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/PatternCaptureTokenFilterTests.java
@@ -28,8 +28,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule;
-import java.util.Collections;
-
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.containsString;
@@ -42,7 +40,7 @@ public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.emptyList());
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("single");
@@ -60,7 +58,7 @@ public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase {
public void testNoPatterns() {
try {
- new PatternCaptureGroupTokenFilterFactory(IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.EMPTY_LIST), null, "pattern_capture", settingsBuilder().put("pattern", "foobar").build());
+ new PatternCaptureGroupTokenFilterFactory(IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY), null, "pattern_capture", settingsBuilder().put("pattern", "foobar").build());
fail ("Expected IllegalArgumentException");
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("required setting 'patterns' is missing"));
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java b/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java
index 9aea985809..90e55e98d7 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/StopAnalyzerTests.java
@@ -28,8 +28,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule;
-import java.util.Collections;
-
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
public class StopAnalyzerTests extends ESTokenStreamTestCase {
@@ -40,7 +38,7 @@ public class StopAnalyzerTests extends ESTokenStreamTestCase {
.put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.emptyList());
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("analyzer1");
diff --git a/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java b/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java
index 67b90590fb..3a6adca1c6 100644
--- a/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java
+++ b/core/src/test/java/org/elasticsearch/index/analysis/synonyms/SynonymsAnalysisTests.java
@@ -42,7 +42,6 @@ import java.io.IOException;
import java.io.InputStream;
import java.nio.file.Files;
import java.nio.file.Path;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo;
@@ -68,7 +67,7 @@ public class SynonymsAnalysisTests extends ESTestCase {
.put("path.home", home)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.emptyList());
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), settings);
analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
diff --git a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java
index de763056fa..56bf966dd4 100644
--- a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java
+++ b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java
@@ -23,13 +23,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.LogByteSizeMergePolicy;
-import org.apache.lucene.index.Term;
+import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.join.BitSetProducer;
@@ -47,7 +41,6 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
-import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
@@ -55,7 +48,7 @@ import static org.hamcrest.Matchers.equalTo;
public class BitSetFilterCacheTests extends ESTestCase {
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.emptyList());
+ private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY);
private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null);
diff --git a/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java b/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java
index b54c7806d6..2e2aa8e96b 100644
--- a/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java
+++ b/core/src/test/java/org/elasticsearch/index/codec/CodecTests.java
@@ -39,19 +39,15 @@ import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
-import org.elasticsearch.Version;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
-import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.similarity.SimilarityService;
-import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
@@ -110,7 +106,7 @@ public class CodecTests extends ESTestCase {
Settings nodeSettings = settingsBuilder()
.put("path.home", createTempDir())
.build();
- IndexSettings settings = IndexSettingsModule.newIndexSettings(new Index("_na"), nodeSettings, Collections.emptyList());
+ IndexSettings settings = IndexSettingsModule.newIndexSettings(new Index("_na"), nodeSettings);
SimilarityService similarityService = new SimilarityService(settings, Collections.EMPTY_MAP);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(settings);
MapperService service = new MapperService(settings, analysisService, similarityService);
diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
index a8acdfa13f..2937461cb8 100644
--- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
+++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java
@@ -62,25 +62,13 @@ import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.indexing.ShardIndexingService;
-import org.elasticsearch.index.mapper.ContentPath;
-import org.elasticsearch.index.mapper.DocumentMapper;
-import org.elasticsearch.index.mapper.DocumentMapperForType;
-import org.elasticsearch.index.mapper.DocumentMapperParser;
+import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
-import org.elasticsearch.index.mapper.MapperBuilders;
-import org.elasticsearch.index.mapper.MapperService;
-import org.elasticsearch.index.mapper.Mapping;
-import org.elasticsearch.index.mapper.MetadataFieldMapper;
import org.elasticsearch.index.mapper.ParseContext.Document;
-import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper;
-import org.elasticsearch.index.shard.IndexSearcherWrapper;
-import org.elasticsearch.index.shard.MergeSchedulerConfig;
-import org.elasticsearch.index.shard.ShardId;
-import org.elasticsearch.index.shard.ShardUtils;
-import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
+import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
@@ -101,16 +89,10 @@ import java.nio.charset.Charset;
import java.nio.file.DirectoryStream;
import java.nio.file.Files;
import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Locale;
-import java.util.Map;
+import java.util.*;
import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@@ -118,17 +100,12 @@ import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap;
import static org.elasticsearch.index.engine.Engine.Operation.Origin.PRIMARY;
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.*;
public class InternalEngineTests extends ESTestCase {
protected final ShardId shardId = new ShardId(new Index("index"), 1);
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.EMPTY, Collections.emptyList());
+ private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.EMPTY);
protected ThreadPool threadPool;
@@ -138,7 +115,7 @@ public class InternalEngineTests extends ESTestCase {
protected InternalEngine engine;
protected InternalEngine replicaEngine;
- private Settings defaultSettings;
+ private IndexSettings defaultSettings;
private String codecName;
private Path primaryTranslogDir;
private Path replicaTranslogDir;
@@ -157,12 +134,12 @@ public class InternalEngineTests extends ESTestCase {
} else {
codecName = "default";
}
- defaultSettings = Settings.builder()
+ defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.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_CODEC_SETTING, codecName)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
- .build(); // TODO randomize more settings
+ .build()); // TODO randomize more settings
threadPool = new ThreadPool(getClass().getName());
store = createStore();
storeReplica = createStore();
@@ -254,13 +231,13 @@ public class InternalEngineTests extends ESTestCase {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy());
}
- protected InternalEngine createEngine(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
+ protected InternalEngine createEngine(IndexSettings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
return new InternalEngine(config(indexSettings, store, translogPath, mergeSchedulerConfig, mergePolicy), false);
}
- public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
+ public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
IndexWriterConfig iwc = newIndexWriterConfig();
- TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, IndexSettingsModule.newIndexSettings(shardId.index(), indexSettings, Collections.emptyList()), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
+ TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, INDEX_SETTINGS), indexSettings
, null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig,
@@ -284,12 +261,12 @@ public class InternalEngineTests extends ESTestCase {
public void testSegments() throws Exception {
try (Store store = createStore();
- Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), NoMergePolicy.INSTANCE)) {
+ Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), NoMergePolicy.INSTANCE)) {
List<Segment> segments = engine.segments(false);
assertThat(segments.isEmpty(), equalTo(true));
assertThat(engine.segmentsStats().getCount(), equalTo(0l));
assertThat(engine.segmentsStats().getMemoryInBytes(), equalTo(0l));
- final boolean defaultCompound = defaultSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, true);
+ final boolean defaultCompound = defaultSettings.getSettings().getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, true);
// create a doc and refresh
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
@@ -1586,7 +1563,7 @@ public class InternalEngineTests extends ESTestCase {
// #10312
public void testDeletesAloneCanTriggerRefresh() throws Exception {
try (Store store = createStore();
- Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()),
+ Engine engine = new InternalEngine(config(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), newMergePolicy()),
false)) {
engine.config().setIndexingBufferSize(new ByteSizeValue(1, ByteSizeUnit.KB));
for (int i = 0; i < 100; i++) {
@@ -1636,7 +1613,9 @@ public class InternalEngineTests extends ESTestCase {
// expected
}
// now it should be OK.
- Settings indexSettings = Settings.builder().put(defaultSettings).put(EngineConfig.INDEX_FORCE_NEW_TRANSLOG, true).build();
+ IndexSettings indexSettings = new IndexSettings(defaultSettings.getIndexMetaData(),
+ Settings.builder().put(defaultSettings.getSettings()).put(EngineConfig.INDEX_FORCE_NEW_TRANSLOG, true).build(),
+ Collections.EMPTY_LIST);
engine = createEngine(indexSettings, store, primaryTranslogDir, new MergeSchedulerConfig(indexSettings), newMergePolicy());
}
@@ -1776,7 +1755,7 @@ public class InternalEngineTests extends ESTestCase {
}
CommitStats commitStats = engine.commitStats();
Map<String, String> userData = commitStats.getUserData();
- assertTrue("userdata dosn't contain uuid",userData.containsKey(Translog.TRANSLOG_UUID_KEY));
+ assertTrue("userdata dosn't contain uuid", userData.containsKey(Translog.TRANSLOG_UUID_KEY));
assertTrue("userdata doesn't contain generation key", userData.containsKey(Translog.TRANSLOG_GENERATION_KEY));
assertFalse("userdata contains legacy marker", userData.containsKey("translog_id"));
}
@@ -1919,7 +1898,7 @@ public class InternalEngineTests extends ESTestCase {
Settings settings = Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder("test");
Index index = new Index(indexName);
- IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings);
AnalysisService analysisService = new AnalysisService(indexSettings, Collections.EMPTY_MAP, Collections.EMPTY_MAP, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
SimilarityService similarityService = new SimilarityService(indexSettings, Collections.EMPTY_MAP);
MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService);
@@ -1968,12 +1947,12 @@ public class InternalEngineTests extends ESTestCase {
EngineConfig config = engine.config();
/* create a TranslogConfig that has been created with a different UUID */
- TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), IndexSettingsModule.newIndexSettings(shardId.index(), config.getIndexSettings(), Collections.EMPTY_LIST), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
+ TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener()
- , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5));
+ , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5));
try {
new InternalEngine(brokenConfig, false);
diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java
index 4ab3b81b59..3fe7a540bf 100644
--- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java
+++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java
@@ -23,13 +23,7 @@ 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.IndexWriterConfig;
-import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
-import org.apache.lucene.index.LiveIndexWriterConfig;
-import org.apache.lucene.index.MergePolicy;
-import org.apache.lucene.index.NoMergePolicy;
-import org.apache.lucene.index.SnapshotDeletionPolicy;
-import org.apache.lucene.index.Term;
+import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory;
@@ -73,25 +67,17 @@ import org.junit.Before;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Arrays;
-import java.util.Collections;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.hasKey;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.*;
/**
* TODO: document me!
*/
public class ShadowEngineTests extends ESTestCase {
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.EMPTY, Collections.emptyList());
-
protected final ShardId shardId = new ShardId(new Index("index"), 1);
protected ThreadPool threadPool;
@@ -103,7 +89,7 @@ public class ShadowEngineTests extends ESTestCase {
protected Engine primaryEngine;
protected Engine replicaEngine;
- private Settings defaultSettings;
+ private IndexSettings defaultSettings;
private String codecName;
private Path dirPath;
@@ -120,12 +106,13 @@ public class ShadowEngineTests extends ESTestCase {
} else {
codecName = "default";
}
- defaultSettings = Settings.builder()
+ defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.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_CODEC_SETTING, codecName)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
- .build(); // TODO randomize more settings
+ .build()); // TODO randomize more settings
+
threadPool = new ThreadPool(getClass().getName());
dirPath = createTempDir();
store = createStore(dirPath);
@@ -185,7 +172,7 @@ public class ShadowEngineTests extends ESTestCase {
protected Store createStore(final Directory directory) throws IOException {
- IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(shardId.index(), Settings.EMPTY, Collections.emptyList());
+ IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(shardId.index(), Settings.EMPTY);
final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
@Override
public Directory newDirectory() throws IOException {
@@ -212,22 +199,22 @@ public class ShadowEngineTests extends ESTestCase {
return createInternalEngine(defaultSettings, store, translogPath);
}
- protected ShadowEngine createShadowEngine(Settings indexSettings, Store store) {
+ protected ShadowEngine createShadowEngine(IndexSettings indexSettings, Store store) {
return new ShadowEngine(config(indexSettings, store, null, new MergeSchedulerConfig(indexSettings), null));
}
- protected InternalEngine createInternalEngine(Settings indexSettings, Store store, Path translogPath) {
+ protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath) {
return createInternalEngine(indexSettings, store, translogPath, newMergePolicy());
}
- protected InternalEngine createInternalEngine(Settings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) {
+ protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) {
return new InternalEngine(config(indexSettings, store, translogPath, new MergeSchedulerConfig(indexSettings), mergePolicy), true);
}
- public EngineConfig config(Settings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
+ public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergeSchedulerConfig mergeSchedulerConfig, MergePolicy mergePolicy) {
IndexWriterConfig iwc = newIndexWriterConfig();
- TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, IndexSettingsModule.newIndexSettings(shardId.index(), indexSettings, Collections.EMPTY_LIST), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
- EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, IndexSettingsModule.newIndexSettings(shardId.index(), indexSettings, Collections.EMPTY_LIST)), indexSettings
+ TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, threadPool);
+ EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, indexSettings), indexSettings
, null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig,
iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(null, logger), new Engine.EventListener() {
@Override
@@ -282,7 +269,7 @@ public class ShadowEngineTests extends ESTestCase {
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);
+ final boolean defaultCompound = defaultSettings.getSettings().getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, true);
// create a doc and refresh
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java
index 2442287079..b2f818c89f 100644
--- a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java
+++ b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java
@@ -26,20 +26,20 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.index.*;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Accountable;
-import org.elasticsearch.Version;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.index.fielddata.plain.*;
+import org.elasticsearch.index.Index;
+import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.fielddata.plain.PagedBytesAtomicFieldData;
+import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData;
+import org.elasticsearch.index.fielddata.plain.SortedNumericDVIndexFieldData;
+import org.elasticsearch.index.fielddata.plain.SortedSetDVOrdinalsIndexFieldData;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.core.*;
-import org.elasticsearch.index.Index;
-import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
@@ -196,7 +196,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
ThreadPool threadPool = new ThreadPool("random_threadpool_name");
try {
IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null, threadPool);
- IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.EMPTY_LIST), cache, null, null);
+ IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY), cache, null, null);
ft.setNames(new Names("some_long"));
ft.setHasDocValues(true);
ifds.getForField(ft); // no exception
diff --git a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java
index 90018e04af..4631036f13 100644
--- a/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java
+++ b/core/src/test/java/org/elasticsearch/index/query/AbstractQueryTestCase.java
@@ -22,13 +22,11 @@ package org.elasticsearch.index.query;
import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator;
import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.io.JsonStringEncoder;
-
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.Query;
-import org.apache.lucene.util.Accountable;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.spans.SpanBoostQuery;
-import org.elasticsearch.ElasticsearchException;
+import org.apache.lucene.util.Accountable;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
@@ -172,7 +170,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
Settings indexSettings = Settings.settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
index = new Index(randomAsciiOfLengthBetween(1, 10));
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.emptyList());
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings);
final TestClusterService clusterService = new TestClusterService();
clusterService.setState(new ClusterState.Builder(clusterService.state()).metaData(new MetaData.Builder().put(
new IndexMetaData.Builder(index.name()).settings(indexSettings).numberOfShards(1).numberOfReplicas(0))));
diff --git a/core/src/test/java/org/elasticsearch/index/query/TemplateQueryParserTests.java b/core/src/test/java/org/elasticsearch/index/query/TemplateQueryParserTests.java
index bcd20fd366..f017355fd3 100644
--- a/core/src/test/java/org/elasticsearch/index/query/TemplateQueryParserTests.java
+++ b/core/src/test/java/org/elasticsearch/index/query/TemplateQueryParserTests.java
@@ -91,7 +91,7 @@ public class TemplateQueryParserTests extends ESTestCase {
throw new UnsupportedOperationException("client is just a dummy");
});
Index index = new Index("test");
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
injector = new ModulesBuilder().add(
new EnvironmentModule(new Environment(settings)),
new SettingsModule(settings),
diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexSearcherWrapperTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexSearcherWrapperTests.java
index b66e5e5c49..76f20afc69 100644
--- a/core/src/test/java/org/elasticsearch/index/shard/IndexSearcherWrapperTests.java
+++ b/core/src/test/java/org/elasticsearch/index/shard/IndexSearcherWrapperTests.java
@@ -24,17 +24,12 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs;
-import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
-import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.engine.Engine;
-import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.test.ESTestCase;
@@ -46,7 +41,6 @@ import java.util.concurrent.atomic.AtomicInteger;
/**
*/
public class IndexSearcherWrapperTests extends ESTestCase {
- private static final EngineConfig ENGINE_CONFIG = new EngineConfig(null, null, null, Settings.EMPTY, null, null, null, null, null, null, new DefaultSimilarity(), null, null, null, null, QueryCachingPolicy.ALWAYS_CACHE, null, TimeValue.timeValueMinutes(5));
public void testReaderCloseListenerIsCalled() throws IOException {
Directory dir = newDirectory();
diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
index 18dc07d4ef..7422820a81 100644
--- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
+++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java
@@ -62,7 +62,6 @@ import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.engine.Engine;
-import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldData;
@@ -80,6 +79,7 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESSingleNodeTestCase;
+import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.test.VersionUtils;
import java.io.IOException;
@@ -106,6 +106,7 @@ import static org.hamcrest.Matchers.equalTo;
* Simple unit-test IndexShard related operations.
*/
public class IndexShardTests extends ESSingleNodeTestCase {
+
public void testFlushOnDeleteSetting() throws Exception {
boolean initValue = randomBoolean();
createIndex("test", settingsBuilder().put(IndexShard.INDEX_FLUSH_ON_CLOSE, initValue).build());
@@ -159,7 +160,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
logger.info("--> paths: [{}]", (Object)shardPaths);
// Should not be able to acquire the lock because it's already open
try {
- NodeEnvironment.acquireFSLockForPaths(Settings.EMPTY, shardPaths);
+ NodeEnvironment.acquireFSLockForPaths(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), shardPaths);
fail("should not have been able to acquire the lock");
} catch (LockObtainFailedException e) {
assertTrue("msg: " + e.getMessage(), e.getMessage().contains("unable to acquire write.lock"));
@@ -169,7 +170,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
// we're green to delete the shard's directory)
ShardLock sLock = new DummyShardLock(new ShardId("test", 0));
try {
- env.deleteShardDirectoryUnderLock(sLock, Settings.builder().build());
+ env.deleteShardDirectoryUnderLock(sLock, IndexSettingsModule.newIndexSettings("test", Settings.EMPTY));
fail("should not have been able to delete the directory");
} catch (LockObtainFailedException e) {
assertTrue("msg: " + e.getMessage(), e.getMessage().contains("unable to acquire write.lock"));
@@ -190,39 +191,39 @@ public class IndexShardTests extends ESSingleNodeTestCase {
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
- assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
- assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
- assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
// test if we still write it even if the shard is not active
ShardRouting inactiveRouting = TestShardRouting.newShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), shard.shardRouting.currentNodeId(), null, null, true, ShardRoutingState.INITIALIZING, shard.shardRouting.version() + 1);
shard.persistMetadata(inactiveRouting, shard.shardRouting);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, getShardStateMetadata(shard));
- assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
shard.updateRoutingEntry(new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1), false);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertFalse("shard state persisted despite of persist=false", shardStateMetaData.equals(getShardStateMetadata(shard)));
- assertEquals("shard state persisted despite of persist=false", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals("shard state persisted despite of persist=false", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
- assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID)));
+ assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings().getUUID()));
}
public void testDeleteShardState() throws IOException {
@@ -275,7 +276,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
if (shardRouting == null) {
return null;
} else {
- return new ShardStateMetaData(shardRouting.version(), shardRouting.primary(), shard.indexSettings.get(IndexMetaData.SETTING_INDEX_UUID));
+ return new ShardStateMetaData(shardRouting.version(), shardRouting.primary(), shard.indexSettings().getUUID());
}
}
diff --git a/core/src/test/java/org/elasticsearch/index/shard/NewPathForShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/NewPathForShardTests.java
index 2013aa058e..449fd1df96 100644
--- a/core/src/test/java/org/elasticsearch/index/shard/NewPathForShardTests.java
+++ b/core/src/test/java/org/elasticsearch/index/shard/NewPathForShardTests.java
@@ -20,13 +20,12 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.mockfile.FilterFileSystemProvider;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.io.PathUtilsForTesting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
-import org.elasticsearch.env.NodeEnvironment.NodePath;
import org.elasticsearch.env.NodeEnvironment;
+import org.elasticsearch.env.NodeEnvironment.NodePath;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTestCase;
@@ -41,16 +40,12 @@ import java.nio.file.Path;
import java.nio.file.attribute.FileAttributeView;
import java.nio.file.attribute.FileStoreAttributeView;
import java.nio.file.spi.FileSystemProvider;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
/** Separate test class from ShardPathTests because we need static (BeforeClass) setup to install mock filesystems... */
public class NewPathForShardTests extends ESTestCase {
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.EMPTY, Collections.emptyList());
+ private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.EMPTY);
// Sneakiness to install mock file stores so we can pretend how much free space we have on each path.data:
private static MockFileStore aFileStore = new MockFileStore("mocka");
diff --git a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java
index 3e97308041..d80eb7f6c5 100644
--- a/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java
+++ b/core/src/test/java/org/elasticsearch/index/shard/ShardPathTests.java
@@ -22,13 +22,11 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
-import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.nio.file.Path;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.containsString;
@@ -46,7 +44,7 @@ public class ShardPathTests extends ESTestCase {
Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path);
- ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings, Collections.EMPTY_LIST));
+ ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings));
assertEquals(path, shardPath.getDataPath());
assertEquals("0xDEADBEEF", shardPath.getIndexUUID());
assertEquals("foo", shardPath.getShardId().getIndex());
@@ -65,7 +63,7 @@ public class ShardPathTests extends ESTestCase {
assumeTrue("This test tests multi data.path but we only got one", paths.length > 1);
int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, paths);
- ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings, Collections.EMPTY_LIST));
+ ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings));
fail("Expected IllegalStateException");
} catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("more than one shard state found"));
@@ -82,7 +80,7 @@ public class ShardPathTests extends ESTestCase {
Path path = randomFrom(paths);
int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, path);
- ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings, Collections.EMPTY_LIST));
+ ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), settings));
fail("Expected IllegalStateException");
} catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("expected: foobar on shard path"));
@@ -136,7 +134,7 @@ public class ShardPathTests extends ESTestCase {
Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path);
- ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), indexSetttings, Collections.EMPTY_LIST));
+ ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, IndexSettingsModule.newIndexSettings(shardId.index(), indexSetttings));
boolean found = false;
for (Path p : env.nodeDataPaths()) {
if (p.equals(shardPath.getRootStatePath())) {
diff --git a/core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java b/core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java
index e403715b8e..ee3ad6b8b2 100644
--- a/core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java
+++ b/core/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java
@@ -34,7 +34,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.nio.file.Path;
-import java.util.Collections;
import java.util.Locale;
/**
@@ -47,7 +46,7 @@ public class IndexStoreTests extends ESTestCase {
final IndexModule.Type type = RandomPicks.randomFrom(random(), values);
Settings settings = Settings.settingsBuilder().put(IndexModule.STORE_TYPE, type.name().toLowerCase(Locale.ROOT))
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
- IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(new Index("foo"), settings, Collections.EMPTY_LIST);
+ IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(new Index("foo"), settings);
FsDirectoryService service = new FsDirectoryService(indexSettings, null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
switch (type) {
@@ -80,7 +79,7 @@ public class IndexStoreTests extends ESTestCase {
public void testStoreDirectoryDefault() throws IOException {
final Path tempDir = createTempDir().resolve("foo").resolve("0");
- FsDirectoryService service = new FsDirectoryService(IndexSettingsModule.newIndexSettings(new Index("foo"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(), Collections.EMPTY_LIST), null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
+ FsDirectoryService service = new FsDirectoryService(IndexSettingsModule.newIndexSettings(new Index("foo"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build()), null, new ShardPath(false, tempDir, tempDir, "foo", new ShardId("foo", 0)));
try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
if (Constants.WINDOWS) {
assertTrue(directory.toString(), directory instanceof MMapDirectory || directory instanceof SimpleFSDirectory);
diff --git a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java
index 86a94056c7..1e1e948766 100644
--- a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java
+++ b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java
@@ -24,35 +24,9 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
import org.apache.lucene.codecs.lucene54.Lucene54Codec;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedDocValuesField;
-import org.apache.lucene.document.StringField;
-import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexFormatTooNewException;
-import org.apache.lucene.index.IndexFormatTooOldException;
-import org.apache.lucene.index.IndexNotFoundException;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy;
-import org.apache.lucene.index.NoDeletionPolicy;
-import org.apache.lucene.index.NoMergePolicy;
-import org.apache.lucene.index.SegmentInfo;
-import org.apache.lucene.index.SegmentInfos;
-import org.apache.lucene.index.SnapshotDeletionPolicy;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.BaseDirectoryWrapper;
-import org.apache.lucene.store.ChecksumIndexInput;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.MockDirectoryWrapper;
-import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.document.*;
+import org.apache.lucene.index.*;
+import org.apache.lucene.store.*;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.TestUtil;
@@ -90,18 +64,11 @@ import java.util.zip.Adler32;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.test.VersionUtils.randomVersion;
-import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.endsWith;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.is;
-import static org.hamcrest.Matchers.not;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.hamcrest.Matchers.nullValue;
+import static org.hamcrest.Matchers.*;
public class StoreTests extends ESTestCase {
-
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT).build(), Collections.emptyList());
+
+ private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT).build());
public void testRefCount() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
@@ -1147,7 +1114,7 @@ public class StoreTests extends ESTestCase {
Settings settings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueMinutes(0)).build();
- Store store = new Store(shardId, IndexSettingsModule.newIndexSettings(new Index("index"), settings, Collections.EMPTY_LIST), directoryService, new DummyShardLock(shardId));
+ Store store = new Store(shardId, IndexSettingsModule.newIndexSettings(new Index("index"), settings), directoryService, new DummyShardLock(shardId));
long initialStoreSize = 0;
for (String extraFiles : store.directory().listAll()) {
assertTrue("expected extraFS file but got: " + extraFiles, extraFiles.startsWith("extra"));
diff --git a/core/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java
index 8a0c3a8d6f..aab980e975 100644
--- a/core/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java
+++ b/core/src/test/java/org/elasticsearch/index/translog/BufferedTranslogTests.java
@@ -23,12 +23,10 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.nio.file.Path;
-import java.util.Collections;
/**
*
@@ -42,7 +40,7 @@ public class BufferedTranslogTests extends TranslogTests {
.put("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024), ByteSizeUnit.BYTES)
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build();
- TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build, Collections.EMPTY_LIST), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
+ TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
return new Translog(translogConfig);
}
}
diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java
index 10d79bbbdd..e1369d5fb0 100644
--- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java
+++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java
@@ -20,7 +20,6 @@
package org.elasticsearch.index.translog;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
-
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.Term;
import org.apache.lucene.store.AlreadyClosedException;
@@ -38,7 +37,6 @@ import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.Index;
-import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
@@ -56,28 +54,14 @@ import java.nio.file.Files;
import java.nio.file.InvalidPathException;
import java.nio.file.Path;
import java.nio.file.StandardOpenOption;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ArrayBlockingQueue;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.BrokenBarrierException;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.TimeUnit;
+import java.util.*;
+import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
-import static org.hamcrest.Matchers.containsString;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.*;
/**
*
@@ -131,7 +115,7 @@ public class TranslogTests extends ESTestCase {
.put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name())
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build();
- TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build, Collections.EMPTY_LIST), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
+ TranslogConfig translogConfig = new TranslogConfig(shardId, path, IndexSettingsModule.newIndexSettings(shardId.index(), build), Translog.Durabilty.REQUEST, BigArrays.NON_RECYCLING_INSTANCE, null);
return new Translog(translogConfig);
}
diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
index e1811b01a1..3ae0ba8f69 100644
--- a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
+++ b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java
@@ -29,9 +29,11 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.GatewayMetaState;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.test.ESSingleNodeTestCase;
+import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.util.concurrent.TimeUnit;
@@ -44,6 +46,7 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
public IndicesService getIndicesService() {
return getInstanceFromNode(IndicesService.class);
}
+
public NodeEnvironment getNodeEnvironment() {
return getInstanceFromNode(NodeEnvironment.class);
}
@@ -56,12 +59,12 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
public void testCanDeleteIndexContent() {
IndicesService indicesService = getIndicesService();
- Settings idxSettings = settings(Version.CURRENT)
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
.put(IndexMetaData.SETTING_SHADOW_REPLICAS, true)
.put(IndexMetaData.SETTING_DATA_PATH, "/foo/bar")
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 4))
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 3))
- .build();
+ .build());
assertFalse("shard on shared filesystem", indicesService.canDeleteIndexContents(new Index("test"), idxSettings, false));
assertTrue("shard on shared filesystem and closed", indicesService.canDeleteIndexContents(new Index("test"), idxSettings, true));
}
@@ -142,7 +145,7 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
ShardPath shardPath = ShardPath.loadShardPath(logger, getNodeEnvironment(), new ShardId(test.index(), 0), test.getIndexSettings());
assertEquals(shardPath, path);
try {
- indicesService.processPendingDeletes(test.index(), test.getIndexSettings().getSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
+ indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
fail("can't get lock");
} catch (LockObtainFailedException ex) {
@@ -151,13 +154,13 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
int numPending = 1;
if (randomBoolean()) {
- indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings().getSettings());
+ indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
} else {
if (randomBoolean()) {
numPending++;
- indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings().getSettings());
+ indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
}
- indicesService.addPendingDelete(test.index(), test.getIndexSettings().getSettings());
+ indicesService.addPendingDelete(test.index(), test.getIndexSettings());
}
assertAcked(client().admin().indices().prepareClose("test"));
assertTrue(path.exists());
@@ -165,17 +168,17 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
assertEquals(indicesService.numPendingDeletes(test.index()), numPending);
// shard lock released... we can now delete
- indicesService.processPendingDeletes(test.index(), test.getIndexSettings().getSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
+ indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
assertEquals(indicesService.numPendingDeletes(test.index()), 0);
assertFalse(path.exists());
if (randomBoolean()) {
- indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings().getSettings());
- indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings().getSettings());
- indicesService.addPendingDelete(new ShardId("bogus", 1), test.getIndexSettings().getSettings());
+ indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
+ indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings());
+ indicesService.addPendingDelete(new ShardId("bogus", 1), test.getIndexSettings());
assertEquals(indicesService.numPendingDeletes(test.index()), 2);
// shard lock released... we can now delete
- indicesService.processPendingDeletes(test.index(), test.getIndexSettings().getSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
+ indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
assertEquals(indicesService.numPendingDeletes(test.index()), 0);
}
assertAcked(client().admin().indices().prepareOpen("test"));
diff --git a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java
index 27fa30d6a2..8346003287 100644
--- a/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java
+++ b/core/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java
@@ -22,8 +22,13 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
-import org.apache.lucene.index.*;
-import org.apache.lucene.store.*;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version;
@@ -40,9 +45,9 @@ import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.node.settings.NodeSettingsService;
+import org.elasticsearch.test.CorruptionUtils;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESTestCase;
-import org.elasticsearch.test.CorruptionUtils;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
@@ -52,10 +57,8 @@ import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
-import static org.hamcrest.Matchers.is;
-
public class RecoverySourceHandlerTests extends ESTestCase {
- private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT).build(), Collections.emptyList());
+ private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("index"), Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT).build());
private final ShardId shardId = new ShardId(INDEX_SETTINGS.getIndex(), 1);
private final NodeSettingsService service = new NodeSettingsService(Settings.EMPTY);
diff --git a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisTestUtils.java b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisTestUtils.java
index 10c9ba0901..e8bfb8840c 100644
--- a/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisTestUtils.java
+++ b/plugins/analysis-icu/src/test/java/org/elasticsearch/index/analysis/AnalysisTestUtils.java
@@ -33,7 +33,6 @@ import org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@@ -49,7 +48,7 @@ public class AnalysisTestUtils {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.emptyList()));
+ final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, indexSettings));
return analysisService;
}
}
diff --git a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java
index f866331c01..aa620d6612 100644
--- a/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java
+++ b/plugins/analysis-kuromoji/src/test/java/org/elasticsearch/index/analysis/KuromojiAnalysisTests.java
@@ -44,12 +44,8 @@ import java.io.Reader;
import java.io.StringReader;
import java.nio.file.Files;
import java.nio.file.Path;
-import java.util.Collections;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.greaterThan;
-import static org.hamcrest.Matchers.instanceOf;
-import static org.hamcrest.Matchers.notNullValue;
+import static org.hamcrest.Matchers.*;
/**
*/
@@ -211,7 +207,7 @@ public class KuromojiAnalysisTests extends ESTestCase {
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList()));
+ return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
}
public static void assertSimpleTSOutput(TokenStream stream,
diff --git a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java
index ce6e993f83..8699b8c85c 100644
--- a/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java
+++ b/plugins/analysis-phonetic/src/test/java/org/elasticsearch/index/analysis/SimplePhoneticAnalysisTests.java
@@ -35,7 +35,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert;
import java.io.IOException;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.instanceOf;
@@ -61,6 +60,6 @@ public class SimplePhoneticAnalysisTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList()));
+ return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
}
}
diff --git a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java
index a899f4aeb7..f5d231968c 100644
--- a/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java
+++ b/plugins/analysis-smartcn/src/test/java/org/elasticsearch/index/analysis/SimpleSmartChineseAnalysisTests.java
@@ -35,7 +35,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert;
import java.io.IOException;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.instanceOf;
@@ -54,7 +53,7 @@ public class SimpleSmartChineseAnalysisTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList()));
+ final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
TokenizerFactory tokenizerFactory = analysisService.tokenizer("smartcn_tokenizer");
MatcherAssert.assertThat(tokenizerFactory, instanceOf(SmartChineseTokenizerTokenizerFactory.class));
}
diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java
index 91990ef4a5..52dcb3a8d5 100644
--- a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java
+++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/PolishAnalysisTests.java
@@ -38,10 +38,8 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert;
import java.io.IOException;
-import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
-import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.hamcrest.Matchers.instanceOf;
/**
@@ -61,7 +59,7 @@ public class PolishAnalysisTests extends ESTestCase {
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList()));
+ final AnalysisService analysisService = parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
TokenFilterFactory tokenizerFactory = analysisService.tokenFilter("polish_stem");
MatcherAssert.assertThat(tokenizerFactory, instanceOf(PolishStemTokenFilterFactory.class));
diff --git a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java
index a1f178166f..d96da6d3c4 100644
--- a/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java
+++ b/plugins/analysis-stempel/src/test/java/org/elasticsearch/index/analysis/SimplePolishTokenFilterTests.java
@@ -40,7 +40,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException;
import java.io.StringReader;
-import java.util.Collections;
import static org.hamcrest.Matchers.equalTo;
@@ -103,6 +102,6 @@ public class SimplePolishTokenFilterTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector();
- return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList()));
+ return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
}
}
diff --git a/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperTestUtils.java b/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperTestUtils.java
index deb29f12a4..7cdec8842c 100644
--- a/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperTestUtils.java
+++ b/plugins/mapper-attachments/src/test/java/org/elasticsearch/mapper/attachments/MapperTestUtils.java
@@ -45,7 +45,7 @@ class MapperTestUtils {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(indexSettings)
.build();
- IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("test"), indexSettings, Collections.emptyList());
+ IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("test"), indexSettings);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(idxSettings);
SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap());
return new MapperService(idxSettings, analysisService, similarityService);
diff --git a/test-framework/src/main/java/org/elasticsearch/test/IndexSettingsModule.java b/test-framework/src/main/java/org/elasticsearch/test/IndexSettingsModule.java
index ae538add68..c040aae129 100644
--- a/test-framework/src/main/java/org/elasticsearch/test/IndexSettingsModule.java
+++ b/test-framework/src/main/java/org/elasticsearch/test/IndexSettingsModule.java
@@ -19,33 +19,13 @@
package org.elasticsearch.test;
import org.elasticsearch.Version;
-import org.elasticsearch.cache.recycler.PageCacheRecycler;
-import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.util.BigArrays;
-import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
-import org.elasticsearch.index.NodeServicesProvider;
-import org.elasticsearch.indices.IndicesWarmer;
-import org.elasticsearch.indices.breaker.CircuitBreakerService;
-import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
-import org.elasticsearch.indices.cache.query.IndicesQueryCache;
-import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
-import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCacheListener;
-import org.elasticsearch.indices.memory.IndexingMemoryController;
-import org.elasticsearch.script.ScriptContextRegistry;
-import org.elasticsearch.script.ScriptEngineService;
-import org.elasticsearch.script.ScriptService;
-import org.elasticsearch.script.mustache.MustacheScriptEngineService;
-import org.elasticsearch.threadpool.ThreadPool;
-import org.elasticsearch.watcher.ResourceWatcherService;
-import java.io.IOException;
-import java.util.*;
-import java.util.function.Consumer;
+import java.util.Collections;
public class IndexSettingsModule extends AbstractModule {
@@ -57,12 +37,17 @@ public class IndexSettingsModule extends AbstractModule {
this.index = index;
}
+
@Override
protected void configure() {
- bind(IndexSettings.class).toInstance(newIndexSettings(index, settings, Collections.EMPTY_LIST));
+ bind(IndexSettings.class).toInstance(newIndexSettings(index, settings));
+ }
+
+ public static IndexSettings newIndexSettings(String index, Settings settings) {
+ return newIndexSettings(new Index(index), settings);
}
- public static IndexSettings newIndexSettings(Index index, Settings settings, Collection<Consumer<Settings>> updateListeners) {
+ public static IndexSettings newIndexSettings(Index index, Settings settings) {
Settings build = Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
diff --git a/test-framework/src/main/java/org/elasticsearch/test/engine/MockEngineSupport.java b/test-framework/src/main/java/org/elasticsearch/test/engine/MockEngineSupport.java
index ab570afdd9..70dfa6847b 100644
--- a/test-framework/src/main/java/org/elasticsearch/test/engine/MockEngineSupport.java
+++ b/test-framework/src/main/java/org/elasticsearch/test/engine/MockEngineSupport.java
@@ -21,11 +21,7 @@ package org.elasticsearch.test.engine;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.AssertingIndexSearcher;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.QueryCache;
-import org.apache.lucene.search.QueryCachingPolicy;
-import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.*;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.logging.ESLogger;
@@ -79,18 +75,18 @@ public final class MockEngineSupport {
}
public MockEngineSupport(EngineConfig config, Class<? extends FilterDirectoryReader> wrapper) {
- Settings indexSettings = config.getIndexSettings();
+ Settings settings = config.getIndexSettings().getSettings();
shardId = config.getShardId();
filterCache = config.getQueryCache();
filterCachingPolicy = config.getQueryCachingPolicy();
- final long seed = indexSettings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
+ final long seed = settings.getAsLong(ESIntegTestCase.SETTING_INDEX_SEED, 0l);
Random random = new Random(seed);
- final double ratio = indexSettings.getAsDouble(WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow
+ final double ratio = settings.getAsDouble(WRAP_READER_RATIO, 0.0d); // DISABLED by default - AssertingDR is crazy slow
boolean wrapReader = random.nextDouble() < ratio;
if (logger.isTraceEnabled()) {
logger.trace("Using [{}] for shard [{}] seed: [{}] wrapReader: [{}]", this.getClass().getName(), shardId, seed, wrapReader);
}
- mockContext = new MockContext(random, wrapReader, wrapper, indexSettings);
+ mockContext = new MockContext(random, wrapReader, wrapper, settings);
this.searcherCloseable = new SearcherCloseable();
LuceneTestCase.closeAfterSuite(searcherCloseable); // only one suite closeable per Engine
}
diff --git a/test-framework/src/main/java/org/elasticsearch/test/engine/MockInternalEngine.java b/test-framework/src/main/java/org/elasticsearch/test/engine/MockInternalEngine.java
index 616d873786..15bb291868 100644
--- a/test-framework/src/main/java/org/elasticsearch/test/engine/MockInternalEngine.java
+++ b/test-framework/src/main/java/org/elasticsearch/test/engine/MockInternalEngine.java
@@ -21,7 +21,6 @@ package org.elasticsearch.test.engine;
import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherManager;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.InternalEngine;
@@ -35,7 +34,7 @@ final class MockInternalEngine extends InternalEngine {
MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery, Class<? extends FilterDirectoryReader> wrapper) throws EngineException {
super(config, skipInitialTranslogRecovery);
- randomizeFlushOnClose = IndexMetaData.isOnSharedFilesystem(config.getIndexSettings()) == false;
+ randomizeFlushOnClose = config.getIndexSettings().isOnSharedFilesystem() == false;
wrapperClass = wrapper;
}
diff --git a/test-framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java b/test-framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java
index 30fed6782b..86cf0ddb56 100644
--- a/test-framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java
+++ b/test-framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java
@@ -19,20 +19,22 @@
package org.elasticsearch.test.store;
-import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexModule;
+import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.plugins.Plugin;
-import java.util.*;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.IdentityHashMap;
+import java.util.Map;
public class MockFSIndexStore extends IndexStore {
@@ -84,7 +86,7 @@ public class MockFSIndexStore extends IndexStore {
if (indexShard != null) {
Boolean remove = shardSet.remove(indexShard);
if (remove == Boolean.TRUE) {
- ESLogger logger = Loggers.getLogger(getClass(), indexShard.indexSettings(), indexShard.shardId());
+ ESLogger logger = Loggers.getLogger(getClass(), indexShard.indexSettings().getSettings(), indexShard.shardId());
MockFSDirectoryService.checkIndex(logger, indexShard.store(), indexShard.shardId());
}
}
@@ -92,7 +94,7 @@ public class MockFSIndexStore extends IndexStore {
@Override
public void indexShardStateChanged(IndexShard indexShard, @Nullable IndexShardState previousState, IndexShardState currentState, @Nullable String reason) {
- if (currentState == IndexShardState.CLOSED && validCheckIndexStates.contains(previousState) && IndexMetaData.isOnSharedFilesystem(indexShard.indexSettings()) == false) {
+ if (currentState == IndexShardState.CLOSED && validCheckIndexStates.contains(previousState) && indexShard.indexSettings().isOnSharedFilesystem() == false) {
shardSet.put(indexShard, Boolean.TRUE);
}