Extend usage of IndexSetting class

I decided to leave external listeners (used by plugins) alone, for now.

Closes #14731
This commit is contained in:
Boaz Leskes 2015-11-12 18:13:22 +01:00
parent 9673ddc5d0
commit ac0da91bf7
62 changed files with 346 additions and 505 deletions

View File

@ -30,6 +30,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.*;
@ -70,7 +71,7 @@ public class NodeIndexDeletedAction extends AbstractComponent {
listeners.remove(listener); 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(); final DiscoveryNodes nodes = clusterState.nodes();
transportService.sendRequest(clusterState.nodes().masterNode(), transportService.sendRequest(clusterState.nodes().masterNode(),
INDEX_DELETED_ACTION_NAME, new NodeIndexDeletedMessage(index, nodeId), EmptyTransportResponseHandler.INSTANCE_SAME); 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 { 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 // 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 // master. If we can't acquire the locks here immediately there might be a shard of this index still holding on to the lock

View File

@ -21,12 +21,7 @@ package org.elasticsearch.env;
import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.*;
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.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData; 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.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.FsDirectoryService; import org.elasticsearch.index.store.FsDirectoryService;
import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.fs.FsInfo;
@ -45,21 +41,8 @@ import org.elasticsearch.monitor.fs.FsProbe;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.nio.file.AtomicMoveNotSupportedException; import java.nio.file.*;
import java.nio.file.DirectoryStream; import java.util.*;
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.util.concurrent.Semaphore; import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; 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 * @param shardId the id of the shard to delete to delete
* @throws IOException if an IOException occurs * @throws IOException if an IOException occurs
*/ */
public void deleteShardDirectorySafe(ShardId shardId, Settings indexSettings) throws IOException { public void deleteShardDirectorySafe(ShardId shardId, IndexSettings indexSettings) throws IOException {
// This is to ensure someone doesn't use Settings.EMPTY
assert indexSettings != Settings.EMPTY;
final Path[] paths = availableShardPaths(shardId); final Path[] paths = availableShardPaths(shardId);
logger.trace("deleting shard {} directory, paths: [{}]", shardId, paths); logger.trace("deleting shard {} directory, paths: [{}]", shardId, paths);
try (ShardLock lock = shardLock(shardId)) { 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 * @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]; Lock[] locks = new Lock[shardPaths.length];
Directory[] dirs = new Directory[shardPaths.length]; Directory[] dirs = new Directory[shardPaths.length];
try { try {
@ -359,15 +340,14 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @throws IOException if an IOException occurs * @throws IOException if an IOException occurs
* @throws ElasticsearchException if the write.lock is not acquirable * @throws ElasticsearchException if the write.lock is not acquirable
*/ */
public void deleteShardDirectoryUnderLock(ShardLock lock, Settings indexSettings) throws IOException { public void deleteShardDirectoryUnderLock(ShardLock lock, IndexSettings indexSettings) throws IOException {
assert indexSettings != Settings.EMPTY;
final ShardId shardId = lock.getShardId(); final ShardId shardId = lock.getShardId();
assert isShardLocked(shardId) : "shard " + shardId + " is not locked"; assert isShardLocked(shardId) : "shard " + shardId + " is not locked";
final Path[] paths = availableShardPaths(shardId); final Path[] paths = availableShardPaths(shardId);
logger.trace("acquiring locks for {}, paths: [{}]", shardId, paths); logger.trace("acquiring locks for {}, paths: [{}]", shardId, paths);
acquireFSLockForPaths(indexSettings, paths); acquireFSLockForPaths(indexSettings, paths);
IOUtils.rm(paths); IOUtils.rm(paths);
if (hasCustomDataPath(indexSettings)) { if (indexSettings.hasCustomDataPath()) {
Path customLocation = resolveCustomLocation(indexSettings, shardId); Path customLocation = resolveCustomLocation(indexSettings, shardId);
logger.trace("acquiring lock for {}, custom path: [{}]", shardId, customLocation); logger.trace("acquiring lock for {}, custom path: [{}]", shardId, customLocation);
acquireFSLockForPaths(indexSettings, customLocation); acquireFSLockForPaths(indexSettings, customLocation);
@ -397,9 +377,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index being deleted * @param indexSettings settings for the index being deleted
* @throws IOException if any of the shards data directories can't be locked or 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 { public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, IndexSettings indexSettings) throws IOException {
// This is to ensure someone doesn't use Settings.EMPTY
assert indexSettings != Settings.EMPTY;
final List<ShardLock> locks = lockAllForIndex(index, indexSettings, lockTimeoutMS); final List<ShardLock> locks = lockAllForIndex(index, indexSettings, lockTimeoutMS);
try { try {
deleteIndexDirectoryUnderLock(index, indexSettings); deleteIndexDirectoryUnderLock(index, indexSettings);
@ -415,13 +393,11 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param index the index to delete * @param index the index to delete
* @param indexSettings settings for the index being deleted * @param indexSettings settings for the index being deleted
*/ */
public void deleteIndexDirectoryUnderLock(Index index, Settings indexSettings) throws IOException { public void deleteIndexDirectoryUnderLock(Index index, IndexSettings indexSettings) throws IOException {
// This is to ensure someone doesn't use Settings.EMPTY
assert indexSettings != Settings.EMPTY;
final Path[] indexPaths = indexPaths(index); final Path[] indexPaths = indexPaths(index);
logger.trace("deleting index {} directory, paths({}): [{}]", index, indexPaths.length, indexPaths); logger.trace("deleting index {} directory, paths({}): [{}]", index, indexPaths.length, indexPaths);
IOUtils.rm(indexPaths); IOUtils.rm(indexPaths);
if (hasCustomDataPath(indexSettings)) { if (indexSettings.hasCustomDataPath()) {
Path customLocation = resolveCustomLocation(indexSettings, index.name()); Path customLocation = resolveCustomLocation(indexSettings, index.name());
logger.trace("deleting custom index {} directory [{}]", index, customLocation); logger.trace("deleting custom index {} directory [{}]", index, customLocation);
IOUtils.rm(customLocation); IOUtils.rm(customLocation);
@ -438,9 +414,9 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @return the {@link ShardLock} instances for this index. * @return the {@link ShardLock} instances for this index.
* @throws IOException if an IOException occurs. * @throws IOException if an IOException occurs.
*/ */
public List<ShardLock> lockAllForIndex(Index index, Settings settings, long lockTimeoutMS) throws IOException { public List<ShardLock> lockAllForIndex(Index index, IndexSettings settings, long lockTimeoutMS) throws IOException {
final Integer numShards = settings.getAsInt(IndexMetaData.SETTING_NUMBER_OF_SHARDS, null); final int numShards = settings.getNumberOfShards();
if (numShards == null || numShards <= 0) { if (numShards <= 0) {
throw new IllegalArgumentException("settings must contain a non-null > 0 number of shards"); throw new IllegalArgumentException("settings must contain a non-null > 0 number of shards");
} }
logger.trace("locking all shards for index {} - [{}]", index, numShards); 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 * 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. * returned paths. The returned array may contain paths to non-existing directories.
* *
* @see #hasCustomDataPath(org.elasticsearch.common.settings.Settings) * @see IndexSettings#hasCustomDataPath()
* @see #resolveCustomLocation(org.elasticsearch.common.settings.Settings, org.elasticsearch.index.shard.ShardId) * @see #resolveCustomLocation(IndexSettings, ShardId)
* *
*/ */
public Path[] availableShardPaths(ShardId shardId) { public Path[] availableShardPaths(ShardId shardId) {
@ -780,14 +756,6 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
return settings; 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. * Resolve the custom path for a index's shard.
* Uses the {@code IndexMetaData.SETTING_DATA_PATH} setting to determine * 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 * @param indexSettings settings for the index
*/ */
private Path resolveCustomLocation(Settings indexSettings) { private Path resolveCustomLocation(IndexSettings indexSettings) {
assert indexSettings != Settings.EMPTY; String customDataDir = indexSettings.customDataPath();
String customDataDir = indexSettings.get(IndexMetaData.SETTING_DATA_PATH);
if (customDataDir != null) { if (customDataDir != null) {
// This assert is because this should be caught by MetaDataCreateIndexService // This assert is because this should be caught by MetaDataCreateIndexService
assert sharedDataPath != null; assert sharedDataPath != null;
@ -819,7 +786,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index * @param indexSettings settings for the index
* @param indexName index to resolve the path for * @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); return resolveCustomLocation(indexSettings).resolve(indexName);
} }
@ -831,7 +798,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
* @param indexSettings settings for the index * @param indexSettings settings for the index
* @param shardId shard to resolve the path to * @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())); return resolveCustomLocation(indexSettings, shardId.index().name()).resolve(Integer.toString(shardId.id()));
} }

View File

@ -52,12 +52,16 @@ import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store; 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.Closeable;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; 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.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; 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) { private void onShardClose(ShardLock lock, boolean ownsShard) {
if (deleted.get()) { // we remove that shards content if this index has been deleted if (deleted.get()) { // we remove that shards content if this index has been deleted
final Settings indexSettings = this.getIndexSettings().getSettings();
try { try {
if (ownsShard) { if (ownsShard) {
try { try {
eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings); eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings.getSettings());
} finally { } finally {
shardStoreDeleter.deleteShardStore("delete index", lock, indexSettings); shardStoreDeleter.deleteShardStore("delete index", lock, indexSettings);
eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings); eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings.getSettings());
} }
} }
} catch (IOException e) { } catch (IOException e) {
@ -533,8 +536,9 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
} }
public interface ShardStoreDeleter { public interface ShardStoreDeleter {
void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException; void deleteShardStore(String reason, ShardLock lock, IndexSettings indexSettings) throws IOException;
void addPendingDelete(ShardId shardId, Settings indexSettings);
void addPendingDelete(ShardId shardId, IndexSettings indexSettings);
} }
final EngineFactory getEngineFactory() { final EngineFactory getEngineFactory() {

View File

@ -25,7 +25,6 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.mapper.internal.AllFieldMapper; import org.elasticsearch.index.mapper.internal.AllFieldMapper;
import java.util.ArrayList; import java.util.ArrayList;
@ -182,7 +181,32 @@ public final class IndexSettings {
* Returns <code>true</code> if the index has a custom data path * Returns <code>true</code> if the index has a custom data path
*/ */
public boolean hasCustomDataPath() { public boolean hasCustomDataPath() {
return NodeEnvironment.hasCustomDataPath(settings); return customDataPath() != null;
}
/**
* Returns the customDataPath for this index, if configured. null <code>o.w.</code>.
*/
public String customDataPath() {
return settings.get(IndexMetaData.SETTING_DATA_PATH);
}
/**
* Returns <code>true</code> iff the given settings indicate that the index
* associated with these settings allocates it's shards on a shared
* filesystem.
*/
public boolean isOnSharedFilesystem() {
return IndexMetaData.isOnSharedFilesystem(getSettings());
}
/**
* Returns <code>true</code> iff the given settings indicate that the index associated
* with these settings uses shadow replicas. Otherwise <code>false</code>. The default
* setting for this is <code>false</code>.
*/
public boolean isIndexUsingShadowReplicas() {
return IndexMetaData.isOnSharedFilesystem(getSettings());
} }
/** /**

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.merge.OnGoingMerge;
import org.elasticsearch.index.shard.MergeSchedulerConfig; import org.elasticsearch.index.shard.MergeSchedulerConfig;
@ -62,11 +63,11 @@ class ElasticsearchConcurrentMergeScheduler extends ConcurrentMergeScheduler {
private final Set<OnGoingMerge> readOnlyOnGoingMerges = Collections.unmodifiableSet(onGoingMerges); private final Set<OnGoingMerge> readOnlyOnGoingMerges = Collections.unmodifiableSet(onGoingMerges);
private final MergeSchedulerConfig config; private final MergeSchedulerConfig config;
public ElasticsearchConcurrentMergeScheduler(ShardId shardId, Settings indexSettings, MergeSchedulerConfig config) { public ElasticsearchConcurrentMergeScheduler(ShardId shardId, IndexSettings indexSettings, MergeSchedulerConfig config) {
this.config = config; this.config = config;
this.shardId = shardId; this.shardId = shardId;
this.indexSettings = indexSettings; this.indexSettings = indexSettings.getSettings();
this.logger = Loggers.getLogger(getClass(), indexSettings, shardId); this.logger = Loggers.getLogger(getClass(), this.indexSettings, shardId);
refreshConfig(); refreshConfig();
} }

View File

@ -54,7 +54,6 @@ import java.io.IOException;
import java.util.*; import java.util.*;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock; import java.util.concurrent.locks.ReentrantLock;
@ -101,7 +100,7 @@ public abstract class Engine implements Closeable {
this.shardId = engineConfig.getShardId(); this.shardId = engineConfig.getShardId();
this.store = engineConfig.getStore(); 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 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.eventListener = engineConfig.getEventListener();
this.deletionPolicy = engineConfig.getDeletionPolicy(); this.deletionPolicy = engineConfig.getDeletionPolicy();
} }

View File

@ -25,10 +25,10 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.shard.MergeSchedulerConfig; 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.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -50,7 +49,7 @@ import java.util.concurrent.TimeUnit;
public final class EngineConfig { public final class EngineConfig {
private final ShardId shardId; private final ShardId shardId;
private final TranslogRecoveryPerformer translogRecoveryPerformer; private final TranslogRecoveryPerformer translogRecoveryPerformer;
private final Settings indexSettings; private final IndexSettings indexSettings;
private volatile ByteSizeValue indexingBufferSize; private volatile ByteSizeValue indexingBufferSize;
private volatile ByteSizeValue versionMapSize; private volatile ByteSizeValue versionMapSize;
private volatile String versionMapSizeSetting; private volatile String versionMapSizeSetting;
@ -115,11 +114,12 @@ public final class EngineConfig {
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig} * Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
*/ */
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService, 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, MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener, Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) { TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) {
this.shardId = shardId; this.shardId = shardId;
final Settings settings = indexSettings.getSettings();
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.threadPool = threadPool; this.threadPool = threadPool;
this.indexingService = indexingService; this.indexingService = indexingService;
@ -132,15 +132,15 @@ public final class EngineConfig {
this.similarity = similarity; this.similarity = similarity;
this.codecService = codecService; this.codecService = codecService;
this.eventListener = eventListener; this.eventListener = eventListener;
this.compoundOnFlush = indexSettings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush); this.compoundOnFlush = settings.getAsBoolean(EngineConfig.INDEX_COMPOUND_ON_FLUSH, compoundOnFlush);
codecName = indexSettings.get(EngineConfig.INDEX_CODEC_SETTING, EngineConfig.DEFAULT_CODEC_NAME); 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: // We start up inactive and rely on IndexingMemoryController to give us our fair share once we start indexing:
indexingBufferSize = IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER; indexingBufferSize = IndexingMemoryController.INACTIVE_SHARD_INDEXING_BUFFER;
gcDeletesInMillis = indexSettings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis(); gcDeletesInMillis = settings.getAsTime(INDEX_GC_DELETES_SETTING, EngineConfig.DEFAULT_GC_DELETES).millis();
versionMapSizeSetting = indexSettings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE); versionMapSizeSetting = settings.get(INDEX_VERSION_MAP_SIZE, DEFAULT_VERSION_MAP_SIZE);
updateVersionMapSize(); updateVersionMapSize();
this.translogRecoveryPerformer = translogRecoveryPerformer; 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.queryCache = queryCache;
this.queryCachingPolicy = queryCachingPolicy; this.queryCachingPolicy = queryCachingPolicy;
this.translogConfig = translogConfig; 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; return indexSettings;
} }

View File

@ -42,10 +42,10 @@ import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader; import org.elasticsearch.common.lucene.index.ElasticsearchLeafReader;
import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.math.MathUtils; 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.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.Uid; import org.elasticsearch.index.mapper.Uid;
import org.elasticsearch.index.merge.MergeStats; 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.Translog;
import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.index.translog.TranslogCorruptedException; import org.elasticsearch.index.translog.TranslogCorruptedException;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
@ -1051,7 +1050,7 @@ public class InternalEngine extends Engine {
private final AtomicInteger numMergesInFlight = new AtomicInteger(0); private final AtomicInteger numMergesInFlight = new AtomicInteger(0);
private final AtomicBoolean isThrottling = new AtomicBoolean(); private final AtomicBoolean isThrottling = new AtomicBoolean();
EngineMergeScheduler(ShardId shardId, Settings indexSettings, MergeSchedulerConfig config) { EngineMergeScheduler(ShardId shardId, IndexSettings indexSettings, MergeSchedulerConfig config) {
super(shardId, indexSettings, config); super(shardId, indexSettings, config);
} }

View File

@ -68,7 +68,7 @@ public class ShadowEngine extends Engine {
public ShadowEngine(EngineConfig engineConfig) { public ShadowEngine(EngineConfig engineConfig) {
super(engineConfig); super(engineConfig);
SearcherFactory searcherFactory = new EngineSearcherFactory(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) .getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT)
.getMillis(); .getMillis();
try { try {

View File

@ -51,7 +51,7 @@ public class ShardIndexingService extends AbstractIndexShardComponent {
public ShardIndexingService(ShardId shardId, IndexSettings indexSettings) { public ShardIndexingService(ShardId shardId, IndexSettings indexSettings) {
super(shardId, indexSettings); super(shardId, indexSettings);
this.slowLog = new IndexingSlowLog(this.indexSettings); this.slowLog = new IndexingSlowLog(this.indexSettings.getSettings());
} }
/** /**

View File

@ -92,7 +92,7 @@ public final class PercolatorQueriesRegistry extends AbstractIndexShardComponent
this.indexingService = indexingService; this.indexingService = indexingService;
this.queryShardContext = queryShardContext; this.queryShardContext = queryShardContext;
this.indexFieldDataService = indexFieldDataService; 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); mapperService.addTypeListener(percolateTypeListener);
} }

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.shard;
import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
/** /**
@ -33,12 +32,12 @@ public abstract class AbstractIndexShardComponent implements IndexShardComponent
protected final ESLogger logger; protected final ESLogger logger;
protected final DeprecationLogger deprecationLogger; protected final DeprecationLogger deprecationLogger;
protected final ShardId shardId; protected final ShardId shardId;
protected final Settings indexSettings; protected final IndexSettings indexSettings;
protected AbstractIndexShardComponent(ShardId shardId, IndexSettings indexSettings) { protected AbstractIndexShardComponent(ShardId shardId, IndexSettings indexSettings) {
this.shardId = shardId; this.shardId = shardId;
this.indexSettings = indexSettings.getSettings(); this.indexSettings = indexSettings;
this.logger = Loggers.getLogger(getClass(), this.indexSettings, shardId); this.logger = Loggers.getLogger(getClass(), this.indexSettings.getSettings(), shardId);
this.deprecationLogger = new DeprecationLogger(logger); this.deprecationLogger = new DeprecationLogger(logger);
} }
@ -48,12 +47,12 @@ public abstract class AbstractIndexShardComponent implements IndexShardComponent
} }
@Override @Override
public Settings indexSettings() { public IndexSettings indexSettings() {
return this.indexSettings; return indexSettings;
} }
public String nodeName() { public String nodeName() {
return indexSettings.get("name", ""); return indexSettings.getNodeName();
} }

View File

@ -30,13 +30,11 @@ import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.ThreadInterruptedException; import org.apache.lucene.util.ThreadInterruptedException;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.elasticsearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest; import org.elasticsearch.action.admin.indices.upgrade.post.UpgradeRequest;
import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsRequest;
import org.elasticsearch.action.termvectors.TermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState; 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.RecoveryFailedException;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.percolator.PercolatorService; 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.CompletionFieldStats;
import org.elasticsearch.search.suggest.completion.CompletionStats;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException; import java.io.IOException;
@ -205,7 +203,8 @@ public class IndexShard extends AbstractIndexShardComponent {
@Nullable EngineFactory engineFactory, @Nullable EngineFactory engineFactory,
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider) { IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider) {
super(shardId, indexSettings); 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.idxSettings = indexSettings;
this.codecService = new CodecService(mapperService, logger); this.codecService = new CodecService(mapperService, logger);
this.warmer = provider.getWarmer(); this.warmer = provider.getWarmer();
@ -215,14 +214,14 @@ public class IndexShard extends AbstractIndexShardComponent {
this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory; this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
this.store = store; this.store = store;
this.indexEventListener = indexEventListener; this.indexEventListener = indexEventListener;
this.mergeSchedulerConfig = new MergeSchedulerConfig(this.indexSettings); this.mergeSchedulerConfig = new MergeSchedulerConfig(indexSettings);
this.threadPool = provider.getThreadPool(); this.threadPool = provider.getThreadPool();
this.mapperService = mapperService; this.mapperService = mapperService;
this.indexCache = indexCache; this.indexCache = indexCache;
this.indexingService = new ShardIndexingService(shardId, indexSettings); this.indexingService = new ShardIndexingService(shardId, indexSettings);
this.getService = new ShardGetService(indexSettings, this, mapperService); this.getService = new ShardGetService(indexSettings, this, mapperService);
this.termVectorsService = provider.getTermVectorsService(); this.termVectorsService = provider.getTermVectorsService();
this.searchService = new ShardSearchStats(this.indexSettings); this.searchService = new ShardSearchStats(settings);
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings); this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
this.indicesQueryCache = provider.getIndicesQueryCache(); this.indicesQueryCache = provider.getIndicesQueryCache();
this.shardQueryCache = new ShardRequestCache(shardId, indexSettings); this.shardQueryCache = new ShardRequestCache(shardId, indexSettings);
@ -230,29 +229,29 @@ public class IndexShard extends AbstractIndexShardComponent {
this.indexFieldDataService = indexFieldDataService; this.indexFieldDataService = indexFieldDataService;
this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings); this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings);
state = IndexShardState.CREATED; state = IndexShardState.CREATED;
this.refreshInterval = this.indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL); this.refreshInterval = settings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL);
this.flushOnClose = this.indexSettings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true); this.flushOnClose = settings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true);
this.path = path; this.path = path;
this.mergePolicyConfig = new MergePolicyConfig(logger, this.indexSettings); this.mergePolicyConfig = new MergePolicyConfig(logger, settings);
/* create engine config */ /* create engine config */
logger.debug("state: [CREATED]"); logger.debug("state: [CREATED]");
this.checkIndexOnStartup = this.indexSettings.get("index.shard.check_on_startup", "false"); this.checkIndexOnStartup = settings.get("index.shard.check_on_startup", "false");
this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, this.indexSettings, Translog.Durabilty.REQUEST), this.translogConfig = new TranslogConfig(shardId, shardPath().resolveTranslog(), indexSettings, getFromSettings(logger, settings, Translog.Durabilty.REQUEST),
provider.getBigArrays(), threadPool); provider.getBigArrays(), threadPool);
final QueryCachingPolicy cachingPolicy; final QueryCachingPolicy cachingPolicy;
// the query cache is a node-level thing, however we want the most popular filters // the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis // 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; cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
} else { } else {
cachingPolicy = new UsageTrackingQueryCachingPolicy(); cachingPolicy = new UsageTrackingQueryCachingPolicy();
} }
this.engineConfig = newEngineConfig(translogConfig, cachingPolicy); 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.flushThresholdOperations = settings.getAsInt(INDEX_TRANSLOG_FLUSH_THRESHOLD_OPS, settings.getAsInt("index.translog.flush_threshold", Integer.MAX_VALUE));
this.flushThresholdSize = this.indexSettings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB)); this.flushThresholdSize = settings.getAsBytesSize(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE, new ByteSizeValue(512, ByteSizeUnit.MB));
this.disableFlush = this.indexSettings.getAsBoolean(INDEX_TRANSLOG_DISABLE_FLUSH, false); this.disableFlush = settings.getAsBoolean(INDEX_TRANSLOG_DISABLE_FLUSH, false);
this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId); this.indexShardOperationCounter = new IndexShardOperationCounter(logger, shardId);
this.provider = provider; this.provider = provider;
this.searcherWrapper = indexSearcherWrapper; this.searcherWrapper = indexSearcherWrapper;
@ -1445,10 +1444,7 @@ public class IndexShard extends AbstractIndexShardComponent {
} }
private String getIndexUUID() { private String getIndexUUID() {
assert indexSettings.get(IndexMetaData.SETTING_INDEX_UUID) != null return indexSettings.getUUID();
|| 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);
} }
private DocumentMapperForType docMapper(String type) { private DocumentMapperForType docMapper(String type) {

View File

@ -19,7 +19,7 @@
package org.elasticsearch.index.shard; 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(); ShardId shardId();
Settings indexSettings(); IndexSettings indexSettings();
} }

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.index.ConcurrentMergeScheduler; import org.apache.lucene.index.ConcurrentMergeScheduler;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.index.IndexSettings;
/** /**
* The merge scheduler (<code>ConcurrentMergeScheduler</code>) controls the execution of * The merge scheduler (<code>ConcurrentMergeScheduler</code>) controls the execution of
@ -61,11 +62,12 @@ public final class MergeSchedulerConfig {
private volatile int maxMergeCount; private volatile int maxMergeCount;
private final boolean notifyOnMergeFailure; private final boolean notifyOnMergeFailure;
public MergeSchedulerConfig(Settings indexSettings) { public MergeSchedulerConfig(IndexSettings indexSettings) {
maxThreadCount = indexSettings.getAsInt(MAX_THREAD_COUNT, Math.max(1, Math.min(4, EsExecutors.boundedNumberOfProcessors(indexSettings) / 2))); final Settings settings = indexSettings.getSettings();
maxMergeCount = indexSettings.getAsInt(MAX_MERGE_COUNT, maxThreadCount + 5); maxThreadCount = settings.getAsInt(MAX_THREAD_COUNT, Math.max(1, Math.min(4, EsExecutors.boundedNumberOfProcessors(settings) / 2)));
this.autoThrottle = indexSettings.getAsBoolean(AUTO_THROTTLE, true); maxMergeCount = settings.getAsInt(MAX_MERGE_COUNT, maxThreadCount + 5);
notifyOnMergeFailure = indexSettings.getAsBoolean(NOTIFY_ON_MERGE_FAILURE, true); this.autoThrottle = settings.getAsBoolean(AUTO_THROTTLE, true);
notifyOnMergeFailure = settings.getAsBoolean(NOTIFY_ON_MERGE_FAILURE, true);
} }
/** /**

View File

@ -139,7 +139,7 @@ public final class ShardPath {
final Path dataPath; final Path dataPath;
final Path statePath = loadedPath; final Path statePath = loadedPath;
if (indexSettings.hasCustomDataPath()) { if (indexSettings.hasCustomDataPath()) {
dataPath = env.resolveCustomLocation(indexSettings.getSettings(), shardId); dataPath = env.resolveCustomLocation(indexSettings, shardId);
} else { } else {
dataPath = statePath; 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) { 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); logger.warn("{} deleting leftover shard on path: [{}] with a different index UUID", lock.getShardId(), path);
assert Files.isDirectory(path) : path + " is not a directory"; assert Files.isDirectory(path) : path + " is not a directory";
NodeEnvironment.acquireFSLockForPaths(indexSettings.getSettings(), paths); NodeEnvironment.acquireFSLockForPaths(indexSettings, paths);
IOUtils.rm(path); IOUtils.rm(path);
} }
} }
@ -203,7 +203,7 @@ public final class ShardPath {
final Path statePath; final Path statePath;
if (indexSettings.hasCustomDataPath()) { if (indexSettings.hasCustomDataPath()) {
dataPath = env.resolveCustomLocation(indexSettings.getSettings(), shardId); dataPath = env.resolveCustomLocation(indexSettings, shardId);
statePath = env.nodePaths()[0].resolve(shardId); statePath = env.nodePaths()[0].resolve(shardId);
} else { } else {

View File

@ -61,8 +61,9 @@ public class FsDirectoryService extends DirectoryService implements StoreRateLim
return indexStore.rateLimiting(); return indexStore.rateLimiting();
} }
public static LockFactory buildLockFactory(Settings indexSettings) { public static LockFactory buildLockFactory(IndexSettings indexSettings) {
String fsLock = indexSettings.get("index.store.fs.lock", indexSettings.get("index.store.fs.fs_lock", "native")); final Settings settings = indexSettings.getSettings();
String fsLock = settings.get("index.store.fs.lock", settings.get("index.store.fs.fs_lock", "native"));
LockFactory lockFactory; LockFactory lockFactory;
if (fsLock.equals("native")) { if (fsLock.equals("native")) {
lockFactory = NativeFSLockFactory.INSTANCE; lockFactory = NativeFSLockFactory.INSTANCE;
@ -101,7 +102,7 @@ public class FsDirectoryService extends DirectoryService implements StoreRateLim
protected Directory newFSDirectory(Path location, LockFactory lockFactory) throws IOException { 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)) { if (IndexModule.Type.FS.match(storeType) || IndexModule.Type.DEFAULT.match(storeType)) {
final FSDirectory open = FSDirectory.open(location, lockFactory); // use lucene defaults final FSDirectory open = FSDirectory.open(location, lockFactory); // use lucene defaults
if (open instanceof MMapDirectory && Constants.WINDOWS == false) { if (open instanceof MMapDirectory && Constants.WINDOWS == false) {

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.store.ByteArrayIndexInput; import org.elasticsearch.common.lucene.store.ByteArrayIndexInput;
import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.lucene.store.InputStreamIndexInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.SingleObjectCache; import org.elasticsearch.common.util.SingleObjectCache;
@ -116,10 +117,11 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
@Inject @Inject
public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException { public Store(ShardId shardId, IndexSettings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException {
super(shardId, indexSettings); 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.shardLock = shardLock;
this.onClose = onClose; 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); this.statsCache = new StoreStatsCache(refreshInterval, directory, directoryService);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval); logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);

View File

@ -34,7 +34,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractLifecycleComponent; 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.io.FileSystemUtils;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; 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.refresh.RefreshStats;
import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.SearchStats;
import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IllegalIndexShardStateException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.PluginsService; 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.emptyMap;
import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder; import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList; import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList;
/** /**
@ -330,8 +328,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
logger.debug("[{}] closed... (reason [{}])", index, reason); logger.debug("[{}] closed... (reason [{}])", index, reason);
listener.afterIndexClosed(indexService.index(), indexService.getIndexSettings().getSettings()); listener.afterIndexClosed(indexService.index(), indexService.getIndexSettings().getSettings());
if (delete) { if (delete) {
final Settings indexSettings = indexService.getIndexSettings().getSettings(); final IndexSettings indexSettings = indexService.getIndexSettings();
listener.afterIndexDeleted(indexService.index(), indexSettings); listener.afterIndexDeleted(indexService.index(), indexSettings.getSettings());
// now we are done - try to wipe data on disk if possible // now we are done - try to wipe data on disk if possible
deleteIndexStore(reason, indexService.index(), indexSettings, false); deleteIndexStore(reason, indexService.index(), indexSettings, false);
} }
@ -412,12 +410,12 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
} }
} }
Index index = new Index(metaData.getIndex()); Index index = new Index(metaData.getIndex());
final Settings indexSettings = buildIndexSettings(metaData); final IndexSettings indexSettings = buildIndexSettings(metaData);
deleteIndexStore(reason, index, indexSettings, closed); 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; boolean success = false;
try { try {
// we are trying to delete the index store here - not a big deal if the lock can't be obtained // we are trying to delete the index store here - not a big deal if the lock can't be obtained
@ -449,7 +447,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @param indexSettings the shards index settings. * @param indexSettings the shards index settings.
* @throws IOException if an IOException occurs * @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(); ShardId shardId = lock.getShardId();
logger.trace("{} deleting shard reason [{}]", shardId, reason); logger.trace("{} deleting shard reason [{}]", shardId, reason);
nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings); nodeEnv.deleteShardDirectoryUnderLock(lock, indexSettings);
@ -470,7 +468,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
public void deleteShardStore(String reason, ShardId shardId, ClusterState clusterState) throws IOException { public void deleteShardStore(String reason, ShardId shardId, ClusterState clusterState) throws IOException {
final IndexMetaData metaData = clusterState.getMetaData().indices().get(shardId.getIndex()); final IndexMetaData metaData = clusterState.getMetaData().indices().get(shardId.getIndex());
final Settings indexSettings = buildIndexSettings(metaData); final IndexSettings indexSettings = buildIndexSettings(metaData);
if (canDeleteShardContent(shardId, indexSettings) == false) { if (canDeleteShardContent(shardId, indexSettings) == false) {
throw new IllegalStateException("Can't delete shard " + shardId); throw new IllegalStateException("Can't delete shard " + shardId);
} }
@ -498,14 +496,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* given index. If the index uses a shared filesystem this method always * given index. If the index uses a shared filesystem this method always
* returns false. * returns false.
* @param index {@code Index} to check whether deletion is allowed * @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 * @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()); final IndexService indexService = this.indices.get(index.name());
// Closed indices may be deleted, even if they are on a shared // Closed indices may be deleted, even if they are on a shared
// filesystem. Since it is closed we aren't deleting it for relocation // 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()) { if (indexService == null && nodeEnv.hasNodeFile()) {
return true; return true;
} }
@ -532,17 +530,17 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
// to decide where the shard content lives. In the future we might even need more info here ie. for shadow replicas // 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 // The plan was to make it harder to miss-use and ask for metadata instead of simple settings
assert shardId.getIndex().equals(metaData.getIndex()); assert shardId.getIndex().equals(metaData.getIndex());
final Settings indexSettings = buildIndexSettings(metaData); final IndexSettings indexSettings = buildIndexSettings(metaData);
return canDeleteShardContent(shardId, indexSettings); 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()); final IndexService indexService = this.indices.get(shardId.getIndex());
if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) { if (indexSettings.isOnSharedFilesystem() == false) {
if (indexService != null && nodeEnv.hasNodeFile()) { if (indexService != null && nodeEnv.hasNodeFile()) {
return indexService.hasShard(shardId.id()) == false; return indexService.hasShard(shardId.id()) == false;
} else if (nodeEnv.hasNodeFile()) { } else if (nodeEnv.hasNodeFile()) {
if (NodeEnvironment.hasCustomDataPath(indexSettings)) { if (indexSettings.hasCustomDataPath()) {
return Files.exists(nodeEnv.resolveCustomLocation(indexSettings, shardId)); return Files.exists(nodeEnv.resolveCustomLocation(indexSettings, shardId));
} else { } else {
return FileSystemUtils.exists(nodeEnv.availableShardPaths(shardId)); return FileSystemUtils.exists(nodeEnv.availableShardPaths(shardId));
@ -554,20 +552,17 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
return false; 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. // 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 // we might run on nodes where we use shard FS and then in the future don't delete
// actual content. // actual content.
Settings.Builder builder = settingsBuilder(); return new IndexSettings(metaData, settings, Collections.EMPTY_LIST);
builder.put(settings);
builder.put(metaData.getSettings());
return builder.build();
} }
/** /**
* Adds a pending delete for the given index shard. * 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) { if (shardId == null) {
throw new IllegalArgumentException("shardId must not be null"); throw new IllegalArgumentException("shardId must not be null");
} }
@ -581,7 +576,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
/** /**
* Adds a pending delete for the given index. * 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); PendingDelete pendingDelete = new PendingDelete(index, settings);
addPendingDelete(index, pendingDelete); addPendingDelete(index, pendingDelete);
} }
@ -600,13 +595,13 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private static final class PendingDelete implements Comparable<PendingDelete> { private static final class PendingDelete implements Comparable<PendingDelete> {
final String index; final String index;
final int shardId; final int shardId;
final Settings settings; final IndexSettings settings;
final boolean deleteIndex; final boolean deleteIndex;
/** /**
* Creates a new pending delete of an index * 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.index = shardId.getIndex();
this.shardId = shardId.getId(); this.shardId = shardId.getId();
this.settings = settings; this.settings = settings;
@ -616,7 +611,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
/** /**
* Creates a new pending delete of a shard * 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.index = index.getName();
this.shardId = -1; this.shardId = -1;
this.settings = settings; this.settings = settings;
@ -648,7 +643,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @param index the index to process the pending deletes for * @param index the index to process the pending deletes for
* @param timeout the timeout used for processing pending deletes * @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); logger.debug("{} processing pending deletes", index);
final long startTimeNS = System.nanoTime(); final long startTimeNS = System.nanoTime();
final List<ShardLock> shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis()); final List<ShardLock> shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis());

View File

@ -45,6 +45,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexShardAlreadyExistsException; import org.elasticsearch.index.IndexShardAlreadyExistsException;
import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
@ -236,15 +237,15 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("[{}] cleaning index, no longer part of the metadata", index); logger.debug("[{}] cleaning index, no longer part of the metadata", index);
} }
final Settings indexSettings;
final IndexService idxService = indicesService.indexService(index); final IndexService idxService = indicesService.indexService(index);
final IndexSettings indexSettings;
if (idxService != null) { if (idxService != null) {
indexSettings = idxService.getIndexSettings().getSettings(); indexSettings = idxService.getIndexSettings();
deleteIndex(index, "index no longer part of the metadata"); deleteIndex(index, "index no longer part of the metadata");
} else { } else {
final IndexMetaData metaData = previousState.metaData().index(index); final IndexMetaData metaData = previousState.metaData().index(index);
assert metaData != null; assert metaData != null;
indexSettings = metaData.getSettings(); indexSettings = new IndexSettings(metaData, settings, Collections.EMPTY_LIST);
indicesService.deleteClosedIndex("closed index no longer part of the metadata", metaData, event.state()); indicesService.deleteClosedIndex("closed index no longer part of the metadata", metaData, event.state());
} }
try { try {

View File

@ -21,7 +21,6 @@ package org.elasticsearch.indices.recovery;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
@ -108,7 +107,7 @@ public class RecoverySource extends AbstractComponent implements IndexEventListe
logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated()); logger.trace("[{}][{}] starting recovery to {}, mark_as_relocated {}", request.shardId().index().name(), request.shardId().id(), request.targetNode(), request.markAsRelocated());
final RecoverySourceHandler handler; final RecoverySourceHandler handler;
if (IndexMetaData.isOnSharedFilesystem(shard.indexSettings())) { if (shard.indexSettings().isOnSharedFilesystem()) {
handler = new SharedFSRecoverySourceHandler(shard, request, recoverySettings, transportService, logger); handler = new SharedFSRecoverySourceHandler(shard, request, recoverySettings, transportService, logger);
} else { } else {
handler = new RecoverySourceHandler(shard, request, recoverySettings, transportService, logger); handler = new RecoverySourceHandler(shard, request, recoverySettings, transportService, logger);

View File

@ -77,7 +77,7 @@ public class RecoveryStatus extends AbstractRefCounted {
super("recovery_status"); super("recovery_status");
this.recoveryId = idGenerator.incrementAndGet(); this.recoveryId = idGenerator.incrementAndGet();
this.listener = listener; this.listener = listener;
this.logger = Loggers.getLogger(getClass(), indexShard.indexSettings(), indexShard.shardId()); this.logger = Loggers.getLogger(getClass(), indexShard.indexSettings().getSettings(), indexShard.shardId());
this.indexShard = indexShard; this.indexShard = indexShard;
this.sourceNode = sourceNode; this.sourceNode = sourceNode;
this.shardId = indexShard.shardId(); this.shardId = indexShard.shardId();

View File

@ -34,7 +34,6 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.List; import java.util.List;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -58,7 +57,7 @@ public class TransportAnalyzeActionTests extends ESTestCase {
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter") .putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.analyzer.custom_analyzer.tokenizer", "whitespace") .put("index.analysis.analyzer.custom_analyzer.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter").build(); .putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter").build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings, Collections.EMPTY_LIST); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(new Index("index"), indexSettings);
environment = new Environment(settings); environment = new Environment(settings);
registry = new AnalysisRegistry(null, environment); registry = new AnalysisRegistry(null, environment);
analysisService = registry.build(idxSettings); analysisService = registry.build(idxSettings);

View File

@ -26,8 +26,10 @@ import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
@ -38,13 +40,11 @@ import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.equalTo;
@LuceneTestCase.SuppressFileSystems("ExtrasFS") // TODO: fix test to allow extras @LuceneTestCase.SuppressFileSystems("ExtrasFS") // TODO: fix test to allow extras
public class NodeEnvironmentTests extends ESTestCase { public class NodeEnvironmentTests extends ESTestCase {
private final Settings idxSettings = Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 1).build(); private final IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("foo", Settings.EMPTY);
public void testNodeLockSingleEnvironment() throws IOException { public void testNodeLockSingleEnvironment() throws IOException {
NodeEnvironment env = newNodeEnvironment(Settings.builder() NodeEnvironment env = newNodeEnvironment(Settings.builder()
@ -100,9 +100,8 @@ public class NodeEnvironmentTests extends ESTestCase {
Files.createDirectories(path.resolve("0")); Files.createDirectories(path.resolve("0"));
Files.createDirectories(path.resolve("1")); Files.createDirectories(path.resolve("1"));
} }
Settings settings = settingsBuilder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 10)).build();
try { try {
env.lockAllForIndex(new Index("foo"), settings, randomIntBetween(0, 10)); env.lockAllForIndex(new Index("foo"), idxSettings, randomIntBetween(0, 10));
fail("shard 0 is locked"); fail("shard 0 is locked");
} catch (LockObtainFailedException ex) { } catch (LockObtainFailedException ex) {
// expected // expected
@ -112,7 +111,7 @@ public class NodeEnvironmentTests extends ESTestCase {
// can lock again? // can lock again?
env.shardLock(new ShardId("foo", 0)).close(); env.shardLock(new ShardId("foo", 0)).close();
List<ShardLock> locks = env.lockAllForIndex(new Index("foo"), settings, randomIntBetween(0, 10)); List<ShardLock> locks = env.lockAllForIndex(new Index("foo"), idxSettings, randomIntBetween(0, 10));
try { try {
env.shardLock(new ShardId("foo", 0)); env.shardLock(new ShardId("foo", 0));
fail("shard is locked"); fail("shard is locked");
@ -244,7 +243,7 @@ public class NodeEnvironmentTests extends ESTestCase {
flipFlop[i] = new AtomicInteger(); 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 CountDownLatch latch = new CountDownLatch(1);
final int iters = scaledRandomIntBetween(10000, 100000); final int iters = scaledRandomIntBetween(10000, 100000);
for (int i = 0; i < threads.length; i++) { for (int i = 0; i < threads.length; i++) {
@ -257,7 +256,7 @@ public class NodeEnvironmentTests extends ESTestCase {
fail(e.getMessage()); fail(e.getMessage());
} }
for (int i = 0; i < iters; i++) { for (int i = 0; i < iters; i++) {
int shard = randomIntBetween(0, counts.length-1); int shard = randomIntBetween(0, counts.length - 1);
try { try {
try (ShardLock autoCloses = env.shardLock(new ShardId("foo", shard), scaledRandomIntBetween(0, 10))) { try (ShardLock autoCloses = env.shardLock(new ShardId("foo", shard), scaledRandomIntBetween(0, 10))) {
counts[shard].value++; counts[shard].value++;
@ -293,18 +292,16 @@ public class NodeEnvironmentTests extends ESTestCase {
String[] dataPaths = tmpPaths(); String[] dataPaths = tmpPaths();
NodeEnvironment env = newNodeEnvironment(dataPaths, "/tmp", Settings.EMPTY); NodeEnvironment env = newNodeEnvironment(dataPaths, "/tmp", Settings.EMPTY);
Settings s1 = Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).build(); IndexSettings s1 = IndexSettingsModule.newIndexSettings("myindex", Settings.EMPTY);
Settings s2 = Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, "/tmp/foo").build(); IndexSettings s2 = IndexSettingsModule.newIndexSettings("myindex", Settings.builder().put(IndexMetaData.SETTING_DATA_PATH, "/tmp/foo").build());
ShardId sid = new ShardId("myindex", 0); ShardId sid = new ShardId("myindex", 0);
Index i = new Index("myindex"); Index i = new Index("myindex");
assertFalse("no settings should mean no custom data path", NodeEnvironment.hasCustomDataPath(s1)); assertFalse("no settings should mean no custom data path", s1.hasCustomDataPath());
assertTrue("settings with path_data should have a custom data path", NodeEnvironment.hasCustomDataPath(s2)); assertTrue("settings with path_data should have a custom data path", s2.hasCustomDataPath());
assertThat(env.availableShardPaths(sid), equalTo(env.availableShardPaths(sid))); 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"))); 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", assertThat("shard paths with a custom data_path should contain only regular paths",
env.availableShardPaths(sid), env.availableShardPaths(sid),

View File

@ -39,7 +39,6 @@ import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.cache.query.QueryCache; import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache; import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache; import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexSearcherWrapper;
@ -83,10 +82,10 @@ public class IndexModuleTests extends ESTestCase {
private NodeServicesProvider nodeServicesProvider; private NodeServicesProvider nodeServicesProvider;
private IndexService.ShardStoreDeleter deleter = new IndexService.ShardStoreDeleter() { private IndexService.ShardStoreDeleter deleter = new IndexService.ShardStoreDeleter() {
@Override @Override
public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException { public void deleteShardStore(String reason, ShardLock lock, IndexSettings indexSettings) throws IOException {
} }
@Override @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(); super.setUp();
index = new Index("foo"); index = new Index("foo");
settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build(); 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); environment = new Environment(settings);
nodeServicesProvider = newNodeServiceProvider(settings, environment, null); nodeServicesProvider = newNodeServiceProvider(settings, environment, null);
nodeEnvironment = new NodeEnvironment(settings, environment); nodeEnvironment = new NodeEnvironment(settings, environment);
@ -142,7 +141,7 @@ public class IndexModuleTests extends ESTestCase {
public void testRegisterIndexStore() throws IOException { public void testRegisterIndexStore() throws IOException {
final Index index = new Index("foo"); 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(); 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)); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
module.addIndexStore("foo_store", FooStore::new); module.addIndexStore("foo_store", FooStore::new);
IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
@ -164,7 +163,7 @@ public class IndexModuleTests extends ESTestCase {
atomicBoolean.set(true); 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)); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
Consumer<Settings> listener = (s) -> {}; Consumer<Settings> listener = (s) -> {};
module.addIndexSettingsListener(listener); module.addIndexSettingsListener(listener);
@ -213,7 +212,7 @@ public class IndexModuleTests extends ESTestCase {
.put("index.similarity.my_similarity.key", "there is a key") .put("index.similarity.my_similarity.key", "there is a key")
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.build(); .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() { module.addSimilarity("test_similarity", (string, settings) -> new SimilarityProvider() {
@Override @Override
public String name() { public String name() {
@ -241,7 +240,7 @@ public class IndexModuleTests extends ESTestCase {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.build(); .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 { try {
module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider); module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
@ -255,7 +254,7 @@ public class IndexModuleTests extends ESTestCase {
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build(); .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 { try {
module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider); module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
@ -267,7 +266,7 @@ public class IndexModuleTests extends ESTestCase {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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 { try {
module.registerQueryCache("index", IndexQueryCache::new); module.registerQueryCache("index", IndexQueryCache::new);
fail("only once"); fail("only once");
@ -295,7 +294,7 @@ public class IndexModuleTests extends ESTestCase {
.put(IndexModule.QUERY_CACHE_TYPE, "custom") .put(IndexModule.QUERY_CACHE_TYPE, "custom")
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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()); module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
try { try {
module.registerQueryCache("custom", (a, b) -> new CustomQueryCache()); module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
@ -313,7 +312,7 @@ public class IndexModuleTests extends ESTestCase {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.cache().query() instanceof IndexQueryCache); assertTrue(indexService.cache().query() instanceof IndexQueryCache);
indexService.close("simon says", false); indexService.close("simon says", false);

View File

@ -66,7 +66,7 @@ public class AnalysisModuleTests extends ModuleTestCase {
public AnalysisService getAnalysisService(AnalysisRegistry registry, Settings settings) throws IOException { public AnalysisService getAnalysisService(AnalysisRegistry registry, Settings settings) throws IOException {
Index index = new Index("test"); Index index = new Index("test");
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
return registry.build(idxSettings); return registry.build(idxSettings);
} }

View File

@ -50,7 +50,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testDefaultAnalyzers() throws IOException { public void testDefaultAnalyzers() throws IOException {
Version version = VersionUtils.randomVersion(getRandom()); Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).put("path.home", createTempDir().toString()).build(); 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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
assertThat(analysisService.defaultSearchAnalyzer().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 { public void testOverrideDefaultAnalyzer() throws IOException {
Version version = VersionUtils.randomVersion(getRandom()); Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); 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.singletonMap("default", analyzerProvider("default")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); 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); Version version = VersionUtils.randomVersionBetween(getRandom(), Version.V_3_0_0, Version.CURRENT);
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
try { 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.singletonMap("default_index", new PreBuiltAnalyzerProvider("default_index", AnalyzerScope.INDEX, new EnglishAnalyzer())),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
fail("Expected ISE"); fail("Expected ISE");
@ -85,7 +85,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testBackCompatOverrideDefaultIndexAnalyzer() { public void testBackCompatOverrideDefaultIndexAnalyzer() {
Version version = VersionUtils.randomVersionBetween(getRandom(), VersionUtils.getFirstVersion(), VersionUtils.getPreviousVersion(Version.V_3_0_0)); 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(); 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.singletonMap("default_index", analyzerProvider("default_index")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
@ -96,7 +96,7 @@ public class AnalysisServiceTests extends ESTestCase {
public void testOverrideDefaultSearchAnalyzer() { public void testOverrideDefaultSearchAnalyzer() {
Version version = VersionUtils.randomVersion(getRandom()); Version version = VersionUtils.randomVersion(getRandom());
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); 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.singletonMap("default_search", analyzerProvider("default_search")),
Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class)); assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(StandardAnalyzer.class));
@ -110,7 +110,7 @@ public class AnalysisServiceTests extends ESTestCase {
Map<String, AnalyzerProvider> analyzers = new HashMap<>(); Map<String, AnalyzerProvider> analyzers = new HashMap<>();
analyzers.put("default_index", analyzerProvider("default_index")); analyzers.put("default_index", analyzerProvider("default_index"));
analyzers.put("default_search", analyzerProvider("default_search")); 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()); analyzers, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap());
assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class)); assertThat(analysisService.defaultIndexAnalyzer().analyzer(), instanceOf(EnglishAnalyzer.class));
assertThat(analysisService.defaultSearchAnalyzer().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") .put("index.analysis.analyzer.custom_analyzer_1.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer_1.filter", "lowercase", "word_delimiter").build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
try (NamedAnalyzer custom_analyser = analysisService.analyzer("custom_analyzer")) { try (NamedAnalyzer custom_analyser = analysisService.analyzer("custom_analyzer")) {
assertNotNull(custom_analyser); assertNotNull(custom_analyser);
@ -174,7 +174,7 @@ public class AnalysisServiceTests extends ESTestCase {
.putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter") .putArray("index.analysis.analyzer.custom_analyzer.filter", "lowercase", "wordDelimiter")
.put("index.analysis.analyzer.custom_analyzer_1.tokenizer", "whitespace") .put("index.analysis.analyzer.custom_analyzer_1.tokenizer", "whitespace")
.putArray("index.analysis.analyzer.custom_analyzer_1.filter", "lowercase", "word_delimiter").build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
TokenFilterFactory word_delimiter = analysisService.tokenFilter("word_delimiter"); TokenFilterFactory word_delimiter = analysisService.tokenFilter("word_delimiter");
@ -185,7 +185,7 @@ public class AnalysisServiceTests extends ESTestCase {
//unconfigured //unconfigured
IndexSettings idxSettings1 = IndexSettingsModule.newIndexSettings(new Index("index"), settingsBuilder() 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); AnalysisService analysisService1 = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings1);
assertSame(analysisService1.tokenFilter("wordDelimiter"), analysisService1.tokenFilter("word_delimiter")); assertSame(analysisService1.tokenFilter("wordDelimiter"), analysisService1.tokenFilter("word_delimiter"));
assertSame(analysisService1.tokenFilter("porterStem"), analysisService1.tokenFilter("porter_stem")); 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 settings = Settings.builder().put("path.home", createTempDir().toString()).build();
Settings indexSettings = settingsBuilder() Settings indexSettings = settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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 analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
AnalysisService otherAnalysisSergice = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings); AnalysisService otherAnalysisSergice = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
final int numIters = randomIntBetween(5, 20); final int numIters = randomIntBetween(5, 20);

View File

@ -49,7 +49,7 @@ public class AnalysisTestsHelper {
if (settings.get(IndexMetaData.SETTING_VERSION_CREATED) == null) { if (settings.get(IndexMetaData.SETTING_VERSION_CREATED) == null) {
settings = Settings.builder().put(settings).put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); 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); Environment environment = new Environment(settings);
return new AnalysisRegistry(new HunspellService(settings, environment, Collections.EMPTY_MAP), environment).build(idxSettings); return new AnalysisRegistry(new HunspellService(settings, environment, Collections.EMPTY_MAP), environment).build(idxSettings);
} }

View File

@ -27,8 +27,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; 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") .putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "my_mapping")
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter"); 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") .putArray("index.analysis.analyzer.custom_with_char_filter.char_filter", "html_strip")
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter"); NamedAnalyzer analyzer1 = analysisService.analyzer("custom_with_char_filter");

View File

@ -42,9 +42,7 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.hasItems;
import static org.hamcrest.Matchers.instanceOf;
/** /**
*/ */
@ -52,7 +50,7 @@ public class CompoundAnalysisTests extends ESTestCase {
public void testDefaultsCompoundAnalysis() throws Exception { public void testDefaultsCompoundAnalysis() throws Exception {
Index index = new Index("test"); Index index = new Index("test");
Settings settings = getJsonSettings(); 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), AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings),
Collections.EMPTY_MAP,Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings); Collections.EMPTY_MAP,Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings);
@ -71,7 +69,7 @@ public class CompoundAnalysisTests extends ESTestCase {
private List<String> analyze(Settings settings, String analyzerName, String text) throws IOException { private List<String> analyze(Settings settings, String analyzerName, String text) throws IOException {
Index index = new Index("test"); 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), AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings),
Collections.EMPTY_MAP, Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings); Collections.EMPTY_MAP, Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new),Collections.EMPTY_MAP,Collections.EMPTY_MAP).build(idxSettings);

View File

@ -22,12 +22,7 @@ package org.elasticsearch.index.analysis;
import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.analysis.MockTokenizer;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.analysis.Tokenizer;
import org.apache.lucene.analysis.ngram.EdgeNGramTokenFilter; import org.apache.lucene.analysis.ngram.*;
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.reverse.ReverseStringFilter; import org.apache.lucene.analysis.reverse.ReverseStringFilter;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -42,7 +37,10 @@ import java.io.IOException;
import java.io.StringReader; import java.io.StringReader;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.lang.reflect.Modifier; 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 com.carrotsearch.randomizedtesting.RandomizedTest.scaledRandomIntBetween;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
@ -52,7 +50,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final Index index = new Index("test"); final Index index = new Index("test");
final String name = "ngr"; final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build(); 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")) { 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(); final Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", tokenChars).build();
try { try {
@ -64,7 +62,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} }
for (String tokenChars : Arrays.asList("letter", " digit ", "punctuation", "DIGIT", "CoNtRoL", "dash_punctuation")) { 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(); 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(); new NGramTokenizerFactory(indexProperties, null, name, settings).create();
// no exception // no exception
@ -76,7 +74,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final String name = "ngr"; final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build(); final Settings indexSettings = newAnalysisSettingsBuilder().build();
final Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 4).putArray("token_chars", new String[0]).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")); tokenizer.setReader(new StringReader("1.34"));
assertTokenStreamContents(tokenizer, new String[] {"1.", "1.3", "1.34", ".3", ".34", "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 String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build(); final Settings indexSettings = newAnalysisSettingsBuilder().build();
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit").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 ")); tokenizer.setReader(new StringReader("Åbc déf g\uD801\uDC00f "));
assertTokenStreamContents(tokenizer, assertTokenStreamContents(tokenizer,
new String[] {"Åb", "Åbc", "bc", "", "déf", "éf", "g\uD801\uDC00", "g\uD801\uDC00f", "\uD801\uDC00f"}); new String[] {"Åb", "Åbc", "bc", "", "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(); 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")); tokenizer.setReader(new StringReader(" a!$ 9"));
assertTokenStreamContents(tokenizer, assertTokenStreamContents(tokenizer,
new String[] {" a", " a!", "a!", "a!$", "!$", "!$ ", "$ ", "$ 9", " 9"}); new String[] {" a", " a!", "a!", "a!$", "!$", "!$ ", "$ ", "$ 9", " 9"});
@ -104,12 +102,12 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
final String name = "ngr"; final String name = "ngr";
final Settings indexSettings = newAnalysisSettingsBuilder().build(); final Settings indexSettings = newAnalysisSettingsBuilder().build();
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("token_chars", "letter,digit").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 ")); tokenizer.setReader(new StringReader("Åbc déf g\uD801\uDC00f "));
assertTokenStreamContents(tokenizer, assertTokenStreamContents(tokenizer,
new String[] {"Åb", "Åbc", "", "déf", "g\uD801\uDC00", "g\uD801\uDC00f"}); new String[] {"Åb", "Åbc", "", "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(); 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")); tokenizer.setReader(new StringReader(" a!$ 9"));
assertTokenStreamContents(tokenizer, assertTokenStreamContents(tokenizer,
new String[] {" a", " a!"}); new String[] {" a", " a!"});
@ -130,7 +128,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} }
Settings settings = builder.build(); Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).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")); edgeNGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) { if (compatVersion) {
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class)); assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class));
@ -141,7 +139,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} else { } else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).put("side", "back").build(); 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(); 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")); edgeNGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(edgeNGramTokenizer, instanceOf(Lucene43EdgeNGramTokenizer.class)); 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 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(); Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
try { 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"); fail("should fail side:back is not supported anymore");
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
} }
@ -170,7 +168,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} }
Settings settings = builder.build(); Settings settings = builder.build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).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")); nGramTokenizer.setReader(new StringReader("foo bar"));
if (compatVersion) { if (compatVersion) {
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class)); assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class));
@ -181,7 +179,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
} else { } else {
Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).build(); Settings settings = newAnalysisSettingsBuilder().put("min_gram", 2).put("max_gram", 3).build();
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).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")); nGramTokenizer.setReader(new StringReader("foo bar"));
assertThat(nGramTokenizer, instanceOf(Lucene43NGramTokenizer.class)); 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(); Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer tokenizer = new MockTokenizer(); Tokenizer tokenizer = new MockTokenizer();
tokenizer.setReader(new StringReader("foo bar")); 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) { if (reverse) {
assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class)); assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class));
} else if (compatVersion) { } else if (compatVersion) {
@ -227,7 +225,7 @@ public class NGramTokenizerFactoryTests extends ESTokenStreamTestCase {
Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build(); Settings indexSettings = newAnalysisSettingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, v.id).build();
Tokenizer tokenizer = new MockTokenizer(); Tokenizer tokenizer = new MockTokenizer();
tokenizer.setReader(new StringReader("foo bar")); 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) { if (reverse) {
assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class)); assertThat(edgeNGramTokenFilter, instanceOf(ReverseStringFilter.class));
} else { } else {

View File

@ -28,8 +28,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
@ -42,7 +40,7 @@ public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("single"); NamedAnalyzer analyzer1 = analysisService.analyzer("single");
@ -60,7 +58,7 @@ public class PatternCaptureTokenFilterTests extends ESTokenStreamTestCase {
public void testNoPatterns() { public void testNoPatterns() {
try { 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"); fail ("Expected IllegalArgumentException");
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
assertThat(e.getMessage(), containsString("required setting 'patterns' is missing")); assertThat(e.getMessage(), containsString("required setting 'patterns' is missing"));

View File

@ -28,8 +28,6 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTokenStreamTestCase; import org.elasticsearch.test.ESTokenStreamTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
public class StopAnalyzerTests extends ESTokenStreamTestCase { public class StopAnalyzerTests extends ESTokenStreamTestCase {
@ -40,7 +38,7 @@ public class StopAnalyzerTests extends ESTokenStreamTestCase {
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString())
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);
NamedAnalyzer analyzer1 = analysisService.analyzer("analyzer1"); NamedAnalyzer analyzer1 = analysisService.analyzer("analyzer1");

View File

@ -42,7 +42,6 @@ import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
@ -68,7 +67,7 @@ public class SynonymsAnalysisTests extends ESTestCase {
.put("path.home", home) .put("path.home", home)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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); analysisService = new AnalysisRegistry(null, new Environment(settings)).build(idxSettings);

View File

@ -23,13 +23,7 @@ import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.*;
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.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.join.BitSetProducer; import org.apache.lucene.search.join.BitSetProducer;
@ -47,7 +41,6 @@ import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
@ -55,7 +48,7 @@ import static org.hamcrest.Matchers.equalTo;
public class BitSetFilterCacheTests extends ESTestCase { 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); private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null);

View File

@ -39,19 +39,15 @@ import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.SegmentReader; import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; 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.logging.ESLoggerFactory;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment; import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
@ -110,7 +106,7 @@ public class CodecTests extends ESTestCase {
Settings nodeSettings = settingsBuilder() Settings nodeSettings = settingsBuilder()
.put("path.home", createTempDir()) .put("path.home", createTempDir())
.build(); .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); SimilarityService similarityService = new SimilarityService(settings, Collections.EMPTY_MAP);
AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(settings); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(settings);
MapperService service = new MapperService(settings, analysisService, similarityService); MapperService service = new MapperService(settings, analysisService, similarityService);

View File

@ -62,25 +62,13 @@ import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.codec.CodecService; import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.engine.Engine.Searcher; import org.elasticsearch.index.engine.Engine.Searcher;
import org.elasticsearch.index.indexing.ShardIndexingService; import org.elasticsearch.index.indexing.ShardIndexingService;
import org.elasticsearch.index.mapper.ContentPath; import org.elasticsearch.index.mapper.*;
import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.DocumentMapperForType;
import org.elasticsearch.index.mapper.DocumentMapperParser;
import org.elasticsearch.index.mapper.Mapper.BuilderContext; 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.ParseContext.Document;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.SourceFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.RootObjectMapper; import org.elasticsearch.index.mapper.object.RootObjectMapper;
import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.*;
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.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils; import org.elasticsearch.index.store.DirectoryUtils;
@ -101,16 +89,10 @@ import java.nio.charset.Charset;
import java.nio.file.DirectoryStream; import java.nio.file.DirectoryStream;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.ArrayList; import java.util.*;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.BrokenBarrierException;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.CyclicBarrier; import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
@ -118,17 +100,12 @@ import java.util.concurrent.atomic.AtomicReference;
import static java.util.Collections.emptyMap; 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.PRIMARY;
import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA; import static org.elasticsearch.index.engine.Engine.Operation.Origin.REPLICA;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.*;
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;
public class InternalEngineTests extends ESTestCase { public class InternalEngineTests extends ESTestCase {
protected final ShardId shardId = new ShardId(new Index("index"), 1); 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; protected ThreadPool threadPool;
@ -138,7 +115,7 @@ public class InternalEngineTests extends ESTestCase {
protected InternalEngine engine; protected InternalEngine engine;
protected InternalEngine replicaEngine; protected InternalEngine replicaEngine;
private Settings defaultSettings; private IndexSettings defaultSettings;
private String codecName; private String codecName;
private Path primaryTranslogDir; private Path primaryTranslogDir;
private Path replicaTranslogDir; private Path replicaTranslogDir;
@ -157,12 +134,12 @@ public class InternalEngineTests extends ESTestCase {
} else { } else {
codecName = "default"; codecName = "default";
} }
defaultSettings = Settings.builder() defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
.put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, randomBoolean()) .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_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us
.put(EngineConfig.INDEX_CODEC_SETTING, codecName) .put(EngineConfig.INDEX_CODEC_SETTING, codecName)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build(); // TODO randomize more settings .build()); // TODO randomize more settings
threadPool = new ThreadPool(getClass().getName()); threadPool = new ThreadPool(getClass().getName());
store = createStore(); store = createStore();
storeReplica = createStore(); storeReplica = createStore();
@ -254,13 +231,13 @@ public class InternalEngineTests extends ESTestCase {
return createEngine(defaultSettings, store, translogPath, new MergeSchedulerConfig(defaultSettings), newMergePolicy()); 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); 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(); 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 EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, INDEX_SETTINGS), indexSettings
, null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig, , null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig,
@ -284,12 +261,12 @@ public class InternalEngineTests extends ESTestCase {
public void testSegments() throws Exception { public void testSegments() throws Exception {
try (Store store = createStore(); try (Store store = createStore();
Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), NoMergePolicy.INSTANCE)) { Engine engine = createEngine(defaultSettings, store, createTempDir(), new MergeSchedulerConfig(defaultSettings), NoMergePolicy.INSTANCE)) {
List<Segment> segments = engine.segments(false); List<Segment> segments = engine.segments(false);
assertThat(segments.isEmpty(), equalTo(true)); assertThat(segments.isEmpty(), equalTo(true));
assertThat(engine.segmentsStats().getCount(), equalTo(0l)); assertThat(engine.segmentsStats().getCount(), equalTo(0l));
assertThat(engine.segmentsStats().getMemoryInBytes(), 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 // create a doc and refresh
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);
@ -1586,7 +1563,7 @@ public class InternalEngineTests extends ESTestCase {
// #10312 // #10312
public void testDeletesAloneCanTriggerRefresh() throws Exception { public void testDeletesAloneCanTriggerRefresh() throws Exception {
try (Store store = createStore(); 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)) { false)) {
engine.config().setIndexingBufferSize(new ByteSizeValue(1, ByteSizeUnit.KB)); engine.config().setIndexingBufferSize(new ByteSizeValue(1, ByteSizeUnit.KB));
for (int i = 0; i < 100; i++) { for (int i = 0; i < 100; i++) {
@ -1636,7 +1613,9 @@ public class InternalEngineTests extends ESTestCase {
// expected // expected
} }
// now it should be OK. // 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()); engine = createEngine(indexSettings, store, primaryTranslogDir, new MergeSchedulerConfig(indexSettings), newMergePolicy());
} }
@ -1776,7 +1755,7 @@ public class InternalEngineTests extends ESTestCase {
} }
CommitStats commitStats = engine.commitStats(); CommitStats commitStats = engine.commitStats();
Map<String, String> userData = commitStats.getUserData(); Map<String, String> userData = commitStats.getUserData();
assertTrue("userdata dosn't contain uuid",userData.containsKey(Translog.TRANSLOG_UUID_KEY)); assertTrue("userdata dosn't contain uuid", userData.containsKey(Translog.TRANSLOG_UUID_KEY));
assertTrue("userdata doesn't contain generation key", userData.containsKey(Translog.TRANSLOG_GENERATION_KEY)); assertTrue("userdata doesn't contain generation key", userData.containsKey(Translog.TRANSLOG_GENERATION_KEY));
assertFalse("userdata contains legacy marker", userData.containsKey("translog_id")); 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(); Settings settings = Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder("test"); RootObjectMapper.Builder rootBuilder = new RootObjectMapper.Builder("test");
Index index = new Index(indexName); 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); 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); SimilarityService similarityService = new SimilarityService(indexSettings, Collections.EMPTY_MAP);
MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService); MapperService mapperService = new MapperService(indexSettings, analysisService, similarityService);
@ -1968,12 +1947,12 @@ public class InternalEngineTests extends ESTestCase {
EngineConfig config = engine.config(); EngineConfig config = engine.config();
/* create a TranslogConfig that has been created with a different UUID */ /* 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() EngineConfig brokenConfig = new EngineConfig(shardId, threadPool, config.getIndexingService(), config.getIndexSettings()
, null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(), , null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getMergeSchedulerConfig(),
config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener() 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 { try {
new InternalEngine(brokenConfig, false); new InternalEngine(brokenConfig, false);

View File

@ -23,13 +23,7 @@ import org.apache.lucene.codecs.Codec;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.document.NumericDocValuesField; import org.apache.lucene.document.NumericDocValuesField;
import org.apache.lucene.document.TextField; import org.apache.lucene.document.TextField;
import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.*;
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.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
@ -73,25 +67,17 @@ import org.junit.Before;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.*;
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;
/** /**
* TODO: document me! * TODO: document me!
*/ */
public class ShadowEngineTests extends ESTestCase { 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 final ShardId shardId = new ShardId(new Index("index"), 1);
protected ThreadPool threadPool; protected ThreadPool threadPool;
@ -103,7 +89,7 @@ public class ShadowEngineTests extends ESTestCase {
protected Engine primaryEngine; protected Engine primaryEngine;
protected Engine replicaEngine; protected Engine replicaEngine;
private Settings defaultSettings; private IndexSettings defaultSettings;
private String codecName; private String codecName;
private Path dirPath; private Path dirPath;
@ -120,12 +106,13 @@ public class ShadowEngineTests extends ESTestCase {
} else { } else {
codecName = "default"; codecName = "default";
} }
defaultSettings = Settings.builder() defaultSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
.put(EngineConfig.INDEX_COMPOUND_ON_FLUSH, randomBoolean()) .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_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us
.put(EngineConfig.INDEX_CODEC_SETTING, codecName) .put(EngineConfig.INDEX_CODEC_SETTING, codecName)
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.build(); // TODO randomize more settings .build()); // TODO randomize more settings
threadPool = new ThreadPool(getClass().getName()); threadPool = new ThreadPool(getClass().getName());
dirPath = createTempDir(); dirPath = createTempDir();
store = createStore(dirPath); store = createStore(dirPath);
@ -185,7 +172,7 @@ public class ShadowEngineTests extends ESTestCase {
protected Store createStore(final Directory directory) throws IOException { 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) { final DirectoryService directoryService = new DirectoryService(shardId, indexSettings) {
@Override @Override
public Directory newDirectory() throws IOException { public Directory newDirectory() throws IOException {
@ -212,22 +199,22 @@ public class ShadowEngineTests extends ESTestCase {
return createInternalEngine(defaultSettings, store, translogPath); 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)); 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()); 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); 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(); 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); TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, 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 EngineConfig config = new EngineConfig(shardId, threadPool, new ShardIndexingService(shardId, indexSettings), indexSettings
, null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig, , null, store, createSnapshotDeletionPolicy(), mergePolicy, mergeSchedulerConfig,
iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(null, logger), new Engine.EventListener() { iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(null, logger), new Engine.EventListener() {
@Override @Override
@ -282,7 +269,7 @@ public class ShadowEngineTests extends ESTestCase {
assertThat(segments.isEmpty(), equalTo(true)); assertThat(segments.isEmpty(), equalTo(true));
assertThat(primaryEngine.segmentsStats().getCount(), equalTo(0l)); assertThat(primaryEngine.segmentsStats().getCount(), equalTo(0l));
assertThat(primaryEngine.segmentsStats().getMemoryInBytes(), 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 // create a doc and refresh
ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null);

View File

@ -26,20 +26,20 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.index.*; import org.apache.lucene.index.*;
import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Accountable; 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.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.Index;
import org.elasticsearch.index.fielddata.plain.*; 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.ContentPath;
import org.elasticsearch.index.mapper.MappedFieldType; import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MappedFieldType.Names; import org.elasticsearch.index.mapper.MappedFieldType.Names;
import org.elasticsearch.index.mapper.Mapper.BuilderContext; import org.elasticsearch.index.mapper.Mapper.BuilderContext;
import org.elasticsearch.index.mapper.MapperBuilders; import org.elasticsearch.index.mapper.MapperBuilders;
import org.elasticsearch.index.mapper.core.*; 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.index.shard.ShardId;
import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
@ -196,7 +196,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
ThreadPool threadPool = new ThreadPool("random_threadpool_name"); ThreadPool threadPool = new ThreadPool("random_threadpool_name");
try { try {
IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null, threadPool); 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.setNames(new Names("some_long"));
ft.setHasDocValues(true); ft.setHasDocValues(true);
ifds.getForField(ft); // no exception ifds.getForField(ft); // no exception

View File

@ -22,13 +22,11 @@ package org.elasticsearch.index.query;
import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator;
import com.fasterxml.jackson.core.JsonParseException; import com.fasterxml.jackson.core.JsonParseException;
import com.fasterxml.jackson.core.io.JsonStringEncoder; import com.fasterxml.jackson.core.io.JsonStringEncoder;
import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.spans.SpanBoostQuery; import org.apache.lucene.search.spans.SpanBoostQuery;
import org.elasticsearch.ElasticsearchException; import org.apache.lucene.util.Accountable;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest;
@ -172,7 +170,7 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); .put(IndexMetaData.SETTING_VERSION_CREATED, version).build();
index = new Index(randomAsciiOfLengthBetween(1, 10)); 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(); final TestClusterService clusterService = new TestClusterService();
clusterService.setState(new ClusterState.Builder(clusterService.state()).metaData(new MetaData.Builder().put( clusterService.setState(new ClusterState.Builder(clusterService.state()).metaData(new MetaData.Builder().put(
new IndexMetaData.Builder(index.name()).settings(indexSettings).numberOfShards(1).numberOfReplicas(0)))); new IndexMetaData.Builder(index.name()).settings(indexSettings).numberOfShards(1).numberOfReplicas(0))));

View File

@ -91,7 +91,7 @@ public class TemplateQueryParserTests extends ESTestCase {
throw new UnsupportedOperationException("client is just a dummy"); throw new UnsupportedOperationException("client is just a dummy");
}); });
Index index = new Index("test"); Index index = new Index("test");
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings);
injector = new ModulesBuilder().add( injector = new ModulesBuilder().add(
new EnvironmentModule(new Environment(settings)), new EnvironmentModule(new Environment(settings)),
new SettingsModule(settings), new SettingsModule(settings),

View File

@ -24,17 +24,12 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField; import org.apache.lucene.document.TextField;
import org.apache.lucene.index.*; import org.apache.lucene.index.*;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; 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.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -46,7 +41,6 @@ import java.util.concurrent.atomic.AtomicInteger;
/** /**
*/ */
public class IndexSearcherWrapperTests extends ESTestCase { 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 { public void testReaderCloseListenerIsCalled() throws IOException {
Directory dir = newDirectory(); Directory dir = newDirectory();

View File

@ -62,7 +62,6 @@ import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.fielddata.FieldDataStats; import org.elasticsearch.index.fielddata.FieldDataStats;
import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.IndexFieldData;
@ -80,6 +79,7 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.VersionUtils;
import java.io.IOException; import java.io.IOException;
@ -106,6 +106,7 @@ import static org.hamcrest.Matchers.equalTo;
* Simple unit-test IndexShard related operations. * Simple unit-test IndexShard related operations.
*/ */
public class IndexShardTests extends ESSingleNodeTestCase { public class IndexShardTests extends ESSingleNodeTestCase {
public void testFlushOnDeleteSetting() throws Exception { public void testFlushOnDeleteSetting() throws Exception {
boolean initValue = randomBoolean(); boolean initValue = randomBoolean();
createIndex("test", settingsBuilder().put(IndexShard.INDEX_FLUSH_ON_CLOSE, initValue).build()); 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); logger.info("--> paths: [{}]", (Object)shardPaths);
// Should not be able to acquire the lock because it's already open // Should not be able to acquire the lock because it's already open
try { try {
NodeEnvironment.acquireFSLockForPaths(Settings.EMPTY, shardPaths); NodeEnvironment.acquireFSLockForPaths(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), shardPaths);
fail("should not have been able to acquire the lock"); fail("should not have been able to acquire the lock");
} catch (LockObtainFailedException e) { } catch (LockObtainFailedException e) {
assertTrue("msg: " + e.getMessage(), e.getMessage().contains("unable to acquire write.lock")); 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) // we're green to delete the shard's directory)
ShardLock sLock = new DummyShardLock(new ShardId("test", 0)); ShardLock sLock = new DummyShardLock(new ShardId("test", 0));
try { 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"); fail("should not have been able to delete the directory");
} catch (LockObtainFailedException e) { } catch (LockObtainFailedException e) {
assertTrue("msg: " + e.getMessage(), e.getMessage().contains("unable to acquire write.lock")); 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)); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard)); 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); routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true); shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard)); 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); routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true); shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard)); 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 // 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); 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); shard.persistMetadata(inactiveRouting, shard.shardRouting);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); 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, 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); shard.updateRoutingEntry(new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1), false);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertFalse("shard state persisted despite of persist=false", shardStateMetaData.equals(getShardStateMetadata(shard))); 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); routing = new ShardRouting(shard.shardRouting, shard.shardRouting.version() + 1);
shard.updateRoutingEntry(routing, true); shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId)); shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard)); 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 { public void testDeleteShardState() throws IOException {
@ -275,7 +276,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
if (shardRouting == null) { if (shardRouting == null) {
return null; return null;
} else { } 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());
} }
} }

