Remove guice from the index level

This commit removes guice from the index level and adds a simple extension point
to add class instances with an index-lifecycle scope.
This commit is contained in:
Simon Willnauer 2015-11-04 11:42:18 +01:00
parent 45f7844948
commit 487af301ea
49 changed files with 551 additions and 743 deletions

View File

@ -55,6 +55,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
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.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperParsingException; import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
@ -95,12 +96,14 @@ public class MetaDataCreateIndexService extends AbstractComponent {
private final AliasValidator aliasValidator; private final AliasValidator aliasValidator;
private final IndexTemplateFilter indexTemplateFilter; private final IndexTemplateFilter indexTemplateFilter;
private final Environment env; private final Environment env;
private final NodeServicesProvider nodeServicesProvider;
@Inject @Inject
public MetaDataCreateIndexService(Settings settings, ClusterService clusterService, public MetaDataCreateIndexService(Settings settings, ClusterService clusterService,
IndicesService indicesService, AllocationService allocationService, IndicesService indicesService, AllocationService allocationService,
Version version, AliasValidator aliasValidator, Version version, AliasValidator aliasValidator,
Set<IndexTemplateFilter> indexTemplateFilters, Environment env) { Set<IndexTemplateFilter> indexTemplateFilters, Environment env, NodeServicesProvider nodeServicesProvider) {
super(settings); super(settings);
this.clusterService = clusterService; this.clusterService = clusterService;
this.indicesService = indicesService; this.indicesService = indicesService;
@ -108,6 +111,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
this.version = version; this.version = version;
this.aliasValidator = aliasValidator; this.aliasValidator = aliasValidator;
this.env = env; this.env = env;
this.nodeServicesProvider = nodeServicesProvider;
if (indexTemplateFilters.isEmpty()) { if (indexTemplateFilters.isEmpty()) {
this.indexTemplateFilter = DEFAULT_INDEX_TEMPLATE_FILTER; this.indexTemplateFilter = DEFAULT_INDEX_TEMPLATE_FILTER;
@ -295,7 +299,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
// Set up everything, now locally create the index to see that things are ok, and apply // Set up everything, now locally create the index to see that things are ok, and apply
final IndexMetaData tmpImd = IndexMetaData.builder(request.index()).settings(actualIndexSettings).build(); final IndexMetaData tmpImd = IndexMetaData.builder(request.index()).settings(actualIndexSettings).build();
// create the index here (on the master) to validate it can be created, as well as adding the mapping // create the index here (on the master) to validate it can be created, as well as adding the mapping
indicesService.createIndex(tmpImd, Collections.EMPTY_LIST); indicesService.createIndex(nodeServicesProvider, tmpImd, Collections.EMPTY_LIST);
indexCreated = true; indexCreated = true;
// now add the mappings // now add the mappings
IndexService indexService = indicesService.indexServiceSafe(request.index()); IndexService indexService = indicesService.indexServiceSafe(request.index());

View File

@ -33,6 +33,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
@ -49,12 +50,15 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
private final AliasValidator aliasValidator; private final AliasValidator aliasValidator;
private final NodeServicesProvider nodeServicesProvider;
@Inject @Inject
public MetaDataIndexAliasesService(Settings settings, ClusterService clusterService, IndicesService indicesService, AliasValidator aliasValidator) { public MetaDataIndexAliasesService(Settings settings, ClusterService clusterService, IndicesService indicesService, AliasValidator aliasValidator, NodeServicesProvider nodeServicesProvider) {
super(settings); super(settings);
this.clusterService = clusterService; this.clusterService = clusterService;
this.indicesService = indicesService; this.indicesService = indicesService;
this.aliasValidator = aliasValidator; this.aliasValidator = aliasValidator;
this.nodeServicesProvider = nodeServicesProvider;
} }
public void indicesAliases(final IndicesAliasesClusterStateUpdateRequest request, final ActionListener<ClusterStateUpdateResponse> listener) { public void indicesAliases(final IndicesAliasesClusterStateUpdateRequest request, final ActionListener<ClusterStateUpdateResponse> listener) {
@ -95,7 +99,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
if (indexService == null) { if (indexService == null) {
// temporarily create the index and add mappings so we can parse the filter // temporarily create the index and add mappings so we can parse the filter
try { try {
indexService = indicesService.createIndex(indexMetaData, Collections.EMPTY_LIST); indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.EMPTY_LIST);
if (indexMetaData.getMappings().containsKey(MapperService.DEFAULT_MAPPING)) { if (indexMetaData.getMappings().containsKey(MapperService.DEFAULT_MAPPING)) {
indexService.mapperService().merge(MapperService.DEFAULT_MAPPING, indexMetaData.getMappings().get(MapperService.DEFAULT_MAPPING).source(), false, false); indexService.mapperService().merge(MapperService.DEFAULT_MAPPING, indexMetaData.getMappings().get(MapperService.DEFAULT_MAPPING).source(), false, false);
} }

View File

@ -35,6 +35,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeMappingException;
@ -57,12 +58,14 @@ public class MetaDataMappingService extends AbstractComponent {
private final List<MappingTask> refreshOrUpdateQueue = new ArrayList<>(); private final List<MappingTask> refreshOrUpdateQueue = new ArrayList<>();
private long refreshOrUpdateInsertOrder; private long refreshOrUpdateInsertOrder;
private long refreshOrUpdateProcessedInsertOrder; private long refreshOrUpdateProcessedInsertOrder;
private final NodeServicesProvider nodeServicesProvider;
@Inject @Inject
public MetaDataMappingService(Settings settings, ClusterService clusterService, IndicesService indicesService) { public MetaDataMappingService(Settings settings, ClusterService clusterService, IndicesService indicesService, NodeServicesProvider nodeServicesProvider) {
super(settings); super(settings);
this.clusterService = clusterService; this.clusterService = clusterService;
this.indicesService = indicesService; this.indicesService = indicesService;
this.nodeServicesProvider = nodeServicesProvider;
} }
static class MappingTask { static class MappingTask {
@ -172,7 +175,7 @@ public class MetaDataMappingService extends AbstractComponent {
IndexService indexService = indicesService.indexService(index); IndexService indexService = indicesService.indexService(index);
if (indexService == null) { if (indexService == null) {
// we need to create the index here, and add the current mapping to it, so we can merge // we need to create the index here, and add the current mapping to it, so we can merge
indexService = indicesService.createIndex(indexMetaData, Collections.EMPTY_LIST); indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.EMPTY_LIST);
removeIndex = true; removeIndex = true;
Set<String> typesToIntroduce = new HashSet<>(); Set<String> typesToIntroduce = new HashSet<>();
for (MappingTask task : tasks) { for (MappingTask task : tasks) {
@ -350,7 +353,7 @@ public class MetaDataMappingService extends AbstractComponent {
continue; continue;
} }
final IndexMetaData indexMetaData = currentState.metaData().index(index); final IndexMetaData indexMetaData = currentState.metaData().index(index);
IndexService indexService = indicesService.createIndex(indexMetaData, Collections.EMPTY_LIST); IndexService indexService = indicesService.createIndex(nodeServicesProvider, indexMetaData, Collections.EMPTY_LIST);
indicesToClose.add(indexMetaData.getIndex()); indicesToClose.add(indexMetaData.getIndex());
// make sure to add custom default mapping if exists // make sure to add custom default mapping if exists
if (indexMetaData.getMappings().containsKey(MapperService.DEFAULT_MAPPING)) { if (indexMetaData.getMappings().containsKey(MapperService.DEFAULT_MAPPING)) {

View File

@ -20,20 +20,14 @@
package org.elasticsearch.index; package org.elasticsearch.index;
import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.SetOnce;
import org.elasticsearch.ElasticsearchException;
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.env.NodeEnvironment;
import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
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.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.similarity.BM25SimilarityProvider; import org.elasticsearch.index.similarity.BM25SimilarityProvider;
@ -41,7 +35,7 @@ import org.elasticsearch.index.similarity.SimilarityProvider;
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.IndexStoreConfig; import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import java.io.IOException; import java.io.IOException;
@ -61,7 +55,7 @@ import java.util.function.Consumer;
* <li>Settings update listener - Custom settings update listener can be registered via {@link #addIndexSettingsListener(Consumer)}</li> * <li>Settings update listener - Custom settings update listener can be registered via {@link #addIndexSettingsListener(Consumer)}</li>
* </ul> * </ul>
*/ */
public final class IndexModule extends AbstractModule { public final class IndexModule {
public static final String STORE_TYPE = "index.store.type"; public static final String STORE_TYPE = "index.store.type";
public static final String SIMILARITY_SETTINGS_PREFIX = "index.similarity"; public static final String SIMILARITY_SETTINGS_PREFIX = "index.similarity";
@ -72,10 +66,9 @@ public final class IndexModule extends AbstractModule {
public static final String QUERY_CACHE_EVERYTHING = "index.queries.cache.everything"; public static final String QUERY_CACHE_EVERYTHING = "index.queries.cache.everything";
private final IndexSettings indexSettings; private final IndexSettings indexSettings;
private final IndexStoreConfig indexStoreConfig; private final IndexStoreConfig indexStoreConfig;
private final IndicesQueryCache indicesQueryCache;
private final AnalysisRegistry analysisRegistry; private final AnalysisRegistry analysisRegistry;
// pkg private so tests can mock // pkg private so tests can mock
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class; final SetOnce<EngineFactory> engineFactory = new SetOnce<>();
private SetOnce<IndexSearcherWrapperFactory> indexSearcherWrapper = new SetOnce<>(); private SetOnce<IndexSearcherWrapperFactory> indexSearcherWrapper = new SetOnce<>();
private final Set<Consumer<Settings>> settingsConsumers = new HashSet<>(); private final Set<Consumer<Settings>> settingsConsumers = new HashSet<>();
private final Set<IndexEventListener> indexEventListeners = new HashSet<>(); private final Set<IndexEventListener> indexEventListeners = new HashSet<>();
@ -83,14 +76,11 @@ public final class IndexModule extends AbstractModule {
private final Map<String, BiFunction<String, Settings, SimilarityProvider>> similarities = new HashMap<>(); private final Map<String, BiFunction<String, Settings, SimilarityProvider>> similarities = new HashMap<>();
private final Map<String, BiFunction<IndexSettings, IndexStoreConfig, IndexStore>> storeTypes = new HashMap<>(); private final Map<String, BiFunction<IndexSettings, IndexStoreConfig, IndexStore>> storeTypes = new HashMap<>();
private final Map<String, BiFunction<IndexSettings, IndicesQueryCache, QueryCache>> queryCaches = new HashMap<>(); private final Map<String, BiFunction<IndexSettings, IndicesQueryCache, QueryCache>> queryCaches = new HashMap<>();
private IndicesWarmer indicesWarmer;
public IndexModule(IndexSettings indexSettings, IndexStoreConfig indexStoreConfig, IndicesQueryCache indicesQueryCache, IndicesWarmer warmer, AnalysisRegistry analysisRegistry) { public IndexModule(IndexSettings indexSettings, IndexStoreConfig indexStoreConfig, AnalysisRegistry analysisRegistry) {
this.indexStoreConfig = indexStoreConfig; this.indexStoreConfig = indexStoreConfig;
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.indicesQueryCache = indicesQueryCache;
this.indicesWarmer = warmer;
this.analysisRegistry = analysisRegistry; this.analysisRegistry = analysisRegistry;
registerQueryCache(INDEX_QUERY_CACHE, IndexQueryCache::new); registerQueryCache(INDEX_QUERY_CACHE, IndexQueryCache::new);
registerQueryCache(NONE_QUERY_CACHE, (a, b) -> new NoneQueryCache(a)); registerQueryCache(NONE_QUERY_CACHE, (a, b) -> new NoneQueryCache(a));
@ -220,50 +210,6 @@ public final class IndexModule extends AbstractModule {
return false; return false;
} }
@Override
protected void configure() {
final IndexSettings settings = indexSettings.newWithListener(settingsConsumers);
try {
bind(AnalysisService.class).toInstance(analysisRegistry.build(settings));
} catch (IOException e) {
throw new ElasticsearchException("can't create analysis service", e);
}
bind(EngineFactory.class).to(engineFactoryImpl).asEagerSingleton();
bind(IndexSearcherWrapperFactory.class).toInstance(indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get());
bind(IndexEventListener.class).toInstance(freeze());
bind(IndexService.class).asEagerSingleton();
bind(IndexServicesProvider.class).asEagerSingleton();
bind(MapperService.class).asEagerSingleton();
bind(IndexFieldDataService.class).asEagerSingleton();
bind(IndexSettings.class).toInstance(settings);
final String storeType = settings.getSettings().get(STORE_TYPE);
final IndexStore store;
if (storeType == null || isBuiltinType(storeType)) {
store = new IndexStore(settings, indexStoreConfig);
} else {
BiFunction<IndexSettings, IndexStoreConfig, IndexStore> factory = storeTypes.get(storeType);
if (factory == null) {
throw new IllegalArgumentException("Unknown store type [" + storeType + "]");
}
store = factory.apply(settings, indexStoreConfig);
if (store == null) {
throw new IllegalStateException("store must not be null");
}
}
final String queryCacheType = settings.getSettings().get(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE);
BiFunction<IndexSettings, IndicesQueryCache, QueryCache> queryCacheProvider = queryCaches.get(queryCacheType);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(settings, indicesWarmer);
QueryCache queryCache = queryCacheProvider.apply(settings, indicesQueryCache);
IndexCache indexCache = new IndexCache(settings, queryCache, bitsetFilterCache);
bind(QueryCache.class).toInstance(queryCache);
bind(IndexCache.class).toInstance(indexCache);
bind(BitsetFilterCache.class).toInstance(bitsetFilterCache);
bind(IndexStore.class).toInstance(store);
bind(SimilarityService.class).toInstance(new SimilarityService(settings, similarities));
}
public enum Type { public enum Type {
NIOFS, NIOFS,
MMAPFS, MMAPFS,
@ -291,4 +237,29 @@ public final class IndexModule extends AbstractModule {
*/ */
IndexSearcherWrapper newWrapper(final IndexService indexService); IndexSearcherWrapper newWrapper(final IndexService indexService);
} }
public IndexService newIndexService(NodeEnvironment environment, IndexService.ShardStoreDeleter shardStoreDeleter, NodeServicesProvider servicesProvider) throws IOException {
final IndexSettings settings = indexSettings.newWithListener(settingsConsumers);
IndexSearcherWrapperFactory searcherWrapperFactory = indexSearcherWrapper.get() == null ? (shard) -> null : indexSearcherWrapper.get();
IndexEventListener eventListener = freeze();
final String storeType = settings.getSettings().get(STORE_TYPE);
final IndexStore store;
if (storeType == null || isBuiltinType(storeType)) {
store = new IndexStore(settings, indexStoreConfig);
} else {
BiFunction<IndexSettings, IndexStoreConfig, IndexStore> factory = storeTypes.get(storeType);
if (factory == null) {
throw new IllegalArgumentException("Unknown store type [" + storeType + "]");
}
store = factory.apply(settings, indexStoreConfig);
if (store == null) {
throw new IllegalStateException("store must not be null");
}
}
final String queryCacheType = settings.getSettings().get(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE);
final BiFunction<IndexSettings, IndicesQueryCache, QueryCache> queryCacheProvider = queryCaches.get(queryCacheType);
final QueryCache queryCache = queryCacheProvider.apply(settings, servicesProvider.getIndicesQueryCache());
return new IndexService(settings, environment, new SimilarityService(settings, similarities), shardStoreDeleter, analysisRegistry, engineFactory.get(),
servicesProvider, queryCache, store, eventListener, searcherWrapperFactory);
}
} }

View File

@ -35,9 +35,12 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock; import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.fielddata.FieldDataType; import org.elasticsearch.index.fielddata.FieldDataType;
import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataCache;
import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.IndexFieldDataService;
@ -65,17 +68,21 @@ import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
/** /**
* *
*/ */
public class IndexService extends AbstractIndexComponent implements IndexComponent, Iterable<IndexShard> { public class IndexService extends AbstractIndexComponent implements IndexComponent, Iterable<IndexShard>{
private final IndexEventListener eventListener; private final IndexEventListener eventListener;
private final AnalysisService analysisService; private final AnalysisService analysisService;
private final IndexFieldDataService indexFieldData; private final IndexFieldDataService indexFieldData;
private final BitsetFilterCache bitsetFilterCache; private final BitsetFilterCache bitsetFilterCache;
private final NodeEnvironment nodeEnv; private final NodeEnvironment nodeEnv;
private final IndicesService indicesServices; private final ShardStoreDeleter shardStoreDeleter;
private final IndexServicesProvider indexServicesProvider; private final NodeServicesProvider nodeServicesProvider;
private final IndexStore indexStore; private final IndexStore indexStore;
private final IndexSearcherWrapper searcherWrapper; private final IndexSearcherWrapper searcherWrapper;
private final IndexCache indexCache;
private final MapperService mapperService;
private final SimilarityService similarityService;
private final EngineFactory engineFactory;
private volatile Map<Integer, IndexShard> shards = emptyMap(); private volatile Map<Integer, IndexShard> shards = emptyMap();
private final AtomicBoolean closed = new AtomicBoolean(false); private final AtomicBoolean closed = new AtomicBoolean(false);
private final AtomicBoolean deleted = new AtomicBoolean(false); private final AtomicBoolean deleted = new AtomicBoolean(false);
@ -83,27 +90,31 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
@Inject @Inject
public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv, public IndexService(IndexSettings indexSettings, NodeEnvironment nodeEnv,
AnalysisService analysisService, SimilarityService similarityService,
IndexFieldDataService indexFieldData, ShardStoreDeleter shardStoreDeleter,
BitsetFilterCache bitSetFilterCache, AnalysisRegistry registry,
IndicesService indicesServices, @Nullable EngineFactory engineFactory,
IndexServicesProvider indexServicesProvider, NodeServicesProvider nodeServicesProvider,
QueryCache queryCache,
IndexStore indexStore, IndexStore indexStore,
IndexEventListener eventListener, IndexEventListener eventListener,
IndexModule.IndexSearcherWrapperFactory wrapperFactory) { IndexModule.IndexSearcherWrapperFactory wrapperFactory) throws IOException {
super(indexSettings); super(indexSettings);
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.analysisService = analysisService; this.analysisService = registry.build(indexSettings);
this.indexFieldData = indexFieldData; this.similarityService = similarityService;
this.bitsetFilterCache = bitSetFilterCache; this.mapperService = new MapperService(indexSettings, analysisService, similarityService);
this.indicesServices = indicesServices; this.indexFieldData = new IndexFieldDataService(indexSettings, nodeServicesProvider.getIndicesFieldDataCache(), nodeServicesProvider.getCircuitBreakerService(), mapperService);
this.shardStoreDeleter = shardStoreDeleter;
this.eventListener = eventListener; this.eventListener = eventListener;
this.nodeEnv = nodeEnv; this.nodeEnv = nodeEnv;
this.indexServicesProvider = indexServicesProvider; this.nodeServicesProvider = nodeServicesProvider;
this.indexStore = indexStore; this.indexStore = indexStore;
indexFieldData.setListener(new FieldDataCacheListener(this)); indexFieldData.setListener(new FieldDataCacheListener(this));
bitSetFilterCache.setListener(new BitsetCacheListener(this));
this.searcherWrapper = wrapperFactory.newWrapper(this); this.searcherWrapper = wrapperFactory.newWrapper(this);
this.bitsetFilterCache = new BitsetFilterCache(indexSettings, nodeServicesProvider.getWarmer(), new BitsetCacheListener(this));
this.indexCache = new IndexCache(indexSettings, queryCache, bitsetFilterCache);
this.engineFactory = engineFactory;
} }
public int numberOfShards() { public int numberOfShards() {
@ -145,39 +156,37 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
public Set<Integer> shardIds() { return shards.keySet(); } public Set<Integer> shardIds() { return shards.keySet(); }
public IndexCache cache() { public IndexCache cache() {
return indexServicesProvider.getIndexCache(); return indexCache;
} }
public IndexFieldDataService fieldData() { public IndexFieldDataService fieldData() { return indexFieldData; }
return indexFieldData;
}
public BitsetFilterCache bitsetFilterCache() {
return bitsetFilterCache;
}
public AnalysisService analysisService() { public AnalysisService analysisService() {
return this.analysisService; return this.analysisService;
} }
public MapperService mapperService() { public MapperService mapperService() {
return indexServicesProvider.getMapperService(); return mapperService;
} }
public SimilarityService similarityService() { public SimilarityService similarityService() {
return indexServicesProvider.getSimilarityService(); return similarityService;
} }
public synchronized void close(final String reason, boolean delete) { public synchronized void close(final String reason, boolean delete) throws IOException {
if (closed.compareAndSet(false, true)) { if (closed.compareAndSet(false, true)) {
deleted.compareAndSet(false, delete); deleted.compareAndSet(false, delete);
final Set<Integer> shardIds = shardIds(); try {
for (final int shardId : shardIds) { final Set<Integer> shardIds = shardIds();
try { for (final int shardId : shardIds) {
removeShard(shardId, reason); try {
} catch (Throwable t) { removeShard(shardId, reason);
logger.warn("failed to close shard", t); } catch (Throwable t) {
logger.warn("failed to close shard", t);
}
} }
} finally {
IOUtils.close(bitsetFilterCache, indexCache, mapperService, indexFieldData, analysisService);
} }
} }
} }
@ -262,11 +271,11 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
// if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary. // if we are on a shared FS we only own the shard (ie. we can safely delete it) if we are the primary.
final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false || final boolean canDeleteShardContent = IndexMetaData.isOnSharedFilesystem(indexSettings) == false ||
(primary && IndexMetaData.isOnSharedFilesystem(indexSettings)); (primary && IndexMetaData.isOnSharedFilesystem(indexSettings));
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> indexServicesProvider.getIndicesQueryCache().onClose(shardId))); store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock, new StoreCloseListener(shardId, canDeleteShardContent, () -> nodeServicesProvider.getIndicesQueryCache().onClose(shardId)));
if (useShadowEngine(primary, indexSettings)) { if (useShadowEngine(primary, indexSettings)) {
indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, searcherWrapper, indexServicesProvider); indexShard = new ShadowIndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider);
} else { } else {
indexShard = new IndexShard(shardId, this.indexSettings, path, store, searcherWrapper, indexServicesProvider); indexShard = new IndexShard(shardId, this.indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldData, engineFactory, eventListener, searcherWrapper, nodeServicesProvider);
} }
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
@ -339,19 +348,19 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
try { try {
eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings); eventListener.beforeIndexShardDeleted(lock.getShardId(), indexSettings);
} finally { } finally {
indicesServices.deleteShardStore("delete index", lock, indexSettings); shardStoreDeleter.deleteShardStore("delete index", lock, indexSettings);
eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings); eventListener.afterIndexShardDeleted(lock.getShardId(), indexSettings);
} }
} }
} catch (IOException e) { } catch (IOException e) {
indicesServices.addPendingDelete(lock.getShardId(), indexSettings); shardStoreDeleter.addPendingDelete(lock.getShardId(), indexSettings);
logger.debug("[{}] failed to delete shard content - scheduled a retry", e, lock.getShardId().id()); logger.debug("[{}] failed to delete shard content - scheduled a retry", e, lock.getShardId().id());
} }
} }
} }
public IndexServicesProvider getIndexServices() { public NodeServicesProvider getIndexServices() {
return indexServicesProvider; return nodeServicesProvider;
} }
public IndexSettings getIndexSettings() { public IndexSettings getIndexSettings() {
@ -359,7 +368,7 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
} }
public QueryShardContext getQueryShardContext() { public QueryShardContext getQueryShardContext() {
return new QueryShardContext(indexSettings, indexServicesProvider.getClient(), bitsetFilterCache(), indexServicesProvider.getIndexFieldDataService(), mapperService(), similarityService(), indexServicesProvider.getScriptService(), indexServicesProvider.getIndicesQueriesRegistry()); return new QueryShardContext(indexSettings, nodeServicesProvider.getClient(), indexCache.bitsetFilterCache(), indexFieldData, mapperService(), similarityService(), nodeServicesProvider.getScriptService(), nodeServicesProvider.getIndicesQueriesRegistry());
} }
private class StoreCloseListener implements Store.OnClose { private class StoreCloseListener implements Store.OnClose {
@ -521,4 +530,23 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
} }
} }
} }
public interface ShardStoreDeleter {
void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException;
void addPendingDelete(ShardId shardId, Settings indexSettings);
}
final EngineFactory getEngineFactory() {
return engineFactory;
}
final IndexSearcherWrapper getSearcherWrapper() {
return searcherWrapper;
}
final IndexStore getIndexStore() {
return indexStore;
}
} }

View File

@ -23,15 +23,11 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.index.termvectors.TermVectorsService;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.cache.query.IndicesQueryCache; import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptService; import org.elasticsearch.script.ScriptService;
@ -42,58 +38,39 @@ import org.elasticsearch.threadpool.ThreadPool;
* a shard. * a shard.
* This is just a temporary solution until we cleaned up index creation and removed injectors on that level as well. * This is just a temporary solution until we cleaned up index creation and removed injectors on that level as well.
*/ */
public final class IndexServicesProvider { public final class NodeServicesProvider {
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final MapperService mapperService;
private final IndexCache indexCache;
private final IndicesQueryCache indicesQueryCache; private final IndicesQueryCache indicesQueryCache;
private final TermVectorsService termVectorsService; private final TermVectorsService termVectorsService;
private final IndexFieldDataService indexFieldDataService;
private final IndicesWarmer warmer; private final IndicesWarmer warmer;
private final SimilarityService similarityService;
private final EngineFactory factory;
private final BigArrays bigArrays; private final BigArrays bigArrays;
private final IndexingMemoryController indexingMemoryController; private final IndexingMemoryController indexingMemoryController;
private final IndexEventListener listener;
private final Client client; private final Client client;
private final IndicesQueriesRegistry indicesQueriesRegistry; private final IndicesQueriesRegistry indicesQueriesRegistry;
private final ScriptService scriptService; private final ScriptService scriptService;
private final IndicesFieldDataCache indicesFieldDataCache;
private final CircuitBreakerService circuitBreakerService;
@Inject @Inject
public IndexServicesProvider(IndexEventListener listener, ThreadPool threadPool, MapperService mapperService, IndexCache indexCache, IndicesQueryCache indicesQueryCache, TermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, @Nullable IndicesWarmer warmer, SimilarityService similarityService, EngineFactory factory, BigArrays bigArrays, IndexingMemoryController indexingMemoryController, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry) { public NodeServicesProvider(ThreadPool threadPool, IndicesQueryCache indicesQueryCache, TermVectorsService termVectorsService, @Nullable IndicesWarmer warmer, BigArrays bigArrays, IndexingMemoryController indexingMemoryController, Client client, ScriptService scriptService, IndicesQueriesRegistry indicesQueriesRegistry, IndicesFieldDataCache indicesFieldDataCache, CircuitBreakerService circuitBreakerService) {
this.listener = listener;
this.threadPool = threadPool; this.threadPool = threadPool;
this.mapperService = mapperService;
this.indexCache = indexCache;
this.indicesQueryCache = indicesQueryCache; this.indicesQueryCache = indicesQueryCache;
this.termVectorsService = termVectorsService; this.termVectorsService = termVectorsService;
this.indexFieldDataService = indexFieldDataService;
this.warmer = warmer; this.warmer = warmer;
this.similarityService = similarityService;
this.factory = factory;
this.bigArrays = bigArrays; this.bigArrays = bigArrays;
this.indexingMemoryController = indexingMemoryController; this.indexingMemoryController = indexingMemoryController;
this.client = client; this.client = client;
this.indicesQueriesRegistry = indicesQueriesRegistry; this.indicesQueriesRegistry = indicesQueriesRegistry;
this.scriptService = scriptService; this.scriptService = scriptService;
this.indicesFieldDataCache = indicesFieldDataCache;
this.circuitBreakerService = circuitBreakerService;
} }
public IndexEventListener getIndexEventListener() {
return listener;
}
public ThreadPool getThreadPool() { public ThreadPool getThreadPool() {
return threadPool; return threadPool;
} }
public MapperService getMapperService() {
return mapperService;
}
public IndexCache getIndexCache() {
return indexCache;
}
public IndicesQueryCache getIndicesQueryCache() { public IndicesQueryCache getIndicesQueryCache() {
return indicesQueryCache; return indicesQueryCache;
} }
@ -102,22 +79,10 @@ public final class IndexServicesProvider {
return termVectorsService; return termVectorsService;
} }
public IndexFieldDataService getIndexFieldDataService() {
return indexFieldDataService;
}
public IndicesWarmer getWarmer() { public IndicesWarmer getWarmer() {
return warmer; return warmer;
} }
public SimilarityService getSimilarityService() {
return similarityService;
}
public EngineFactory getFactory() {
return factory;
}
public BigArrays getBigArrays() { return bigArrays; } public BigArrays getBigArrays() { return bigArrays; }
public Client getClient() { public Client getClient() {
@ -135,4 +100,12 @@ public final class IndexServicesProvider {
public IndexingMemoryController getIndexingMemoryController() { public IndexingMemoryController getIndexingMemoryController() {
return indexingMemoryController; return indexingMemoryController;
} }
public IndicesFieldDataCache getIndicesFieldDataCache() {
return indicesFieldDataCache;
}
public CircuitBreakerService getCircuitBreakerService() {
return circuitBreakerService;
}
} }

View File

@ -69,44 +69,26 @@ import java.util.concurrent.Executor;
public final class BitsetFilterCache extends AbstractIndexComponent implements LeafReader.CoreClosedListener, RemovalListener<Object, Cache<Query, BitsetFilterCache.Value>>, Closeable { public final class BitsetFilterCache extends AbstractIndexComponent implements LeafReader.CoreClosedListener, RemovalListener<Object, Cache<Query, BitsetFilterCache.Value>>, Closeable {
public static final String LOAD_RANDOM_ACCESS_FILTERS_EAGERLY = "index.load_fixed_bitset_filters_eagerly"; public static final String LOAD_RANDOM_ACCESS_FILTERS_EAGERLY = "index.load_fixed_bitset_filters_eagerly";
private static final Listener DEFAULT_NOOP_LISTENER = new Listener() {
@Override
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
};
private final boolean loadRandomAccessFiltersEagerly; private final boolean loadRandomAccessFiltersEagerly;
private final Cache<Object, Cache<Query, Value>> loadedFilters; private final Cache<Object, Cache<Query, Value>> loadedFilters;
private volatile Listener listener = DEFAULT_NOOP_LISTENER; private final Listener listener;
private final BitSetProducerWarmer warmer; private final BitSetProducerWarmer warmer;
private final IndicesWarmer indicesWarmer; private final IndicesWarmer indicesWarmer;
public BitsetFilterCache(IndexSettings indexSettings, IndicesWarmer indicesWarmer) { public BitsetFilterCache(IndexSettings indexSettings, IndicesWarmer indicesWarmer, Listener listener) {
super(indexSettings); super(indexSettings);
if (listener == null) {
throw new IllegalArgumentException("listener must not be null");
}
this.loadRandomAccessFiltersEagerly = this.indexSettings.getSettings().getAsBoolean(LOAD_RANDOM_ACCESS_FILTERS_EAGERLY, true); this.loadRandomAccessFiltersEagerly = this.indexSettings.getSettings().getAsBoolean(LOAD_RANDOM_ACCESS_FILTERS_EAGERLY, true);
this.loadedFilters = CacheBuilder.<Object, Cache<Query, Value>>builder().removalListener(this).build(); this.loadedFilters = CacheBuilder.<Object, Cache<Query, Value>>builder().removalListener(this).build();
this.warmer = new BitSetProducerWarmer(); this.warmer = new BitSetProducerWarmer();
this.indicesWarmer = indicesWarmer; this.indicesWarmer = indicesWarmer;
indicesWarmer.addListener(warmer); indicesWarmer.addListener(warmer);
}
/**
* Sets a listener that is invoked for all subsequent cache and removal events.
* @throws IllegalStateException if the listener is set more than once
*/
public void setListener(Listener listener) {
if (listener == null) {
throw new IllegalArgumentException("listener must not be null");
}
if (this.listener != DEFAULT_NOOP_LISTENER) {
throw new IllegalStateException("can't set listener more than once");
}
this.listener = listener; this.listener = listener;
} }
public BitSetProducer getBitSetProducer(Query query) { public BitSetProducer getBitSetProducer(Query query) {

View File

@ -44,6 +44,8 @@ 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;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
@ -54,7 +56,7 @@ import static java.util.Collections.unmodifiableMap;
/** /**
*/ */
public class IndexFieldDataService extends AbstractIndexComponent { public class IndexFieldDataService extends AbstractIndexComponent implements Closeable {
public static final String FIELDDATA_CACHE_KEY = "index.fielddata.cache"; public static final String FIELDDATA_CACHE_KEY = "index.fielddata.cache";
public static final String FIELDDATA_CACHE_VALUE_NODE = "node"; public static final String FIELDDATA_CACHE_VALUE_NODE = "node";
@ -260,4 +262,8 @@ public class IndexFieldDataService extends AbstractIndexComponent {
this.listener = listener; this.listener = listener;
} }
@Override
public void close() throws IOException {
clear();
}
} }

View File

@ -43,7 +43,6 @@ import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Booleans;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.lease.Releasables; import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.ESLogger;
@ -61,7 +60,7 @@ import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.gateway.MetaDataStateFormat; import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.IndexModule; import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexServicesProvider; import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.cache.IndexCache; import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache;
@ -153,7 +152,7 @@ public class IndexShard extends AbstractIndexShardComponent {
private final IndicesQueryCache indicesQueryCache; private final IndicesQueryCache indicesQueryCache;
private final IndexEventListener indexEventListener; private final IndexEventListener indexEventListener;
private final IndexSettings idxSettings; private final IndexSettings idxSettings;
private final IndexServicesProvider provider; private final NodeServicesProvider provider;
private TimeValue refreshInterval; private TimeValue refreshInterval;
@ -201,22 +200,24 @@ public class IndexShard extends AbstractIndexShardComponent {
private final IndexingMemoryController indexingMemoryController; private final IndexingMemoryController indexingMemoryController;
@Inject public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache,
public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexSearcherWrapper indexSearcherWrapper, IndexServicesProvider provider) { MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService,
@Nullable EngineFactory engineFactory,
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, NodeServicesProvider provider) {
super(shardId, indexSettings); super(shardId, indexSettings);
this.idxSettings = indexSettings; this.idxSettings = indexSettings;
this.codecService = new CodecService(provider.getMapperService(), logger); this.codecService = new CodecService(mapperService, logger);
this.warmer = provider.getWarmer(); this.warmer = provider.getWarmer();
this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
this.similarityService = provider.getSimilarityService(); this.similarityService = similarityService;
Objects.requireNonNull(store, "Store must be provided to the index shard"); Objects.requireNonNull(store, "Store must be provided to the index shard");
this.engineFactory = provider.getFactory(); this.engineFactory = engineFactory == null ? new InternalEngineFactory() : engineFactory;
this.store = store; this.store = store;
this.indexEventListener = provider.getIndexEventListener(); this.indexEventListener = indexEventListener;
this.mergeSchedulerConfig = new MergeSchedulerConfig(this.indexSettings); this.mergeSchedulerConfig = new MergeSchedulerConfig(this.indexSettings);
this.threadPool = provider.getThreadPool(); this.threadPool = provider.getThreadPool();
this.mapperService = provider.getMapperService(); this.mapperService = mapperService;
this.indexCache = provider.getIndexCache(); 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();
@ -225,7 +226,7 @@ public class IndexShard extends AbstractIndexShardComponent {
this.indicesQueryCache = provider.getIndicesQueryCache(); this.indicesQueryCache = provider.getIndicesQueryCache();
this.shardQueryCache = new ShardRequestCache(shardId, indexSettings); this.shardQueryCache = new ShardRequestCache(shardId, indexSettings);
this.shardFieldData = new ShardFieldData(); this.shardFieldData = new ShardFieldData();
this.indexFieldDataService = provider.getIndexFieldDataService(); 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 = this.indexSettings.getAsTime(INDEX_REFRESH_INTERVAL, EngineConfig.DEFAULT_REFRESH_INTERVAL);
@ -233,7 +234,6 @@ public class IndexShard extends AbstractIndexShardComponent {
this.path = path; this.path = path;
this.mergePolicyConfig = new MergePolicyConfig(logger, this.indexSettings); this.mergePolicyConfig = new MergePolicyConfig(logger, this.indexSettings);
/* 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 = this.indexSettings.get("index.shard.check_on_startup", "false");
@ -1620,4 +1620,8 @@ public class IndexShard extends AbstractIndexShardComponent {
return queryShardContextCache.get(); return queryShardContextCache.get();
} }
EngineFactory getEngineFactory() {
return engineFactory;
}
} }

View File

@ -21,11 +21,17 @@ package org.elasticsearch.index.shard;
import java.io.IOException; import java.io.IOException;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexServicesProvider; import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.index.translog.TranslogStats;
@ -37,8 +43,9 @@ import org.elasticsearch.index.translog.TranslogStats;
*/ */
public final class ShadowIndexShard extends IndexShard { public final class ShadowIndexShard extends IndexShard {
public ShadowIndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexSearcherWrapper wrapper, IndexServicesProvider provider) throws IOException { public ShadowIndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory,
super(shardId, indexSettings, path, store, wrapper, provider); IndexEventListener indexEventListener, IndexSearcherWrapper wrapper, NodeServicesProvider provider) throws IOException {
super(shardId, indexSettings, path, store, indexCache, mapperService, similarityService, indexFieldDataService, engineFactory, indexEventListener, wrapper, provider);
} }
/** /**

View File

@ -24,6 +24,7 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.common.geo.ShapesAvailability; import org.elasticsearch.common.geo.ShapesAvailability;
import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.util.ExtensionPoint; import org.elasticsearch.common.util.ExtensionPoint;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.query.*; import org.elasticsearch.index.query.*;
import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser; import org.elasticsearch.index.query.functionscore.FunctionScoreQueryParser;
import org.elasticsearch.index.termvectors.TermVectorsService; import org.elasticsearch.index.termvectors.TermVectorsService;
@ -134,6 +135,7 @@ public class IndicesModule extends AbstractModule {
bind(MetaDataIndexUpgradeService.class).asEagerSingleton(); bind(MetaDataIndexUpgradeService.class).asEagerSingleton();
bind(IndicesFieldDataCacheListener.class).asEagerSingleton(); bind(IndicesFieldDataCacheListener.class).asEagerSingleton();
bind(TermVectorsService.class).asEagerSingleton(); bind(TermVectorsService.class).asEagerSingleton();
bind(NodeServicesProvider.class).asEagerSingleton();
} }
protected void bindQueryParsersExtension() { protected void bindQueryParsersExtension() {

View File

@ -28,7 +28,6 @@ import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag;
import org.elasticsearch.action.admin.indices.stats.IndexShardStats; import org.elasticsearch.action.admin.indices.stats.IndexShardStats;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -37,7 +36,6 @@ 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.*;
import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.regex.Regex;
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.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
@ -46,15 +44,10 @@ import org.elasticsearch.env.ShardLock;
import org.elasticsearch.gateway.MetaDataStateFormat; import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.*; import org.elasticsearch.index.*;
import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.flush.FlushStats; import org.elasticsearch.index.flush.FlushStats;
import org.elasticsearch.index.get.GetStats; import org.elasticsearch.index.get.GetStats;
import org.elasticsearch.index.indexing.IndexingStats; import org.elasticsearch.index.indexing.IndexingStats;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.index.recovery.RecoveryStats; 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;
@ -65,11 +58,9 @@ 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.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.PluginsService;
import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.util.*; import java.util.*;
@ -78,7 +69,6 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.function.Predicate; import java.util.function.Predicate;
import java.util.stream.Stream;
import static java.util.Collections.emptyMap; import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap; import static java.util.Collections.unmodifiableMap;
@ -89,46 +79,18 @@ import static org.elasticsearch.common.util.CollectionUtils.arrayAsArrayList;
/** /**
* *
*/ */
public class IndicesService extends AbstractLifecycleComponent<IndicesService> implements Iterable<IndexService> { public class IndicesService extends AbstractLifecycleComponent<IndicesService> implements Iterable<IndexService>, IndexService.ShardStoreDeleter {
public static final String INDICES_SHARDS_CLOSED_TIMEOUT = "indices.shards_closed_timeout"; public static final String INDICES_SHARDS_CLOSED_TIMEOUT = "indices.shards_closed_timeout";
private final Injector injector;
private final PluginsService pluginsService; private final PluginsService pluginsService;
private final NodeEnvironment nodeEnv; private final NodeEnvironment nodeEnv;
private final TimeValue shardsClosedTimeout; private final TimeValue shardsClosedTimeout;
private final IndicesWarmer indicesWarmer;
private final IndicesQueryCache indicesQueryCache;
private final AnalysisRegistry analysisRegistry; private final AnalysisRegistry analysisRegistry;
private final IndicesQueriesRegistry indicesQueriesRegistry; private final IndicesQueriesRegistry indicesQueriesRegistry;
private final ClusterService clusterService; private final ClusterService clusterService;
private final IndexNameExpressionResolver indexNameExpressionResolver; private final IndexNameExpressionResolver indexNameExpressionResolver;
private volatile Map<String, IndexService> indices = emptyMap();
private volatile Map<String, IndexServiceInjectorPair> indices = emptyMap();
public AnalysisRegistry getAnalysis() {
return analysisRegistry;
}
static class IndexServiceInjectorPair {
private final IndexService indexService;
private final Injector injector;
public IndexServiceInjectorPair(IndexService indexService, Injector injector) {
this.indexService = indexService;
this.injector = injector;
}
public IndexService getIndexService() {
return indexService;
}
public Injector getInjector() {
return injector;
}
}
private final Map<Index, List<PendingDelete>> pendingDeletes = new HashMap<>(); private final Map<Index, List<PendingDelete>> pendingDeletes = new HashMap<>();
private final OldShardsStats oldShardsStats = new OldShardsStats(); private final OldShardsStats oldShardsStats = new OldShardsStats();
private final IndexStoreConfig indexStoreConfig; private final IndexStoreConfig indexStoreConfig;
@ -137,13 +99,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
} }
@Inject @Inject
public IndicesService(Settings settings, Injector injector, PluginsService pluginsService, NodeEnvironment nodeEnv, NodeSettingsService nodeSettingsService, IndicesQueryCache indicesQueryCache, IndicesWarmer indicesWarmer, AnalysisRegistry analysisRegistry, IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService) { public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvironment nodeEnv, NodeSettingsService nodeSettingsService, AnalysisRegistry analysisRegistry, IndicesQueriesRegistry indicesQueriesRegistry, IndexNameExpressionResolver indexNameExpressionResolver, ClusterService clusterService) {
super(settings); super(settings);
this.injector = injector;
this.pluginsService = pluginsService; this.pluginsService = pluginsService;
this.nodeEnv = nodeEnv; this.nodeEnv = nodeEnv;
this.indicesWarmer = indicesWarmer;
this.indicesQueryCache = indicesQueryCache;
this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS)); this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS));
this.indexStoreConfig = new IndexStoreConfig(settings); this.indexStoreConfig = new IndexStoreConfig(settings);
this.analysisRegistry = analysisRegistry; this.analysisRegistry = analysisRegistry;
@ -184,8 +143,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
@Override @Override
protected void doClose() { protected void doClose() {
IOUtils.closeWhileHandlingException(injector.getInstance(RecoverySettings.class), IOUtils.closeWhileHandlingException(analysisRegistry);
analysisRegistry);
} }
/** /**
@ -230,8 +188,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
} }
Map<Index, List<IndexShardStats>> statsByShard = new HashMap<>(); Map<Index, List<IndexShardStats>> statsByShard = new HashMap<>();
for (IndexServiceInjectorPair value : indices.values()) { for (IndexService indexService : indices.values()) {
IndexService indexService = value.getIndexService();
for (IndexShard indexShard : indexService) { for (IndexShard indexShard : indexService) {
try { try {
if (indexShard.routingEntry() == null) { if (indexShard.routingEntry() == null) {
@ -262,7 +219,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
@Override @Override
public Iterator<IndexService> iterator() { public Iterator<IndexService> iterator() {
return indices.values().stream().map((p) -> p.getIndexService()).iterator(); return indices.values().iterator();
} }
public boolean hasIndex(String index) { public boolean hasIndex(String index) {
@ -275,12 +232,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
*/ */
@Nullable @Nullable
public IndexService indexService(String index) { public IndexService indexService(String index) {
IndexServiceInjectorPair indexServiceInjectorPair = indices.get(index); return indices.get(index);
if (indexServiceInjectorPair == null) {
return null;
} else {
return indexServiceInjectorPair.getIndexService();
}
} }
/** /**
@ -302,7 +254,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the per-index listeners * @param builtInListeners a list of built-in lifecycle {@link IndexEventListener} that should should be used along side with the per-index listeners
* @throws IndexAlreadyExistsException if the index already exists. * @throws IndexAlreadyExistsException if the index already exists.
*/ */
public synchronized IndexService createIndex(IndexMetaData indexMetaData, List<IndexEventListener> builtInListeners) { public synchronized IndexService createIndex(final NodeServicesProvider nodeServicesProvider, IndexMetaData indexMetaData, List<IndexEventListener> builtInListeners) throws IOException {
if (!lifecycle.started()) { if (!lifecycle.started()) {
throw new IllegalStateException("Can't create an index [" + indexMetaData.getIndex() + "], node is closed"); throw new IllegalStateException("Can't create an index [" + indexMetaData.getIndex() + "], node is closed");
} }
@ -319,36 +271,29 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
idxSettings.getNumberOfReplicas(), idxSettings.getNumberOfReplicas(),
idxSettings.isShadowReplicaIndex() ? "s" : ""); idxSettings.isShadowReplicaIndex() ? "s" : "");
ModulesBuilder modules = new ModulesBuilder();
// plugin modules must be added here, before others or we can get crazy injection errors... // plugin modules must be added here, before others or we can get crazy injection errors...
for (Module pluginModule : pluginsService.indexModules(idxSettings.getSettings())) { final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, analysisRegistry);
modules.add(pluginModule); pluginsService.onIndexModule(indexModule);
}
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, indicesQueryCache, indicesWarmer, analysisRegistry);
for (IndexEventListener listener : builtInListeners) { for (IndexEventListener listener : builtInListeners) {
indexModule.addIndexEventListener(listener); indexModule.addIndexEventListener(listener);
} }
indexModule.addIndexEventListener(oldShardsStats); indexModule.addIndexEventListener(oldShardsStats);
modules.add(indexModule);
pluginsService.processModules(modules);
final IndexEventListener listener = indexModule.freeze(); final IndexEventListener listener = indexModule.freeze();
listener.beforeIndexCreated(index, idxSettings.getSettings()); listener.beforeIndexCreated(index, idxSettings.getSettings());
final IndexService indexService = indexModule.newIndexService(nodeEnv, this, nodeServicesProvider);
Injector indexInjector; boolean success = false;
try { try {
indexInjector = modules.createChildInjector(injector); assert indexService.getIndexEventListener() == listener;
} catch (CreationException e) { listener.afterIndexCreated(indexService);
throw new IndexCreationException(index, Injectors.getFirstErrorFailure(e)); indices = newMapBuilder(indices).put(index.name(), indexService).immutableMap();
} catch (Throwable e) { success = true;
throw new IndexCreationException(index, e); return indexService;
} finally {
if (success == false) {
indexService.close("plugins_failed", true);
}
} }
IndexService indexService = indexInjector.getInstance(IndexService.class);
assert indexService.getIndexEventListener() == listener;
listener.afterIndexCreated(indexService);
indices = newMapBuilder(indices).put(index.name(), new IndexServiceInjectorPair(indexService, indexInjector)).immutableMap();
return indexService;
} }
/** /**
@ -364,7 +309,6 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private void removeIndex(String index, String reason, boolean delete) { private void removeIndex(String index, String reason, boolean delete) {
try { try {
final IndexService indexService; final IndexService indexService;
final Injector indexInjector;
final IndexEventListener listener; final IndexEventListener listener;
synchronized (this) { synchronized (this) {
if (indices.containsKey(index) == false) { if (indices.containsKey(index) == false) {
@ -372,10 +316,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
} }
logger.debug("[{}] closing ... (reason [{}])", index, reason); logger.debug("[{}] closing ... (reason [{}])", index, reason);
Map<String, IndexServiceInjectorPair> newIndices = new HashMap<>(indices); Map<String, IndexService> newIndices = new HashMap<>(indices);
IndexServiceInjectorPair remove = newIndices.remove(index); indexService = newIndices.remove(index);
indexService = remove.getIndexService();
indexInjector = remove.getInjector();
indices = unmodifiableMap(newIndices); indices = unmodifiableMap(newIndices);
listener = indexService.getIndexEventListener(); listener = indexService.getIndexEventListener();
} }
@ -384,21 +326,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
if (delete) { if (delete) {
listener.beforeIndexDeleted(indexService); listener.beforeIndexDeleted(indexService);
} }
Stream<Closeable> closeables = pluginsService.indexServices().stream().map(p -> indexInjector.getInstance(p));
IOUtils.close(closeables::iterator);
logger.debug("[{}] closing index service (reason [{}])", index, reason); logger.debug("[{}] closing index service (reason [{}])", index, reason);
indexService.close(reason, delete); indexService.close(reason, delete);
logger.debug("[{}] closing index cache (reason [{}])", index, reason);
indexInjector.getInstance(IndexCache.class).close();
logger.debug("[{}] clearing index field data (reason [{}])", index, reason);
indexInjector.getInstance(IndexFieldDataService.class).clear();
logger.debug("[{}] closing analysis service (reason [{}])", index, reason);
indexInjector.getInstance(AnalysisService.class).close();
logger.debug("[{}] closing mapper service (reason [{}])", index, reason);
indexInjector.getInstance(MapperService.class).close();
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) {
@ -473,7 +402,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
synchronized (this) { synchronized (this) {
String indexName = metaData.getIndex(); String indexName = metaData.getIndex();
if (indices.containsKey(indexName)) { if (indices.containsKey(indexName)) {
String localUUid = indices.get(indexName).getIndexService().indexUUID(); String localUUid = indices.get(indexName).indexUUID();
throw new IllegalStateException("Can't delete index store for [" + indexName + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]"); throw new IllegalStateException("Can't delete index store for [" + indexName + "] - it's still part of the indices service [" + localUUid + "] [" + metaData.getIndexUUID() + "]");
} }
if (clusterState.metaData().hasIndex(indexName) && (clusterState.nodes().localNode().masterNode() == true)) { if (clusterState.metaData().hasIndex(indexName) && (clusterState.nodes().localNode().masterNode() == true)) {
@ -574,11 +503,11 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
* @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, Settings indexSettings, boolean closed) {
final IndexServiceInjectorPair indexServiceInjectorPair = 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 (IndexMetaData.isOnSharedFilesystem(indexSettings) == false || closed) {
if (indexServiceInjectorPair == null && nodeEnv.hasNodeFile()) { if (indexService == null && nodeEnv.hasNodeFile()) {
return true; return true;
} }
} else { } else {
@ -609,10 +538,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
} }
private boolean canDeleteShardContent(ShardId shardId, Settings indexSettings) { private boolean canDeleteShardContent(ShardId shardId, Settings indexSettings) {
final IndexServiceInjectorPair indexServiceInjectorPair = this.indices.get(shardId.getIndex()); final IndexService indexService = this.indices.get(shardId.getIndex());
if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) { if (IndexMetaData.isOnSharedFilesystem(indexSettings) == false) {
if (indexServiceInjectorPair != null && nodeEnv.hasNodeFile()) { if (indexService != null && nodeEnv.hasNodeFile()) {
final IndexService indexService = indexServiceInjectorPair.getIndexService();
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 (NodeEnvironment.hasCustomDataPath(indexSettings)) {
@ -805,4 +733,9 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
public IndicesQueriesRegistry getIndicesQueryRegistry() { public IndicesQueriesRegistry getIndicesQueryRegistry() {
return indicesQueriesRegistry; return indicesQueriesRegistry;
} }
public AnalysisRegistry getAnalysis() {
return analysisRegistry;
}
} }

View File

@ -46,6 +46,7 @@ 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.IndexShardAlreadyExistsException; import org.elasticsearch.index.IndexShardAlreadyExistsException;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.DocumentMapper;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.shard.*; import org.elasticsearch.index.shard.*;
@ -76,6 +77,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
private final ShardStateAction shardStateAction; private final ShardStateAction shardStateAction;
private final NodeIndexDeletedAction nodeIndexDeletedAction; private final NodeIndexDeletedAction nodeIndexDeletedAction;
private final NodeMappingRefreshAction nodeMappingRefreshAction; private final NodeMappingRefreshAction nodeMappingRefreshAction;
private final NodeServicesProvider nodeServicesProvider;
private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new NoOpShardStateActionListener(); private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new NoOpShardStateActionListener();
@ -110,7 +112,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
ThreadPool threadPool, RecoveryTarget recoveryTarget, ThreadPool threadPool, RecoveryTarget recoveryTarget,
ShardStateAction shardStateAction, ShardStateAction shardStateAction,
NodeIndexDeletedAction nodeIndexDeletedAction, NodeIndexDeletedAction nodeIndexDeletedAction,
NodeMappingRefreshAction nodeMappingRefreshAction, RepositoriesService repositoriesService, RestoreService restoreService, SearchService searchService, SyncedFlushService syncedFlushService, RecoverySource recoverySource) { NodeMappingRefreshAction nodeMappingRefreshAction, RepositoriesService repositoriesService, RestoreService restoreService, SearchService searchService, SyncedFlushService syncedFlushService, RecoverySource recoverySource, NodeServicesProvider nodeServicesProvider) {
super(settings); super(settings);
this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService); this.buildInIndexListener = Arrays.asList(recoverySource, recoveryTarget, searchService, syncedFlushService);
this.indicesService = indicesService; this.indicesService = indicesService;
@ -123,6 +125,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
this.restoreService = restoreService; this.restoreService = restoreService;
this.repositoriesService = repositoriesService; this.repositoriesService = repositoriesService;
this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true); this.sendRefreshMapping = this.settings.getAsBoolean("indices.cluster.send_refresh_mapping", true);
this.nodeServicesProvider = nodeServicesProvider;
} }
@Override @Override
@ -302,7 +305,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
logger.debug("[{}] creating index", indexMetaData.getIndex()); logger.debug("[{}] creating index", indexMetaData.getIndex());
} }
try { try {
indicesService.createIndex(indexMetaData, buildInIndexListener); indicesService.createIndex(nodeServicesProvider, indexMetaData, buildInIndexListener);
} catch (Throwable e) { } catch (Throwable e) {
sendFailShard(shard, indexMetaData.getIndexUUID(), "failed to create index", e); sendFailShard(shard, indexMetaData.getIndexUUID(), "failed to create index", e);
} }

View File

@ -63,6 +63,7 @@ import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.cluster.IndicesClusterStateService; import org.elasticsearch.indices.cluster.IndicesClusterStateService;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.indices.memory.IndexingMemoryController; import org.elasticsearch.indices.memory.IndexingMemoryController;
import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.store.IndicesStore; import org.elasticsearch.indices.store.IndicesStore;
import org.elasticsearch.indices.ttl.IndicesTTLService; import org.elasticsearch.indices.ttl.IndicesTTLService;
import org.elasticsearch.monitor.MonitorModule; import org.elasticsearch.monitor.MonitorModule;
@ -308,6 +309,7 @@ public class Node implements Releasable {
for (Class<? extends LifecycleComponent> plugin : pluginsService.nodeServices()) { for (Class<? extends LifecycleComponent> plugin : pluginsService.nodeServices()) {
injector.getInstance(plugin).stop(); injector.getInstance(plugin).stop();
} }
injector.getInstance(RecoverySettings.class).close();
// we should stop this last since it waits for resources to get released // we should stop this last since it waits for resources to get released
// if we had scroll searchers etc or recovery going on we wait for to finish. // if we had scroll searchers etc or recovery going on we wait for to finish.
injector.getInstance(IndicesService.class).stop(); injector.getInstance(IndicesService.class).stop();

View File

@ -444,7 +444,7 @@ public class PercolateContext extends SearchContext {
@Override @Override
public BitsetFilterCache bitsetFilterCache() { public BitsetFilterCache bitsetFilterCache() {
return indexService.bitsetFilterCache(); return indexService.cache().bitsetFilterCache();
} }
@Override @Override

View File

@ -22,10 +22,13 @@ package org.elasticsearch.plugins;
import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexService;
import java.io.Closeable; import java.io.Closeable;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List;
/** /**
* An extension point allowing to plug in custom functionality. * An extension point allowing to plug in custom functionality.
@ -59,20 +62,6 @@ public abstract class Plugin {
return Collections.emptyList(); return Collections.emptyList();
} }
/**
* Per index modules.
*/
public Collection<Module> indexModules(Settings indexSettings) {
return Collections.emptyList();
}
/**
* Per index services that will be automatically closed.
*/
public Collection<Class<? extends Closeable>> indexServices() {
return Collections.emptyList();
}
/** /**
* Additional node settings loaded by the plugin. Note that settings that are explicit in the nodes settings can't be * Additional node settings loaded by the plugin. Note that settings that are explicit in the nodes settings can't be
* overwritten with the additional settings. These settings added if they don't exist. * overwritten with the additional settings. These settings added if they don't exist.
@ -80,4 +69,12 @@ public abstract class Plugin {
public Settings additionalSettings() { public Settings additionalSettings() {
return Settings.Builder.EMPTY_SETTINGS; return Settings.Builder.EMPTY_SETTINGS;
} }
public List<Closeable> indexService(IndexService indexService) {
return Collections.EMPTY_LIST;
}
public void onIndexModule(IndexModule indexModule) {}
public final void onModule(IndexModule indexModule) {}
} }

View File

@ -25,6 +25,7 @@ import org.apache.lucene.analysis.util.TokenizerFactory;
import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat; import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.PostingsFormat; import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.admin.cluster.node.info.PluginsInfo; import org.elasticsearch.action.admin.cluster.node.info.PluginsInfo;
import org.elasticsearch.bootstrap.JarHell; import org.elasticsearch.bootstrap.JarHell;
@ -37,6 +38,10 @@ import org.elasticsearch.common.io.FileSystemUtils;
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.IndexModule;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -239,22 +244,26 @@ public class PluginsService extends AbstractComponent {
return services; return services;
} }
public Collection<Module> indexModules(Settings indexSettings) { public void onIndexModule(IndexModule indexModule) {
List<Module> modules = new ArrayList<>();
for (Tuple<PluginInfo, Plugin> plugin : plugins) { for (Tuple<PluginInfo, Plugin> plugin : plugins) {
modules.addAll(plugin.v2().indexModules(indexSettings)); plugin.v2().onIndexModule(indexModule);
} }
return modules; indexModule.addIndexEventListener(new IndexEventListener() {
} final List<Closeable> closeables = new ArrayList<>();
@Override
public void afterIndexCreated(IndexService indexService) {
for (Tuple<PluginInfo, Plugin> plugin : plugins) {
List<Closeable> services = plugin.v2().indexService(indexService);
closeables.addAll(services);
}
}
public Collection<Class<? extends Closeable>> indexServices() { @Override
List<Class<? extends Closeable>> services = new ArrayList<>(); public void beforeIndexClosed(IndexService indexService) {
for (Tuple<PluginInfo, Plugin> plugin : plugins) { IOUtils.closeWhileHandlingException(closeables);
services.addAll(plugin.v2().indexServices()); }
} });
return services;
} }
/** /**
* Get information about plugins (jvm and site plugins). * Get information about plugins (jvm and site plugins).
*/ */

View File

@ -450,7 +450,7 @@ public class DefaultSearchContext extends SearchContext {
@Override @Override
public BitsetFilterCache bitsetFilterCache() { public BitsetFilterCache bitsetFilterCache() {
return indexService.bitsetFilterCache(); return indexService.cache().bitsetFilterCache();
} }
@Override @Override

View File

@ -77,8 +77,8 @@ public class MetaDataIndexTemplateServiceTests extends ESTestCase {
Version.CURRENT, Version.CURRENT,
null, null,
new HashSet<>(), new HashSet<>(),
null null,
); null);
MetaDataIndexTemplateService service = new MetaDataIndexTemplateService(Settings.EMPTY, null, createIndexService, null); MetaDataIndexTemplateService service = new MetaDataIndexTemplateService(Settings.EMPTY, null, createIndexService, null);
final List<Throwable> throwables = new ArrayList<>(); final List<Throwable> throwables = new ArrayList<>();

View File

@ -63,7 +63,7 @@ public class SettingsFilteringIT extends ESIntegTestCase {
} }
@Override @Override
public Collection<Module> indexModules(Settings indexSettings) { public Collection<Module> nodeModules() {
return Collections.<Module>singletonList(new SettingsFilteringModule()); return Collections.<Module>singletonList(new SettingsFilteringModule());
} }
} }

View File

@ -18,6 +18,7 @@
*/ */
package org.elasticsearch.index; package org.elasticsearch.index;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.DirectoryReader; import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.FieldInvertState; import org.apache.lucene.index.FieldInvertState;
import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.LeafReaderContext;
@ -25,10 +26,16 @@ import org.apache.lucene.search.*;
import org.apache.lucene.search.similarities.BM25Similarity; import org.apache.lucene.search.similarities.BM25Similarity;
import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.Similarity;
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.ModuleTestCase; import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
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.env.Environment;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.analysis.AnalysisRegistry; 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;
@ -39,57 +46,119 @@ import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.engine.InternalEngineFactory;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexSearcherWrapper; import org.elasticsearch.index.shard.IndexSearcherWrapper;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.similarity.SimilarityProvider; import org.elasticsearch.index.similarity.SimilarityProvider;
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.IndexStoreConfig; import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.IndicesWarmer; 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.query.IndicesQueriesRegistry;
import org.elasticsearch.script.ScriptContextRegistry;
import org.elasticsearch.script.ScriptEngineService;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.script.mustache.MustacheScriptEngineService;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.test.engine.MockEngineFactory; import org.elasticsearch.test.engine.MockEngineFactory;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.watcher.ResourceWatcherService;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer; import java.util.function.Consumer;
public class IndexModuleTests extends ModuleTestCase { public class IndexModuleTests extends ModuleTestCase {
private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null); private Index index;
public void testWrapperIsBound() { private Settings settings;
final Index index = new Index("foo"); private IndexSettings indexSettings;
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build(); private Environment environment;
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST); private NodeEnvironment nodeEnvironment;
IndexModule module = new IndexModule(indexSettings, null, null, warmer, new AnalysisRegistry(null, new Environment(settings))); private NodeServicesProvider nodeServicesProvider;
assertInstanceBinding(module, IndexModule.IndexSearcherWrapperFactory.class, (x) -> x.newWrapper(null) == null); private IndexService.ShardStoreDeleter deleter = new IndexService.ShardStoreDeleter() {
module.setSearcherWrapper((s) -> new Wrapper()); @Override
assertInstanceBinding(module, IndexModule.IndexSearcherWrapperFactory.class, (x) -> x.newWrapper(null) instanceof Wrapper); public void deleteShardStore(String reason, ShardLock lock, Settings indexSettings) throws IOException {
}
@Override
public void addPendingDelete(ShardId shardId, Settings indexSettings) {
}
};
static NodeServicesProvider newNodeServiceProvider(Settings settings, Environment environment, Client client, ScriptEngineService... scriptEngineServices) throws IOException {
// TODO this can be used in other place too - lets first refactor the IndicesQueriesRegistry
ThreadPool threadPool = new ThreadPool("test");
IndicesWarmer warmer = new IndicesWarmer(settings, threadPool);
IndicesQueryCache indicesQueryCache = new IndicesQueryCache(settings);
CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService();
PageCacheRecycler recycler = new PageCacheRecycler(settings, threadPool);
BigArrays bigArrays = new BigArrays(recycler, circuitBreakerService);
IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndicesFieldDataCacheListener(circuitBreakerService), threadPool);
Set<ScriptEngineService> scriptEngines = new HashSet<>();
scriptEngines.add(new MustacheScriptEngineService(settings));
scriptEngines.addAll(Arrays.asList(scriptEngineServices));
ScriptService scriptService = new ScriptService(settings, environment, scriptEngines, new ResourceWatcherService(settings, threadPool), new ScriptContextRegistry(Collections.EMPTY_LIST));
IndicesQueriesRegistry indicesQueriesRegistry = new IndicesQueriesRegistry(settings, Collections.EMPTY_SET, new NamedWriteableRegistry());
return new NodeServicesProvider(threadPool, indicesQueryCache, null, warmer, bigArrays, null, client, scriptService, indicesQueriesRegistry, indicesFieldDataCache, circuitBreakerService);
} }
public void testEngineFactoryBound() { @Override
final Index index = new Index("foo"); public void setUp() throws Exception {
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build(); super.setUp();
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST); index = new Index("foo");
IndexModule module = new IndexModule(indexSettings, null, null, warmer, new AnalysisRegistry(null, new Environment(settings))); settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build();
assertBinding(module, EngineFactory.class, InternalEngineFactory.class); indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
module.engineFactoryImpl = MockEngineFactory.class; environment = new Environment(settings);
assertBinding(module, EngineFactory.class, MockEngineFactory.class); nodeServicesProvider = newNodeServiceProvider(settings, environment, null);
nodeEnvironment = new NodeEnvironment(settings, environment);
} }
public void testRegisterIndexStore() { @Override
public void tearDown() throws Exception {
super.tearDown();
nodeEnvironment.close();
nodeServicesProvider.getThreadPool().shutdown();
if (nodeServicesProvider.getThreadPool().awaitTermination(10, TimeUnit.SECONDS) == false) {
nodeServicesProvider.getThreadPool().shutdownNow();
}
}
public void testWrapperIsBound() throws IOException {
IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
module.setSearcherWrapper((s) -> new Wrapper());
module.engineFactory.set(new MockEngineFactory(AssertingDirectoryReader.class));
IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.getSearcherWrapper() instanceof Wrapper);
assertSame(indexService.getEngineFactory(), module.engineFactory.get());
indexService.close("simon says", false);
}
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, Collections.EMPTY_LIST);
IndexModule module = new IndexModule(indexSettings, null, null, warmer, new AnalysisRegistry(null, new Environment(settings))); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
module.addIndexStore("foo_store", FooStore::new); module.addIndexStore("foo_store", FooStore::new);
assertInstanceBinding(module, IndexStore.class, (x) -> x.getClass() == FooStore.class); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.getIndexStore() instanceof FooStore);
try { try {
module.addIndexStore("foo_store", FooStore::new); module.addIndexStore("foo_store", FooStore::new);
fail("already registered"); fail("already registered");
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
// fine // fine
} }
indexService.close("simon says", false);
} }
public void testOtherServiceBound() { public void testOtherServiceBound() throws IOException {
final AtomicBoolean atomicBoolean = new AtomicBoolean(false); final AtomicBoolean atomicBoolean = new AtomicBoolean(false);
final IndexEventListener eventListener = new IndexEventListener() { final IndexEventListener eventListener = new IndexEventListener() {
@Override @Override
@ -97,31 +166,24 @@ public class IndexModuleTests extends ModuleTestCase {
atomicBoolean.set(true); atomicBoolean.set(true);
} }
}; };
final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build();
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
IndexModule module = new IndexModule(indexSettings, null, null, warmer, new AnalysisRegistry(null, new Environment(settings))); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
Consumer<Settings> listener = (s) -> {}; Consumer<Settings> listener = (s) -> {};
module.addIndexSettingsListener(listener); module.addIndexSettingsListener(listener);
module.addIndexEventListener(eventListener); module.addIndexEventListener(eventListener);
assertBinding(module, IndexService.class, IndexService.class); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertBinding(module, IndexServicesProvider.class, IndexServicesProvider.class); IndexSettings x = indexService.getIndexSettings();
assertInstanceBinding(module, IndexEventListener.class, (x) -> { assertEquals(x.getSettings().getAsMap(), indexSettings.getSettings().getAsMap());
x.beforeIndexDeleted(null); assertEquals(x.getIndex(), index);
return atomicBoolean.get(); assertSame(x.getUpdateListeners().get(0), listener);
}); indexService.getIndexEventListener().beforeIndexDeleted(null);
assertInstanceBinding(module, IndexSettings.class, (x) -> x.getSettings().getAsMap().equals(indexSettings.getSettings().getAsMap())); assertTrue(atomicBoolean.get());
assertInstanceBinding(module, IndexSettings.class, (x) -> x.getIndex().equals(indexSettings.getIndex())); indexService.close("simon says", false);
assertInstanceBinding(module, IndexSettings.class, (x) -> x.getUpdateListeners().get(0) == listener);
assertInstanceBinding(module, IndexStore.class, (x) -> x.getClass() == IndexStore.class);
} }
public void testListener() { public void testListener() throws IOException {
final Index index = new Index("foo"); IndexModule module = new IndexModule(indexSettings, null, new AnalysisRegistry(null, environment));
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).put("path.home", createTempDir().toString()).build();
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
IndexModule module = new IndexModule(indexSettings, null, null, warmer, new AnalysisRegistry(null, new Environment(settings)));
Consumer<Settings> listener = (s) -> { Consumer<Settings> listener = (s) -> {
}; };
module.addIndexSettingsListener(listener); module.addIndexSettingsListener(listener);
@ -139,18 +201,21 @@ public class IndexModuleTests extends ModuleTestCase {
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
} }
assertInstanceBinding(module, IndexSettings.class, (x) -> x.getUpdateListeners().size() == 1); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertInstanceBinding(module, IndexSettings.class, (x) -> x.getUpdateListeners().get(0) == listener); IndexSettings x = indexService.getIndexSettings();
assertEquals(1, x.getUpdateListeners().size());
assertSame(x.getUpdateListeners().get(0), listener);
indexService.close("simon says", false);
} }
public void testAddSimilarity() { public void testAddSimilarity() throws IOException {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put("index.similarity.my_similarity.type", "test_similarity") .put("index.similarity.my_similarity.type", "test_similarity")
.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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), 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() {
@ -162,42 +227,39 @@ public class IndexModuleTests extends ModuleTestCase {
return new TestSimilarity(settings.get("key")); return new TestSimilarity(settings.get("key"));
} }
}); });
assertInstanceBinding(module, SimilarityService.class, (inst) -> {
if (inst instanceof SimilarityService) { IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertNotNull(inst.getSimilarity("my_similarity")); SimilarityService similarityService = indexService.similarityService();
assertTrue(inst.getSimilarity("my_similarity").get() instanceof TestSimilarity); assertNotNull(similarityService.getSimilarity("my_similarity"));
assertEquals("my_similarity", inst.getSimilarity("my_similarity").name()); assertTrue(similarityService.getSimilarity("my_similarity").get() instanceof TestSimilarity);
assertEquals("there is a key" , ((TestSimilarity)inst.getSimilarity("my_similarity").get()).key); assertEquals("my_similarity", similarityService.getSimilarity("my_similarity").name());
return true; assertEquals("there is a key", ((TestSimilarity) similarityService.getSimilarity("my_similarity").get()).key);
} indexService.close("simon says", false);
return false;
});
} }
public void testSetupUnknownSimilarity() { public void testSetupUnknownSimilarity() throws IOException {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put("index.similarity.my_similarity.type", "test_similarity") .put("index.similarity.my_similarity.type", "test_similarity")
.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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
try { try {
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService); module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage()); assertEquals("Unknown Similarity type [test_similarity] for [my_similarity]", ex.getMessage());
} }
} }
public void testSetupWithoutType() throws IOException {
public void testSetupWithoutType() {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.put("index.similarity.my_similarity.foo", "bar") .put("index.similarity.my_similarity.foo", "bar")
.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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null,new AnalysisRegistry(null, environment));
try { try {
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService); module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage()); assertEquals("Similarity [my_similarity] must have an associated type", ex.getMessage());
} }
@ -207,7 +269,7 @@ public class IndexModuleTests extends ModuleTestCase {
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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
try { try {
module.registerQueryCache("index", IndexQueryCache::new); module.registerQueryCache("index", IndexQueryCache::new);
fail("only once"); fail("only once");
@ -230,12 +292,12 @@ public class IndexModuleTests extends ModuleTestCase {
} }
} }
public void testRegisterCustomQueryCache() { public void testRegisterCustomQueryCache() throws IOException {
Settings indexSettings = Settings.settingsBuilder() Settings indexSettings = Settings.settingsBuilder()
.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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), 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());
@ -243,15 +305,20 @@ public class IndexModuleTests extends ModuleTestCase {
} catch (IllegalArgumentException e) { } catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]"); assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]");
} }
assertInstanceBinding(module, QueryCache.class, (x) -> x instanceof CustomQueryCache);
IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.cache().query() instanceof CustomQueryCache);
indexService.close("simon says", false);
} }
public void testDefaultQueryCacheImplIsSelected() { public void testDefaultQueryCacheImplIsSelected() throws IOException {
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, null, warmer, new AnalysisRegistry(null, new Environment(indexSettings))); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, new AnalysisRegistry(null, environment));
assertInstanceBinding(module, QueryCache.class, (x) -> x instanceof IndexQueryCache); IndexService indexService = module.newIndexService(nodeEnvironment, deleter, nodeServicesProvider);
assertTrue(indexService.cache().query() instanceof IndexQueryCache);
indexService.close("simon says", false);
} }
class CustomQueryCache implements QueryCache { class CustomQueryCache implements QueryCache {
@ -275,8 +342,6 @@ public class IndexModuleTests extends ModuleTestCase {
} }
} }
private static class TestSimilarity extends Similarity { private static class TestSimilarity extends Similarity {
private final Similarity delegate = new BM25Similarity(); private final Similarity delegate = new BM25Similarity();
private final String key; private final String key;
@ -305,6 +370,15 @@ public class IndexModuleTests extends ModuleTestCase {
} }
} }
public static final class FooStore extends IndexStore {
public FooStore(IndexSettings indexSettings, IndexStoreConfig config) {
super(indexSettings, config);
}
}
public static final class Wrapper extends IndexSearcherWrapper { public static final class Wrapper extends IndexSearcherWrapper {
@Override @Override
@ -318,11 +392,4 @@ public class IndexModuleTests extends ModuleTestCase {
} }
} }
public static final class FooStore extends IndexStore {
public FooStore(IndexSettings indexSettings, IndexStoreConfig config) {
super(indexSettings, config);
}
}
} }

