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 46f9b7e26ae..0a5ae5ca548 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 288a21a76fb..3a1b430f98b 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 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 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 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) { @@ -780,14 +756,6 @@ public class NodeEnvironment extends AbstractComponent implements Closeable { return settings; } - /** - * @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 @@ -795,9 +763,8 @@ public class NodeEnvironment extends AbstractComponent implements Closeable { * * @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 459d3df876d..4687303e468 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 7d17fa43a36..2b99f14c8c2 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 true 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 o.w.. + */ + public String customDataPath() { + return settings.get(IndexMetaData.SETTING_DATA_PATH); + } + + /** + * Returns true 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 true iff the given settings indicate that the index associated + * with these settings uses shadow replicas. Otherwise false. The default + * setting for this is false. + */ + public 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 81466128c56..fa5b89505c8 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 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 3f303541588..0504fdfa996 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 f4337de266e..c5da8e83b3d 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 9ab09a245ea..1404b61b8ec 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 46677d3a55b..af3e0ae82a8 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 601b94bbcb7..5cf180c3a2e 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 c7e4dbcaaee..eaf562e2127 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 d39f4d19e50..c8719a610e2 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 9c7cf499e50..2370ace0464 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 38886032a7c..a1665a7d5c1 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 f061a95f2af..c329722a135 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 (ConcurrentMergeScheduler) 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 327a080cd6a..d940d1a93cd 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 5c673b2aebc..60752dd774b 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 033fb6d18b1..729e2b65b2a 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 0a145469699..69832ef370e 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 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 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 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 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 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 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 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 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 i private static final class PendingDelete implements Comparable { 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 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 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 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 74f11b4b929..d1532eb540f 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 locks = env.lockAllForIndex(new Index("foo"), settings, randomIntBetween(0, 10)); + List 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 f9ea8579329..ee059924c60 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 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 3e50a71f206..ef13a891a29 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 89ad9594469..cd5138b4e0c 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 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 b6ba4b4dba0..7d4164939b0 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 34f5939559a..dd08d470136 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 89276a8a242..0eb916826f4 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 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 c467f642d73..b7bdbb23279 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 6b11685817c..4b7119df01b 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 9aea985809b..90e55e98d7e 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 67b90590fb9..3a6adca1c67 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 de763056fa1..56bf966dd41 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 b54c7806d62..2e2aa8e96bd 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 a8acdfa13f8..2937461cb8c 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 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 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 4ab3b81b599..3fe7a540bf8 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 24422870794..b2f818c89f0 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 90018e04af3..4631036f13d 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> 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 bcd20fd3669..f017355fd33 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 b66e5e5c491..76f20afc696 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 18dc07d4ef0..7422820a811 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 2013aa058ec..449fd1df96b 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 3e973080416..d80eb7f6c5d 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 e403715b8e5..ee3ad6b8b29 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 86a94056c77..1e1e9487668 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 8a0c3a8d6f9..aab980e975d 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 10d79bbbddd..e1369d5fb0d 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 e1811b01a17..3ae0ba8f69c 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 27fa30d6a24..8346003287c 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 10c9ba09017..e8bfb8840c5 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 f866331c011..aa620d6612c 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 ce6e993f831..8699b8c85cc 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 a899f4aeb7c..f5d231968ce 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 91990ef4a56..52dcb3a8d59 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 a1f178166fd..d96da6d3c42 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 deb29f12a42..7cdec8842cf 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 ae538add68b..c040aae1292 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(Index index, Settings settings, Collection> updateListeners) { + public static IndexSettings newIndexSettings(String index, Settings settings) { + return newIndexSettings(new Index(index), settings); + } + + 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 ab570afdd9d..70dfa6847b4 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 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 616d873786e..15bb2918680 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 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 30fed6782b3..86cf0ddb563 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); }