View File

@ -20,13 +20,12 @@ package org.elasticsearch.index.shard;
import org.apache.lucene.mockfile.FilterFileSystemProvider; import org.apache.lucene.mockfile.FilterFileSystemProvider;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.io.PathUtilsForTesting; import org.elasticsearch.common.io.PathUtilsForTesting;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment; import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment.NodePath;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.NodeEnvironment.NodePath;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTestCase; 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.FileAttributeView;
import java.nio.file.attribute.FileStoreAttributeView; import java.nio.file.attribute.FileStoreAttributeView;
import java.nio.file.spi.FileSystemProvider; import java.nio.file.spi.FileSystemProvider;
import java.util.ArrayList; import java.util.*;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/** Separate test class from ShardPathTests because we need static (BeforeClass) setup to install mock filesystems... */ /** Separate test class from ShardPathTests because we need static (BeforeClass) setup to install mock filesystems... */
public class NewPathForShardTests extends ESTestCase { 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: // 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"); private static MockFileStore aFileStore = new MockFileStore("mocka");

View File

@ -22,13 +22,11 @@ import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.containsString;
@ -46,7 +44,7 @@ public class ShardPathTests extends ESTestCase {
Path[] paths = env.availableShardPaths(shardId); Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths); Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path); 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(path, shardPath.getDataPath());
assertEquals("0xDEADBEEF", shardPath.getIndexUUID()); assertEquals("0xDEADBEEF", shardPath.getIndexUUID());
assertEquals("foo", shardPath.getShardId().getIndex()); 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); assumeTrue("This test tests multi data.path but we only got one", paths.length > 1);
int id = randomIntBetween(1, 10); int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, paths); 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"); fail("Expected IllegalStateException");
} catch (IllegalStateException e) { } catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("more than one shard state found")); assertThat(e.getMessage(), containsString("more than one shard state found"));
@ -82,7 +80,7 @@ public class ShardPathTests extends ESTestCase {
Path path = randomFrom(paths); Path path = randomFrom(paths);
int id = randomIntBetween(1, 10); int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, path); 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"); fail("Expected IllegalStateException");
} catch (IllegalStateException e) { } catch (IllegalStateException e) {
assertThat(e.getMessage(), containsString("expected: foobar on shard path")); assertThat(e.getMessage(), containsString("expected: foobar on shard path"));
@ -136,7 +134,7 @@ public class ShardPathTests extends ESTestCase {
Path[] paths = env.availableShardPaths(shardId); Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths); Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path); 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; boolean found = false;
for (Path p : env.nodeDataPaths()) { for (Path p : env.nodeDataPaths()) {
if (p.equals(shardPath.getRootStatePath())) { if (p.equals(shardPath.getRootStatePath())) {

View File

@ -34,7 +34,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collections;
import java.util.Locale; import java.util.Locale;
/** /**
@ -47,7 +46,7 @@ public class IndexStoreTests extends ESTestCase {
final IndexModule.Type type = RandomPicks.randomFrom(random(), values); final IndexModule.Type type = RandomPicks.randomFrom(random(), values);
Settings settings = Settings.settingsBuilder().put(IndexModule.STORE_TYPE, type.name().toLowerCase(Locale.ROOT)) Settings settings = Settings.settingsBuilder().put(IndexModule.STORE_TYPE, type.name().toLowerCase(Locale.ROOT))
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); .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))); 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)) { try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
switch (type) { switch (type) {
@ -80,7 +79,7 @@ public class IndexStoreTests extends ESTestCase {
public void testStoreDirectoryDefault() throws IOException { public void testStoreDirectoryDefault() throws IOException {
final Path tempDir = createTempDir().resolve("foo").resolve("0"); 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)) { try (final Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) {
if (Constants.WINDOWS) { if (Constants.WINDOWS) {
assertTrue(directory.toString(), directory instanceof MMapDirectory || directory instanceof SimpleFSDirectory); assertTrue(directory.toString(), directory instanceof MMapDirectory || directory instanceof SimpleFSDirectory);

View File

@ -24,35 +24,9 @@ import org.apache.lucene.codecs.FilterCodec;
import org.apache.lucene.codecs.SegmentInfoFormat; import org.apache.lucene.codecs.SegmentInfoFormat;
import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat; import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat;
import org.apache.lucene.codecs.lucene54.Lucene54Codec; import org.apache.lucene.codecs.lucene54.Lucene54Codec;
import org.apache.lucene.document.Document; import org.apache.lucene.document.*;
import org.apache.lucene.document.Field; import org.apache.lucene.index.*;
import org.apache.lucene.document.SortedDocValuesField; import org.apache.lucene.store.*;
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.util.BytesRef; import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.TestUtil; 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.emptyMap;
import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.elasticsearch.test.VersionUtils.randomVersion;
import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.*;
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;
public class StoreTests extends ESTestCase { 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 { public void testRefCount() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1); final ShardId shardId = new ShardId(new Index("index"), 1);
@ -1147,7 +1114,7 @@ public class StoreTests extends ESTestCase {
Settings settings = Settings.builder() Settings settings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueMinutes(0)).build(); .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; long initialStoreSize = 0;
for (String extraFiles : store.directory().listAll()) { for (String extraFiles : store.directory().listAll()) {
assertTrue("expected extraFS file but got: " + extraFiles, extraFiles.startsWith("extra")); assertTrue("expected extraFS file but got: " + extraFiles, extraFiles.startsWith("extra"));

View File

@ -23,12 +23,10 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; 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("index.translog.fs.buffer_size", 10 + randomInt(128 * 1024), ByteSizeUnit.BYTES)
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build(); .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); return new Translog(translogConfig);
} }
} }

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.translog; package org.elasticsearch.index.translog;
import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.index.Term; import org.apache.lucene.index.Term;
import org.apache.lucene.store.AlreadyClosedException; 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.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
@ -56,28 +54,14 @@ import java.nio.file.Files;
import java.nio.file.InvalidPathException; import java.nio.file.InvalidPathException;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardOpenOption; import java.nio.file.StandardOpenOption;
import java.util.ArrayList; import java.util.*;
import java.util.Collections; import java.util.concurrent.*;
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.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
/** /**
* *
@ -131,7 +115,7 @@ public class TranslogTests extends ESTestCase {
.put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name()) .put(TranslogConfig.INDEX_TRANSLOG_FS_TYPE, TranslogWriter.Type.SIMPLE.name())
.put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, org.elasticsearch.Version.CURRENT)
.build(); .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); return new Translog(translogConfig);
} }

View File

@ -29,9 +29,11 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.GatewayMetaState; import org.elasticsearch.gateway.GatewayMetaState;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -44,6 +46,7 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
public IndicesService getIndicesService() { public IndicesService getIndicesService() {
return getInstanceFromNode(IndicesService.class); return getInstanceFromNode(IndicesService.class);
} }
public NodeEnvironment getNodeEnvironment() { public NodeEnvironment getNodeEnvironment() {
return getInstanceFromNode(NodeEnvironment.class); return getInstanceFromNode(NodeEnvironment.class);
} }
@ -56,12 +59,12 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
public void testCanDeleteIndexContent() { public void testCanDeleteIndexContent() {
IndicesService indicesService = getIndicesService(); IndicesService indicesService = getIndicesService();
Settings idxSettings = settings(Version.CURRENT) IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("test", Settings.builder()
.put(IndexMetaData.SETTING_SHADOW_REPLICAS, true) .put(IndexMetaData.SETTING_SHADOW_REPLICAS, true)
.put(IndexMetaData.SETTING_DATA_PATH, "/foo/bar") .put(IndexMetaData.SETTING_DATA_PATH, "/foo/bar")
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 4)) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, randomIntBetween(1, 4))
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 3)) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, randomIntBetween(0, 3))
.build(); .build());
assertFalse("shard on shared filesystem", indicesService.canDeleteIndexContents(new Index("test"), idxSettings, false)); 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)); 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()); ShardPath shardPath = ShardPath.loadShardPath(logger, getNodeEnvironment(), new ShardId(test.index(), 0), test.getIndexSettings());
assertEquals(shardPath, path); assertEquals(shardPath, path);
try { 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"); fail("can't get lock");
} catch (LockObtainFailedException ex) { } catch (LockObtainFailedException ex) {
@ -151,13 +154,13 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
int numPending = 1; int numPending = 1;
if (randomBoolean()) { if (randomBoolean()) {
indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings().getSettings()); indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
} else { } else {
if (randomBoolean()) { if (randomBoolean()) {
numPending++; 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")); assertAcked(client().admin().indices().prepareClose("test"));
assertTrue(path.exists()); assertTrue(path.exists());
@ -165,17 +168,17 @@ public class IndicesServiceTests extends ESSingleNodeTestCase {
assertEquals(indicesService.numPendingDeletes(test.index()), numPending); assertEquals(indicesService.numPendingDeletes(test.index()), numPending);
// shard lock released... we can now delete // 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); assertEquals(indicesService.numPendingDeletes(test.index()), 0);
assertFalse(path.exists()); assertFalse(path.exists());
if (randomBoolean()) { if (randomBoolean()) {
indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings().getSettings()); indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings().getSettings()); indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings());
indicesService.addPendingDelete(new ShardId("bogus", 1), test.getIndexSettings().getSettings()); indicesService.addPendingDelete(new ShardId("bogus", 1), test.getIndexSettings());
assertEquals(indicesService.numPendingDeletes(test.index()), 2); assertEquals(indicesService.numPendingDeletes(test.index()), 2);
// shard lock released... we can now delete // 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); assertEquals(indicesService.numPendingDeletes(test.index()), 0);
} }
assertAcked(client().admin().indices().prepareOpen("test")); assertAcked(client().admin().indices().prepareOpen("test"));

View File

@ -22,8 +22,13 @@ import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.document.StringField; import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField; import org.apache.lucene.document.TextField;
import org.apache.lucene.index.*; import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.*; 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.apache.lucene.util.IOUtils;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.Version; 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.Store;
import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.test.CorruptionUtils;
import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.CorruptionUtils;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
@ -52,10 +57,8 @@ import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import static org.hamcrest.Matchers.is;
public class RecoverySourceHandlerTests extends ESTestCase { 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 ShardId shardId = new ShardId(INDEX_SETTINGS.getIndex(), 1);
private final NodeSettingsService service = new NodeSettingsService(Settings.EMPTY); private final NodeSettingsService service = new NodeSettingsService(Settings.EMPTY);

View File

@ -33,7 +33,6 @@ import org.elasticsearch.plugin.analysis.icu.AnalysisICUPlugin;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -49,7 +48,7 @@ public class AnalysisTestUtils {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings), Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .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; return analysisService;
} }
} }

View File

@ -44,12 +44,8 @@ import java.io.Reader;
import java.io.StringReader; import java.io.StringReader;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.*;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.instanceOf;
import static org.hamcrest.Matchers.notNullValue;
/** /**
*/ */
@ -211,7 +207,7 @@ public class KuromojiAnalysisTests extends ESTestCase {
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .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, public static void assertSimpleTSOutput(TokenStream stream,

View File

@ -35,7 +35,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert; import org.hamcrest.MatcherAssert;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
@ -61,6 +60,6 @@ public class SimplePhoneticAnalysisTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings), Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .createInjector();
return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList())); return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
} }
} }