View File

@ -65,7 +65,8 @@ public class SettingsListenerIT extends ESIntegTestCase {
clusterModule.registerIndexDynamicSetting("index.test.new.setting", Validator.INTEGER); clusterModule.registerIndexDynamicSetting("index.test.new.setting", Validator.INTEGER);
} }
public void onModule(IndexModule module) { @Override
public void onIndexModule(IndexModule module) {
if (module.getIndex().getName().equals("test")) { // only for the test index if (module.getIndex().getName().equals("test")) { // only for the test index
module.addIndexSettingsListener(service); module.addIndexSettingsListener(service);
service.accept(module.getSettings()); service.accept(module.getSettings());

View File

@ -93,7 +93,17 @@ public class BitSetFilterCacheTests extends ESTestCase {
IndexReader reader = DirectoryReader.open(writer, false); IndexReader reader = DirectoryReader.open(writer, false);
IndexSearcher searcher = new IndexSearcher(reader); IndexSearcher searcher = new IndexSearcher(reader);
BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer); BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() {
@Override
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
});
BitSetProducer filter = cache.getBitSetProducer(new TermQuery(new Term("field", "value"))); BitSetProducer filter = cache.getBitSetProducer(new TermQuery(new Term("field", "value")));
assertThat(matchCount(filter, reader), equalTo(3)); assertThat(matchCount(filter, reader), equalTo(3));
@ -136,8 +146,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
final AtomicInteger onCacheCalls = new AtomicInteger(); final AtomicInteger onCacheCalls = new AtomicInteger();
final AtomicInteger onRemoveCalls = new AtomicInteger(); final AtomicInteger onRemoveCalls = new AtomicInteger();
final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer); final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer, new BitsetFilterCache.Listener() {
cache.setListener(new BitsetFilterCache.Listener() {
@Override @Override
public void onCache(ShardId shardId, Accountable accountable) { public void onCache(ShardId shardId, Accountable accountable) {
onCacheCalls.incrementAndGet(); onCacheCalls.incrementAndGet();
@ -174,35 +183,11 @@ public class BitSetFilterCacheTests extends ESTestCase {
assertEquals(0, stats.get()); assertEquals(0, stats.get());
} }
public void testSetListenerTwice() { public void testSetNullListener() {
final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer);
cache.setListener(new BitsetFilterCache.Listener() {
@Override
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
});
try { try {
cache.setListener(new BitsetFilterCache.Listener() { new BitsetFilterCache(INDEX_SETTINGS, warmer, null);
fail("listener can't be null");
@Override } catch (IllegalArgumentException ex) {
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
});
fail("can't set it twice");
} catch (IllegalStateException ex) {
// all is well // all is well
} }
} }

View File

@ -135,7 +135,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
} }
protected Nested createNested(IndexSearcher searcher, Query parentFilter, Query childFilter) throws IOException { protected Nested createNested(IndexSearcher searcher, Query parentFilter, Query childFilter) throws IOException {
BitsetFilterCache s = indexService.bitsetFilterCache(); BitsetFilterCache s = indexService.cache().bitsetFilterCache();
return new Nested(s.getBitSetProducer(parentFilter), searcher.createNormalizedWeight(childFilter, false)); return new Nested(s.getBitSetProducer(parentFilter), searcher.createNormalizedWeight(childFilter, false));
} }

View File

@ -1,33 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.externalvalues;
import org.elasticsearch.common.inject.AbstractModule;
/**
*
*/
public class ExternalIndexModule extends AbstractModule {
@Override
protected void configure() {
bind(RegisterExternalTypes.class).asEagerSingleton();
}
}

View File

@ -19,14 +19,19 @@
package org.elasticsearch.index.mapper.externalvalues; package org.elasticsearch.index.mapper.externalvalues;
import org.elasticsearch.common.inject.Module; import org.elasticsearch.index.IndexService;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import java.util.Collection; import java.io.Closeable;
import java.util.Collections; import java.util.List;
public class ExternalMapperPlugin extends Plugin { public class ExternalMapperPlugin extends Plugin {
public static final String EXTERNAL = "external";
public static final String EXTERNAL_BIS = "external_bis";
public static final String EXTERNAL_UPPER = "external_upper";
@Override @Override
public String name() { public String name() {
return "external-mappers"; return "external-mappers";
@ -38,7 +43,12 @@ public class ExternalMapperPlugin extends Plugin {
} }
@Override @Override
public Collection<Module> indexModules(Settings indexSettings) { public List<Closeable> indexService(IndexService indexService) {
return Collections.<Module>singletonList(new ExternalIndexModule()); final MapperService mapperService = indexService.mapperService();
mapperService.documentMapperParser().putRootTypeParser(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser());
mapperService.documentMapperParser().putTypeParser(EXTERNAL, new ExternalMapper.TypeParser(EXTERNAL, "foo"));
mapperService.documentMapperParser().putTypeParser(EXTERNAL_BIS, new ExternalMapper.TypeParser(EXTERNAL_BIS, "bar"));
mapperService.documentMapperParser().putTypeParser(EXTERNAL_UPPER, new ExternalMapper.TypeParser(EXTERNAL_UPPER, "FOO BAR"));
return super.indexService(indexService);
} }
} }

View File

@ -43,7 +43,7 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
.startObject(ExternalMetadataMapper.CONTENT_TYPE) .startObject(ExternalMetadataMapper.CONTENT_TYPE)
.endObject() .endObject()
.startObject("properties") .startObject("properties")
.startObject("field").field("type", RegisterExternalTypes.EXTERNAL).endObject() .startObject("field").field("type", ExternalMapperPlugin.EXTERNAL).endObject()
.endObject() .endObject()
.endObject().endObject()).execute().get(); .endObject().endObject()).execute().get();
ensureYellow("test-idx"); ensureYellow("test-idx");
@ -85,7 +85,7 @@ public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
prepareCreate("test-idx").addMapping("doc", prepareCreate("test-idx").addMapping("doc",
XContentFactory.jsonBuilder().startObject().startObject("doc").startObject("properties") XContentFactory.jsonBuilder().startObject().startObject("doc").startObject("properties")
.startObject("f") .startObject("f")
.field("type", RegisterExternalTypes.EXTERNAL_UPPER) .field("type", ExternalMapperPlugin.EXTERNAL_UPPER)
.startObject("fields") .startObject("fields")
.startObject("f") .startObject("f")
.field("type", "string") .field("type", "string")

View File

@ -1,41 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.externalvalues;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
public class RegisterExternalTypes extends AbstractIndexComponent {
public static final String EXTERNAL = "external";
public static final String EXTERNAL_BIS = "external_bis";
public static final String EXTERNAL_UPPER = "external_upper";
@Inject
public RegisterExternalTypes(IndexSettings indexSettings, MapperService mapperService) {
super(indexSettings);
mapperService.documentMapperParser().putRootTypeParser(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser());
mapperService.documentMapperParser().putTypeParser(EXTERNAL, new ExternalMapper.TypeParser(EXTERNAL, "foo"));
mapperService.documentMapperParser().putTypeParser(EXTERNAL_BIS, new ExternalMapper.TypeParser(EXTERNAL_BIS, "bar"));
mapperService.documentMapperParser().putTypeParser(EXTERNAL_UPPER, new ExternalMapper.TypeParser(EXTERNAL_UPPER, "FOO BAR"));
}
}

View File

@ -35,8 +35,8 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
MapperService mapperService = createIndex("test").mapperService(); MapperService mapperService = createIndex("test").mapperService();
mapperService.documentMapperParser().putRootTypeParser(ExternalMetadataMapper.CONTENT_TYPE, mapperService.documentMapperParser().putRootTypeParser(ExternalMetadataMapper.CONTENT_TYPE,
new ExternalMetadataMapper.TypeParser()); new ExternalMetadataMapper.TypeParser());
mapperService.documentMapperParser().putTypeParser(RegisterExternalTypes.EXTERNAL, mapperService.documentMapperParser().putTypeParser(ExternalMapperPlugin.EXTERNAL,
new ExternalMapper.TypeParser(RegisterExternalTypes.EXTERNAL, "foo")); new ExternalMapper.TypeParser(ExternalMapperPlugin.EXTERNAL, "foo"));
DocumentMapper documentMapper = mapperService.documentMapperParser().parse( DocumentMapper documentMapper = mapperService.documentMapperParser().parse(
XContentFactory.jsonBuilder().startObject().startObject("type") XContentFactory.jsonBuilder().startObject().startObject("type")
@ -71,13 +71,13 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
public void testExternalValuesWithMultifield() throws Exception { public void testExternalValuesWithMultifield() throws Exception {
MapperService mapperService = createIndex("test").mapperService(); MapperService mapperService = createIndex("test").mapperService();
mapperService.documentMapperParser().putTypeParser(RegisterExternalTypes.EXTERNAL, mapperService.documentMapperParser().putTypeParser(ExternalMapperPlugin.EXTERNAL,
new ExternalMapper.TypeParser(RegisterExternalTypes.EXTERNAL, "foo")); new ExternalMapper.TypeParser(ExternalMapperPlugin.EXTERNAL, "foo"));
DocumentMapper documentMapper = mapperService.documentMapperParser().parse( DocumentMapper documentMapper = mapperService.documentMapperParser().parse(
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("field") .startObject("field")
.field("type", RegisterExternalTypes.EXTERNAL) .field("type", ExternalMapperPlugin.EXTERNAL)
.startObject("fields") .startObject("fields")
.startObject("field") .startObject("field")
.field("type", "string") .field("type", "string")
@ -119,21 +119,21 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
public void testExternalValuesWithMultifieldTwoLevels() throws Exception { public void testExternalValuesWithMultifieldTwoLevels() throws Exception {
MapperService mapperService = createIndex("test").mapperService(); MapperService mapperService = createIndex("test").mapperService();
mapperService.documentMapperParser().putTypeParser(RegisterExternalTypes.EXTERNAL, mapperService.documentMapperParser().putTypeParser(ExternalMapperPlugin.EXTERNAL,
new ExternalMapper.TypeParser(RegisterExternalTypes.EXTERNAL, "foo")); new ExternalMapper.TypeParser(ExternalMapperPlugin.EXTERNAL, "foo"));
mapperService.documentMapperParser().putTypeParser(RegisterExternalTypes.EXTERNAL_BIS, mapperService.documentMapperParser().putTypeParser(ExternalMapperPlugin.EXTERNAL_BIS,
new ExternalMapper.TypeParser(RegisterExternalTypes.EXTERNAL_BIS, "bar")); new ExternalMapper.TypeParser(ExternalMapperPlugin.EXTERNAL_BIS, "bar"));
DocumentMapper documentMapper = mapperService.documentMapperParser().parse( DocumentMapper documentMapper = mapperService.documentMapperParser().parse(
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties") XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")
.startObject("field") .startObject("field")
.field("type", RegisterExternalTypes.EXTERNAL) .field("type", ExternalMapperPlugin.EXTERNAL)
.startObject("fields") .startObject("fields")
.startObject("field") .startObject("field")
.field("type", "string") .field("type", "string")
.startObject("fields") .startObject("fields")
.startObject("generated") .startObject("generated")
.field("type", RegisterExternalTypes.EXTERNAL_BIS) .field("type", ExternalMapperPlugin.EXTERNAL_BIS)
.endObject() .endObject()
.startObject("raw") .startObject("raw")
.field("type", "string") .field("type", "string")

View File

@ -24,6 +24,7 @@ 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.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
@ -73,6 +74,7 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParserMapper; import org.elasticsearch.index.query.functionscore.ScoreFunctionParserMapper;
import org.elasticsearch.index.query.support.QueryParsers; import org.elasticsearch.index.query.support.QueryParsers;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
@ -233,9 +235,6 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
} }
SimilarityService service = new SimilarityService(idxSettings, Collections.emptyMap()); SimilarityService service = new SimilarityService(idxSettings, Collections.emptyMap());
bind(SimilarityService.class).toInstance(service); bind(SimilarityService.class).toInstance(service);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null));
bind(BitsetFilterCache.class).toInstance(bitsetFilterCache);
bind(IndexCache.class).toInstance(new IndexCache(idxSettings, new NoneQueryCache(idxSettings), bitsetFilterCache));
bind(Client.class).toInstance(proxy); bind(Client.class).toInstance(proxy);
Multibinder.newSetBinder(binder(), ScoreFunctionParser.class); Multibinder.newSetBinder(binder(), ScoreFunctionParser.class);
bind(ScoreFunctionParserMapper.class).asEagerSingleton(); bind(ScoreFunctionParserMapper.class).asEagerSingleton();
@ -249,7 +248,17 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
indexFieldDataService = injector.getInstance(IndexFieldDataService.class); indexFieldDataService = injector.getInstance(IndexFieldDataService.class);
ScriptService scriptService = injector.getInstance(ScriptService.class); ScriptService scriptService = injector.getInstance(ScriptService.class);
MapperService mapperService = injector.getInstance(MapperService.class); MapperService mapperService = injector.getInstance(MapperService.class);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null)); BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() {
@Override
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
});
indicesQueriesRegistry = injector.getInstance(IndicesQueriesRegistry.class); indicesQueriesRegistry = injector.getInstance(IndicesQueriesRegistry.class);
queryShardContext = new QueryShardContext(idxSettings, proxy, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, scriptService, indicesQueriesRegistry); queryShardContext = new QueryShardContext(idxSettings, proxy, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, scriptService, indicesQueriesRegistry);
//create some random type with some default field, those types will stick around for all of the subclasses //create some random type with some default field, those types will stick around for all of the subclasses

View File

@ -20,6 +20,7 @@ package org.elasticsearch.index.query;
import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
@ -47,6 +48,7 @@ import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.IndexFieldDataService;
import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.indices.IndicesModule; import org.elasticsearch.indices.IndicesModule;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
@ -114,9 +116,6 @@ public class TemplateQueryParserTests extends ESTestCase {
throw new ElasticsearchException(e); throw new ElasticsearchException(e);
} }
SimilarityService service = new SimilarityService(idxSettings, Collections.EMPTY_MAP); SimilarityService service = new SimilarityService(idxSettings, Collections.EMPTY_MAP);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null));
bind(BitsetFilterCache.class).toInstance(bitsetFilterCache);
bind(IndexCache.class).toInstance(new IndexCache(idxSettings, new NoneQueryCache(idxSettings), bitsetFilterCache));
bind(SimilarityService.class).toInstance(service); bind(SimilarityService.class).toInstance(service);
bind(Client.class).toInstance(proxy); // not needed here bind(Client.class).toInstance(proxy); // not needed here
Multibinder.newSetBinder(binder(), ScoreFunctionParser.class); Multibinder.newSetBinder(binder(), ScoreFunctionParser.class);
@ -129,7 +128,17 @@ public class TemplateQueryParserTests extends ESTestCase {
IndexFieldDataService indexFieldDataService = injector.getInstance(IndexFieldDataService.class); IndexFieldDataService indexFieldDataService = injector.getInstance(IndexFieldDataService.class);
ScriptService scriptService = injector.getInstance(ScriptService.class); ScriptService scriptService = injector.getInstance(ScriptService.class);
MapperService mapperService = injector.getInstance(MapperService.class); MapperService mapperService = injector.getInstance(MapperService.class);
BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null)); BitsetFilterCache bitsetFilterCache = new BitsetFilterCache(idxSettings, new IndicesWarmer(idxSettings.getNodeSettings(), null), new BitsetFilterCache.Listener() {
@Override
public void onCache(ShardId shardId, Accountable accountable) {
}
@Override
public void onRemoval(ShardId shardId, Accountable accountable) {
}
});
IndicesQueriesRegistry indicesQueriesRegistry = injector.getInstance(IndicesQueriesRegistry.class); IndicesQueriesRegistry indicesQueriesRegistry = injector.getInstance(IndicesQueriesRegistry.class);
context = new QueryShardContext(idxSettings, proxy, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, scriptService, indicesQueriesRegistry); context = new QueryShardContext(idxSettings, proxy, bitsetFilterCache, indexFieldDataService, mapperService, similarityService, scriptService, indicesQueriesRegistry);
} }

View File

@ -60,7 +60,7 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock; import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexServicesProvider; 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.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
@ -1014,8 +1014,8 @@ public class IndexShardTests extends ESSingleNodeTestCase {
private final IndexShard reinitWithWrapper(IndexService indexService, IndexShard shard, IndexSearcherWrapper wrapper) throws IOException { private final IndexShard reinitWithWrapper(IndexService indexService, IndexShard shard, IndexSearcherWrapper wrapper) throws IOException {
ShardRouting routing = new ShardRouting(shard.routingEntry()); ShardRouting routing = new ShardRouting(shard.routingEntry());
shard.close("simon says", true); shard.close("simon says", true);
IndexServicesProvider indexServices = indexService.getIndexServices(); NodeServicesProvider indexServices = indexService.getIndexServices();
IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), wrapper, indexServices); IndexShard newShard = new IndexShard(shard.shardId(), indexService.getIndexSettings(), shard.shardPath(), shard.store(), indexService.cache(), indexService.mapperService(), indexService.similarityService(), indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, indexServices);
ShardRoutingHelper.reinit(routing); ShardRoutingHelper.reinit(routing);
newShard.updateRoutingEntry(routing, false); newShard.updateRoutingEntry(routing, false);
DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT); DiscoveryNode localNode = new DiscoveryNode("foo", DummyTransportAddress.INSTANCE, Version.CURRENT);

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.shard.IndexEventListener; import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
@ -90,7 +91,8 @@ public class IndicesLifecycleListenerSingleNodeTests extends ESSingleNodeTestCas
}; };
indicesService.deleteIndex("test", "simon says"); indicesService.deleteIndex("test", "simon says");
try { try {
IndexService index = indicesService.createIndex(metaData, Arrays.asList(countingListener)); NodeServicesProvider nodeServicesProvider = getInstanceFromNode(NodeServicesProvider.class);
IndexService index = indicesService.createIndex(nodeServicesProvider, metaData, Arrays.asList(countingListener));
ShardRouting newRouting = new ShardRouting(shardRouting); ShardRouting newRouting = new ShardRouting(shardRouting);
String nodeId = newRouting.currentNodeId(); String nodeId = newRouting.currentNodeId();
ShardRoutingHelper.moveToUnassigned(newRouting, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "boom")); ShardRoutingHelper.moveToUnassigned(newRouting, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "boom"));