View File

@ -35,7 +35,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert; import org.hamcrest.MatcherAssert;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
@ -54,7 +53,7 @@ public class SimpleSmartChineseAnalysisTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings), Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .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"); TokenizerFactory tokenizerFactory = analysisService.tokenizer("smartcn_tokenizer");
MatcherAssert.assertThat(tokenizerFactory, instanceOf(SmartChineseTokenizerTokenizerFactory.class)); MatcherAssert.assertThat(tokenizerFactory, instanceOf(SmartChineseTokenizerTokenizerFactory.class));
} }

View File

@ -38,10 +38,8 @@ import org.elasticsearch.test.IndexSettingsModule;
import org.hamcrest.MatcherAssert; import org.hamcrest.MatcherAssert;
import java.io.IOException; import java.io.IOException;
import java.util.Collections;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.settings.Settings.Builder.EMPTY_SETTINGS;
import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.instanceOf;
/** /**
@ -61,7 +59,7 @@ public class PolishAnalysisTests extends ESTestCase {
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .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"); TokenFilterFactory tokenizerFactory = analysisService.tokenFilter("polish_stem");
MatcherAssert.assertThat(tokenizerFactory, instanceOf(PolishStemTokenFilterFactory.class)); MatcherAssert.assertThat(tokenizerFactory, instanceOf(PolishStemTokenFilterFactory.class));

View File

@ -40,7 +40,6 @@ import org.elasticsearch.test.IndexSettingsModule;
import java.io.IOException; import java.io.IOException;
import java.io.StringReader; import java.io.StringReader;
import java.util.Collections;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
@ -103,6 +102,6 @@ public class SimplePolishTokenFilterTests extends ESTestCase {
Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings), Injector parentInjector = new ModulesBuilder().add(new SettingsModule(settings),
new EnvironmentModule(new Environment(settings)), analysisModule) new EnvironmentModule(new Environment(settings)), analysisModule)
.createInjector(); .createInjector();
return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings, Collections.emptyList())); return parentInjector.getInstance(AnalysisRegistry.class).build(IndexSettingsModule.newIndexSettings(index, settings));
} }
} }

View File

@ -45,7 +45,7 @@ class MapperTestUtils {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(indexSettings) .put(indexSettings)
.build(); .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); AnalysisService analysisService = new AnalysisRegistry(null, new Environment(nodeSettings)).build(idxSettings);
SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap()); SimilarityService similarityService = new SimilarityService(idxSettings, Collections.emptyMap());
return new MapperService(idxSettings, analysisService, similarityService); return new MapperService(idxSettings, analysisService, similarityService);

View File

@ -19,33 +19,13 @@
package org.elasticsearch.test; package org.elasticsearch.test;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings; 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.Index;
import org.elasticsearch.index.IndexSettings; 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.Collections;
import java.util.*;
import java.util.function.Consumer;
public class IndexSettingsModule extends AbstractModule { public class IndexSettingsModule extends AbstractModule {
@ -57,12 +37,17 @@ public class IndexSettingsModule extends AbstractModule {
this.index = index; this.index = index;
} }
@Override @Override
protected void configure() { 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<Consumer<Settings>> 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) Settings build = Settings.settingsBuilder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)

View File

@ -21,11 +21,7 @@ package org.elasticsearch.test.engine;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.AssertingIndexSearcher; import org.apache.lucene.search.*;
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.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
@ -79,18 +75,18 @@ public final class MockEngineSupport {
} }
public MockEngineSupport(EngineConfig config, Class<? extends FilterDirectoryReader> wrapper) { public MockEngineSupport(EngineConfig config, Class<? extends FilterDirectoryReader> wrapper) {
Settings indexSettings = config.getIndexSettings(); Settings settings = config.getIndexSettings().getSettings();
shardId = config.getShardId(); shardId = config.getShardId();
filterCache = config.getQueryCache(); filterCache = config.getQueryCache();
filterCachingPolicy = config.getQueryCachingPolicy(); 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); 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; boolean wrapReader = random.nextDouble() < ratio;
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("Using [{}] for shard [{}] seed: [{}] wrapReader: [{}]", this.getClass().getName(), shardId, seed, wrapReader); 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(); this.searcherCloseable = new SearcherCloseable();
LuceneTestCase.closeAfterSuite(searcherCloseable); // only one suite closeable per Engine LuceneTestCase.closeAfterSuite(searcherCloseable); // only one suite closeable per Engine
} }

View File

@ -21,7 +21,6 @@ package org.elasticsearch.test.engine;
import org.apache.lucene.index.FilterDirectoryReader; import org.apache.lucene.index.FilterDirectoryReader;
import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherManager; import org.apache.lucene.search.SearcherManager;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.engine.InternalEngine;
@ -35,7 +34,7 @@ final class MockInternalEngine extends InternalEngine {
MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery, Class<? extends FilterDirectoryReader> wrapper) throws EngineException { MockInternalEngine(EngineConfig config, boolean skipInitialTranslogRecovery, Class<? extends FilterDirectoryReader> wrapper) throws EngineException {
super(config, skipInitialTranslogRecovery); super(config, skipInitialTranslogRecovery);
randomizeFlushOnClose = IndexMetaData.isOnSharedFilesystem(config.getIndexSettings()) == false; randomizeFlushOnClose = config.getIndexSettings().isOnSharedFilesystem() == false;
wrapperClass = wrapper; wrapperClass = wrapper;
} }

View File

@ -19,20 +19,22 @@
package org.elasticsearch.test.store; package org.elasticsearch.test.store;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.*; import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.store.DirectoryService; import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.plugins.Plugin; 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 { public class MockFSIndexStore extends IndexStore {
@ -84,7 +86,7 @@ public class MockFSIndexStore extends IndexStore {
if (indexShard != null) { if (indexShard != null) {
Boolean remove = shardSet.remove(indexShard); Boolean remove = shardSet.remove(indexShard);
if (remove == Boolean.TRUE) { 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()); MockFSDirectoryService.checkIndex(logger, indexShard.store(), indexShard.shardId());
} }
} }
@ -92,7 +94,7 @@ public class MockFSIndexStore extends IndexStore {
@Override @Override
public void indexShardStateChanged(IndexShard indexShard, @Nullable IndexShardState previousState, IndexShardState currentState, @Nullable String reason) { 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); shardSet.put(indexShard, Boolean.TRUE);
} }