View File

@ -33,13 +33,13 @@ import org.elasticsearch.common.breaker.CircuitBreaker;
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.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.MockEngineFactoryPlugin;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache; import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper; import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper;
import java.io.IOException; import java.io.IOException;
@ -212,8 +212,9 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
public String description() { public String description() {
return "a mock reader wrapper that throws random exceptions for testing"; return "a mock reader wrapper that throws random exceptions for testing";
} }
public void onModule(MockEngineSupportModule module) {
module.wrapperImpl = RandomExceptionDirectoryReaderWrapper.class; public void onModule(MockEngineFactoryPlugin.MockEngineReaderModule module) {
module.setReaderClass(RandomExceptionDirectoryReaderWrapper.class);
} }
} }

View File

@ -32,12 +32,12 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.settings.Settings.Builder;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.MockEngineFactoryPlugin;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper; import org.elasticsearch.test.engine.ThrowingLeafReaderWrapper;
import java.io.IOException; import java.io.IOException;
@ -159,8 +159,8 @@ public class SearchWithRandomExceptionsIT extends ESIntegTestCase {
public String description() { public String description() {
return "a mock reader wrapper that throws random exceptions for testing"; return "a mock reader wrapper that throws random exceptions for testing";
} }
public void onModule(MockEngineSupportModule module) { public void onModule(MockEngineFactoryPlugin.MockEngineReaderModule module) {
module.wrapperImpl = RandomExceptionDirectoryReaderWrapper.class; module.setReaderClass(RandomExceptionDirectoryReaderWrapper.class);
} }
} }

View File

@ -65,12 +65,6 @@ public class JvmExamplePlugin extends Plugin {
return services; return services;
} }
@Override
public Collection<Module> indexModules(Settings indexSettings) { return Collections.emptyList();}
@Override
public Collection<Class<? extends Closeable>> indexServices() { return Collections.emptyList();}
@Override @Override
public Settings additionalSettings() { public Settings additionalSettings() {
return Settings.EMPTY; return Settings.EMPTY;

View File

@ -1,35 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.murmur3;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
public class RegisterMurmur3FieldMapper extends AbstractIndexComponent {
@Inject
public RegisterMurmur3FieldMapper(IndexSettings indexSettings, MapperService mapperService) {
super(indexSettings);
mapperService.documentMapperParser().putTypeParser(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser());
}
}

View File

@ -1,31 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.plugin.mapper;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.index.mapper.murmur3.RegisterMurmur3FieldMapper;
public class MapperMurmur3IndexModule extends AbstractModule {
@Override
protected void configure() {
bind(RegisterMurmur3FieldMapper.class).asEagerSingleton();
}
}

View File

@ -21,10 +21,14 @@ package org.elasticsearch.plugin.mapper;
import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.murmur3.Murmur3FieldMapper;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import java.io.Closeable;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List;
public class MapperMurmur3Plugin extends Plugin { public class MapperMurmur3Plugin extends Plugin {
@ -39,8 +43,9 @@ public class MapperMurmur3Plugin extends Plugin {
} }
@Override @Override
public Collection<Module> indexModules(Settings settings) { public List<Closeable> indexService(IndexService indexService) {
return Collections.<Module>singletonList(new MapperMurmur3IndexModule()); indexService.mapperService().documentMapperParser().putTypeParser(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser());
return super.indexService(indexService);
} }
} }

View File

@ -1,35 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.index.mapper.size;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
public class RegisterSizeFieldMapper extends AbstractIndexComponent {
@Inject
public RegisterSizeFieldMapper(IndexSettings indexSettings, MapperService mapperService) {
super(indexSettings);
mapperService.documentMapperParser().putRootTypeParser(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser());
}
}

View File

@ -1,32 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.plugin.mapper;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.index.mapper.size.RegisterSizeFieldMapper;
public class MapperSizeIndexModule extends AbstractModule {
@Override
protected void configure() {
bind(RegisterSizeFieldMapper.class).asEagerSingleton();
}
}

View File

@ -21,10 +21,14 @@ package org.elasticsearch.plugin.mapper;
import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.size.SizeFieldMapper;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import java.io.Closeable;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List;
public class MapperSizePlugin extends Plugin { public class MapperSizePlugin extends Plugin {
@ -39,8 +43,10 @@ public class MapperSizePlugin extends Plugin {
} }
@Override @Override
public Collection<Module> indexModules(Settings indexSettings) { public List<Closeable> indexService(IndexService indexService) {
return Collections.<Module>singletonList(new MapperSizeIndexModule()); indexService.mapperService().documentMapperParser().putRootTypeParser(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser());
return super.indexService(indexService);
} }
} }

View File

@ -18,12 +18,12 @@
*/ */
package org.elasticsearch.index; package org.elasticsearch.index;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.engine.MockEngineFactory; import org.elasticsearch.test.engine.MockEngineFactory;
import org.elasticsearch.test.engine.MockEngineSupportModule;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
@ -38,11 +38,27 @@ public class MockEngineFactoryPlugin extends Plugin {
public String description() { public String description() {
return "a mock engine factory for testing"; return "a mock engine factory for testing";
} }
private Class<? extends FilterDirectoryReader> readerWrapper = AssertingDirectoryReader.class;
@Override @Override
public Collection<Module> indexModules(Settings indexSettings) { public void onIndexModule(IndexModule module) {
return Collections.<Module>singletonList(new MockEngineSupportModule()); module.engineFactory.set(new MockEngineFactory(readerWrapper));
} }
public void onModule(IndexModule module) {
module.engineFactoryImpl = MockEngineFactory.class; @Override
public Collection<Module> nodeModules() {
return Collections.singleton(new MockEngineReaderModule());
}
public class MockEngineReaderModule extends AbstractModule {
public void setReaderClass(Class<? extends FilterDirectoryReader> readerWrapper) {
MockEngineFactoryPlugin.this.readerWrapper = readerWrapper;
}
@Override
protected void configure() {
}
} }
} }

View File

@ -19,14 +19,32 @@
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.util.Collection; import java.io.IOException;
import java.util.Collections; import java.util.*;
import java.util.function.Consumer; import java.util.function.Consumer;
public class IndexSettingsModule extends AbstractModule { public class IndexSettingsModule extends AbstractModule {

View File

@ -58,7 +58,8 @@ public final class MockIndexEventListener {
return "a mock index listener for testing only"; return "a mock index listener for testing only";
} }
public void onModule(IndexModule module) { @Override
public void onIndexModule(IndexModule module) {
module.addIndexEventListener(listener); module.addIndexEventListener(listener);
} }

View File

@ -104,7 +104,7 @@ public class TestSearchContext extends SearchContext {
this.bigArrays = bigArrays.withCircuitBreaking(); this.bigArrays = bigArrays.withCircuitBreaking();
this.indexService = indexService; this.indexService = indexService;
this.indexFieldDataService = indexService.fieldData(); this.indexFieldDataService = indexService.fieldData();
this.fixedBitSetFilterCache = indexService.bitsetFilterCache(); this.fixedBitSetFilterCache = indexService.cache().bitsetFilterCache();
this.threadPool = threadPool; this.threadPool = threadPool;
this.indexShard = indexService.getShardOrNull(0); this.indexShard = indexService.getShardOrNull(0);
this.scriptService = scriptService; this.scriptService = scriptService;

View File

@ -33,16 +33,10 @@ import static java.lang.annotation.ElementType.PARAMETER;
import static java.lang.annotation.RetentionPolicy.RUNTIME; import static java.lang.annotation.RetentionPolicy.RUNTIME;
public final class MockEngineFactory implements EngineFactory { public final class MockEngineFactory implements EngineFactory {
@BindingAnnotation
@Target({FIELD, PARAMETER})
@Retention(RUNTIME)
public @interface MockReaderType {
}
private Class<? extends FilterDirectoryReader> wrapper; private final Class<? extends FilterDirectoryReader> wrapper;
@Inject public MockEngineFactory(Class<? extends FilterDirectoryReader> wrapper) {
public MockEngineFactory(@MockReaderType Class wrapper) {
this.wrapper = wrapper; this.wrapper = wrapper;
} }

View File

@ -1,32 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.test.engine;
import org.apache.lucene.index.AssertingDirectoryReader;
import org.apache.lucene.index.FilterDirectoryReader;
import org.elasticsearch.common.inject.AbstractModule;
public class MockEngineSupportModule extends AbstractModule {
public Class<? extends FilterDirectoryReader> wrapperImpl = AssertingDirectoryReader.class;
@Override
protected void configure() {
bind(Class.class).annotatedWith(MockEngineFactory.MockReaderType.class).toInstance(wrapperImpl);
}
}

View File

@ -52,13 +52,14 @@ public class MockFSIndexStore extends IndexStore {
return Settings.builder().put(IndexModule.STORE_TYPE, "mock").build(); return Settings.builder().put(IndexModule.STORE_TYPE, "mock").build();
} }
public void onModule(IndexModule module) { @Override
Settings indexSettings = module.getSettings(); public void onIndexModule(IndexModule indexModule) {
Settings indexSettings = indexModule.getSettings();
if ("mock".equals(indexSettings.get(IndexModule.STORE_TYPE))) { if ("mock".equals(indexSettings.get(IndexModule.STORE_TYPE))) {
if (indexSettings.getAsBoolean(CHECK_INDEX_ON_CLOSE, true)) { if (indexSettings.getAsBoolean(CHECK_INDEX_ON_CLOSE, true)) {
module.addIndexEventListener(new Listener()); indexModule.addIndexEventListener(new Listener());
} }
module.addIndexStore("mock", MockFSIndexStore::new); indexModule.addIndexStore("mock", MockFSIndexStore::new);
} }
} }
} }