Merge pull request #14293 from s1monw/fold_in_index_cache_module

Fold IndexCacheModule into IndexModule
This commit is contained in:
Simon Willnauer 2015-10-27 11:45:40 +01:00
commit 59a71b0441
20 changed files with 329 additions and 397 deletions

View File

@ -22,6 +22,12 @@ package org.elasticsearch.index;
import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.inject.util.Providers; import org.elasticsearch.common.inject.util.Providers;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.ExtensionPoint;
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.index.IndexQueryCache;
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.engine.InternalEngineFactory;
import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.fielddata.IndexFieldDataService;
@ -33,6 +39,8 @@ 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.cache.query.IndicesQueryCache;
import java.util.*; import java.util.*;
import java.util.function.BiFunction; import java.util.function.BiFunction;
@ -54,8 +62,14 @@ public class IndexModule extends AbstractModule {
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";
public static final String INDEX_QUERY_CACHE = "index";
public static final String NONE_QUERY_CACHE = "none";
public static final String QUERY_CACHE_TYPE = "index.queries.cache.type";
// for test purposes only
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;
// pkg private so tests can mock // pkg private so tests can mock
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class; Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
Class<? extends IndexSearcherWrapper> indexSearcherWrapper = null; Class<? extends IndexSearcherWrapper> indexSearcherWrapper = null;
@ -64,11 +78,17 @@ public class IndexModule extends AbstractModule {
private IndexEventListener listener; private IndexEventListener listener;
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 IndicesWarmer indicesWarmer;
public IndexModule(IndexSettings indexSettings, IndexStoreConfig indexStoreConfig) { public IndexModule(IndexSettings indexSettings, IndexStoreConfig indexStoreConfig, IndicesQueryCache indicesQueryCache, IndicesWarmer warmer) {
this.indexStoreConfig = indexStoreConfig; this.indexStoreConfig = indexStoreConfig;
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.indicesQueryCache = indicesQueryCache;
this.indicesWarmer = warmer;
registerQueryCache(INDEX_QUERY_CACHE, IndexQueryCache::new);
registerQueryCache(NONE_QUERY_CACHE, (a, b) -> new NoneQueryCache(a));
} }
/** /**
@ -155,6 +175,20 @@ public class IndexModule extends AbstractModule {
similarities.put(name, similarity); similarities.put(name, similarity);
} }
/**
* Registers a {@link QueryCache} provider for a given name
* @param name the providers / caches name
* @param provider the provider instance
*/
void registerQueryCache(String name, BiFunction<IndexSettings, IndicesQueryCache, QueryCache> provider) { // pkg private - no need to expose this
if (provider == null) {
throw new IllegalArgumentException("provider must not be null");
}
if (queryCaches.containsKey(name)) {
throw new IllegalArgumentException("Can't register the same [query_cache] more than once for [" + name + "]");
}
queryCaches.put(name, provider);
}
public IndexEventListener freeze() { public IndexEventListener freeze() {
// TODO somehow we need to make this pkg private... // TODO somehow we need to make this pkg private...
@ -203,6 +237,15 @@ public class IndexModule extends AbstractModule {
throw new IllegalStateException("store must not be 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(IndexStore.class).toInstance(store);
bind(SimilarityService.class).toInstance(new SimilarityService(settings, similarities)); bind(SimilarityService.class).toInstance(new SimilarityService(settings, similarities));
} }

View File

@ -20,7 +20,6 @@
package org.elasticsearch.index.cache; package org.elasticsearch.index.cache;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache; import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
@ -37,7 +36,6 @@ public class IndexCache extends AbstractIndexComponent implements Closeable {
private final QueryCache queryCache; private final QueryCache queryCache;
private final BitsetFilterCache bitsetFilterCache; private final BitsetFilterCache bitsetFilterCache;
@Inject
public IndexCache(IndexSettings indexSettings, QueryCache queryCache, BitsetFilterCache bitsetFilterCache) { public IndexCache(IndexSettings indexSettings, QueryCache queryCache, BitsetFilterCache bitsetFilterCache) {
super(indexSettings); super(indexSettings);
this.queryCache = queryCache; this.queryCache = queryCache;

View File

@ -1,59 +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.cache;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.ExtensionPoint;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
public class IndexCacheModule extends AbstractModule {
public static final String INDEX_QUERY_CACHE = "index";
public static final String NONE_QUERY_CACHE = "none";
public static final String QUERY_CACHE_TYPE = "index.queries.cache.type";
// for test purposes only
public static final String QUERY_CACHE_EVERYTHING = "index.queries.cache.everything";
private final Settings indexSettings;
private final ExtensionPoint.SelectedType<QueryCache> queryCaches;
public IndexCacheModule(Settings settings) {
this.indexSettings = settings;
this.queryCaches = new ExtensionPoint.SelectedType<>("query_cache", QueryCache.class);
registerQueryCache(INDEX_QUERY_CACHE, IndexQueryCache.class);
registerQueryCache(NONE_QUERY_CACHE, NoneQueryCache.class);
}
public void registerQueryCache(String name, Class<? extends QueryCache> clazz) {
queryCaches.registerExtension(name, clazz);
}
@Override
protected void configure() {
queryCaches.bindType(binder(), indexSettings, QUERY_CACHE_TYPE, INDEX_QUERY_CACHE);
bind(BitsetFilterCache.class).asEagerSingleton();
bind(IndexCache.class).asEagerSingleton();
}
}

View File

@ -32,16 +32,15 @@ import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BitDocIdSet; import org.apache.lucene.util.BitDocIdSet;
import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSet;
import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.Cache;
import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.cache.CacheBuilder;
import org.elasticsearch.common.cache.RemovalListener; import org.elasticsearch.common.cache.RemovalListener;
import org.elasticsearch.common.cache.RemovalNotification; import org.elasticsearch.common.cache.RemovalNotification;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.AbstractIndexComponent; import org.elasticsearch.index.AbstractIndexComponent;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.engine.Engine;
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.object.ObjectMapper; import org.elasticsearch.index.mapper.object.ObjectMapper;
@ -50,7 +49,6 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardUtils; import org.elasticsearch.index.shard.ShardUtils;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.IndicesWarmer.TerminationHandle; import org.elasticsearch.indices.IndicesWarmer.TerminationHandle;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
@ -68,7 +66,7 @@ import java.util.concurrent.Executor;
* and require that it should always be around should use this cache, otherwise the * and require that it should always be around should use this cache, otherwise the
* {@link org.elasticsearch.index.cache.query.QueryCache} should be used instead. * {@link org.elasticsearch.index.cache.query.QueryCache} should be used instead.
*/ */
public 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() { private static final Listener DEFAULT_NOOP_LISTENER = new Listener() {
@ -85,20 +83,13 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
private final Cache<Object, Cache<Query, Value>> loadedFilters; private final Cache<Object, Cache<Query, Value>> loadedFilters;
private volatile Listener listener = DEFAULT_NOOP_LISTENER; private volatile Listener listener = DEFAULT_NOOP_LISTENER;
private final BitSetProducerWarmer warmer; private final BitSetProducerWarmer warmer;
private final IndicesWarmer indicesWarmer;
private IndicesWarmer indicesWarmer; public BitsetFilterCache(IndexSettings indexSettings, IndicesWarmer indicesWarmer) {
@Inject
public BitsetFilterCache(IndexSettings indexSettings) {
super(indexSettings); super(indexSettings);
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();
}
@Inject(optional = true)
public void setIndicesWarmer(IndicesWarmer indicesWarmer) {
this.indicesWarmer = indicesWarmer; this.indicesWarmer = indicesWarmer;
indicesWarmer.addListener(warmer); indicesWarmer.addListener(warmer);
} }
@ -129,10 +120,11 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
@Override @Override
public void close() { public void close() {
if (indicesWarmer != null) { try {
indicesWarmer.removeListener(warmer); indicesWarmer.removeListener(warmer);
} finally {
clear("close");
} }
clear("close");
} }
public void clear(String reason) { public void clear(String reason) {
@ -229,10 +221,10 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
} }
} }
final class BitSetProducerWarmer extends IndicesWarmer.Listener { final class BitSetProducerWarmer implements IndicesWarmer.Listener {
@Override @Override
public IndicesWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaData indexMetaData, IndicesWarmer.WarmerContext context, ThreadPool threadPool) { public IndicesWarmer.TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
if (!loadRandomAccessFiltersEagerly) { if (!loadRandomAccessFiltersEagerly) {
return TerminationHandle.NO_WAIT; return TerminationHandle.NO_WAIT;
} }
@ -258,9 +250,9 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
warmUp.add(Queries.newNonNestedFilter()); warmUp.add(Queries.newNonNestedFilter());
} }
final Executor executor = threadPool.executor(executor()); final Executor executor = indicesWarmer.getExecutor();
final CountDownLatch latch = new CountDownLatch(context.searcher().reader().leaves().size() * warmUp.size()); final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size());
for (final LeafReaderContext ctx : context.searcher().reader().leaves()) { for (final LeafReaderContext ctx : searcher.reader().leaves()) {
for (final Query filterToWarm : warmUp) { for (final Query filterToWarm : warmUp) {
executor.execute(() -> { executor.execute(() -> {
try { try {
@ -281,7 +273,7 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
} }
@Override @Override
public TerminationHandle warmTopReader(IndexShard indexShard, IndexMetaData indexMetaData, IndicesWarmer.WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher) {
return TerminationHandle.NO_WAIT; return TerminationHandle.NO_WAIT;
} }

View File

@ -1056,4 +1056,18 @@ public abstract class Engine implements Closeable {
public long getLastWriteNanos() { public long getLastWriteNanos() {
return this.lastWriteNanos; return this.lastWriteNanos;
} }
/**
* Called for each new opened engine searcher to warm new segments
* @see EngineConfig#getWarmer()
*/
public interface Warmer {
/**
* Called once a new Searcher is opened.
* @param searcher the searcer to warm
* @param isTopLevelReader <code>true</code> iff the searcher is build from a top-level reader.
* Otherwise the searcher might be build from a leaf reader to warm in isolation
*/
void warm(Engine.Searcher searcher, boolean isTopLevelReader);
}
} }

View File

@ -61,8 +61,7 @@ public final class EngineConfig {
private final String codecName; private final String codecName;
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final ShardIndexingService indexingService; private final ShardIndexingService indexingService;
@Nullable private final Engine.Warmer warmer;
private final IndicesWarmer warmer;
private final Store store; private final Store store;
private final SnapshotDeletionPolicy deletionPolicy; private final SnapshotDeletionPolicy deletionPolicy;
private final MergePolicy mergePolicy; private final MergePolicy mergePolicy;
@ -116,7 +115,7 @@ public final class EngineConfig {
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig} * Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
*/ */
public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService, public EngineConfig(ShardId shardId, ThreadPool threadPool, ShardIndexingService indexingService,
Settings indexSettings, IndicesWarmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy, Settings indexSettings, Engine.Warmer warmer, Store store, SnapshotDeletionPolicy deletionPolicy,
MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer, MergePolicy mergePolicy, MergeSchedulerConfig mergeSchedulerConfig, Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener, Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) { TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) {
@ -124,7 +123,7 @@ public final class EngineConfig {
this.indexSettings = indexSettings; this.indexSettings = indexSettings;
this.threadPool = threadPool; this.threadPool = threadPool;
this.indexingService = indexingService; this.indexingService = indexingService;
this.warmer = warmer; this.warmer = warmer == null ? (a,b) -> {} : warmer;
this.store = store; this.store = store;
this.deletionPolicy = deletionPolicy; this.deletionPolicy = deletionPolicy;
this.mergePolicy = mergePolicy; this.mergePolicy = mergePolicy;
@ -267,11 +266,9 @@ public final class EngineConfig {
} }
/** /**
* Returns an {@link org.elasticsearch.indices.IndicesWarmer} used to warm new searchers before they are used for searching. * Returns an {@link org.elasticsearch.index.engine.Engine.Warmer} used to warm new searchers before they are used for searching.
* Note: This method might retrun <code>null</code>
*/ */
@Nullable public Engine.Warmer getWarmer() {
public IndicesWarmer getWarmer() {
return warmer; return warmer;
} }

View File

@ -79,8 +79,7 @@ public class InternalEngine extends Engine {
private volatile long lastDeleteVersionPruneTimeMSec; private volatile long lastDeleteVersionPruneTimeMSec;
private final ShardIndexingService indexingService; private final ShardIndexingService indexingService;
@Nullable private final Engine.Warmer warmer;
private final IndicesWarmer warmer;
private final Translog translog; private final Translog translog;
private final ElasticsearchConcurrentMergeScheduler mergeScheduler; private final ElasticsearchConcurrentMergeScheduler mergeScheduler;
@ -930,8 +929,7 @@ public class InternalEngine extends Engine {
assert isMergedSegment(esLeafReader); assert isMergedSegment(esLeafReader);
if (warmer != null) { if (warmer != null) {
final Engine.Searcher searcher = new Searcher("warmer", searcherFactory.newSearcher(esLeafReader, null)); final Engine.Searcher searcher = new Searcher("warmer", searcherFactory.newSearcher(esLeafReader, null));
final IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, searcher); warmer.warm(searcher, false);
warmer.warmNewReaders(context);
} }
} catch (Throwable t) { } catch (Throwable t) {
// Don't fail a merge if the warm-up failed // Don't fail a merge if the warm-up failed
@ -955,7 +953,7 @@ public class InternalEngine extends Engine {
/** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */ /** Extended SearcherFactory that warms the segments if needed when acquiring a new searcher */
final static class SearchFactory extends EngineSearcherFactory { final static class SearchFactory extends EngineSearcherFactory {
private final IndicesWarmer warmer; private final Engine.Warmer warmer;
private final ShardId shardId; private final ShardId shardId;
private final ESLogger logger; private final ESLogger logger;
private final AtomicBoolean isEngineClosed; private final AtomicBoolean isEngineClosed;
@ -1014,11 +1012,10 @@ public class InternalEngine extends Engine {
} }
if (newSearcher != null) { if (newSearcher != null) {
IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, new Searcher("new_reader_warming", newSearcher)); warmer.warm(new Searcher("new_reader_warming", newSearcher), false);
warmer.warmNewReaders(context);
} }
assert searcher.getIndexReader() instanceof ElasticsearchDirectoryReader : "this class needs an ElasticsearchDirectoryReader but got: " + searcher.getIndexReader().getClass(); assert searcher.getIndexReader() instanceof ElasticsearchDirectoryReader : "this class needs an ElasticsearchDirectoryReader but got: " + searcher.getIndexReader().getClass();
warmer.warmTopReader(new IndicesWarmer.WarmerContext(shardId, new Searcher("top_reader_warming", searcher))); warmer.warm(new Searcher("top_reader_warming", searcher), true);
} catch (Throwable e) { } catch (Throwable e) {
if (isEngineClosed.get() == false) { if (isEngineClosed.get() == false) {
logger.warn("failed to prepare/warm", e); logger.warn("failed to prepare/warm", e);

View File

@ -55,11 +55,11 @@ import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils; 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.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexServicesProvider; import org.elasticsearch.index.IndexServicesProvider;
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.IndexCacheModule;
import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache; import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache;
import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.cache.request.ShardRequestCache; import org.elasticsearch.index.cache.request.ShardRequestCache;
@ -147,6 +147,7 @@ public class IndexShard extends AbstractIndexShardComponent {
private final MergePolicyConfig mergePolicyConfig; private final MergePolicyConfig mergePolicyConfig;
private final IndicesQueryCache indicesQueryCache; private final IndicesQueryCache indicesQueryCache;
private final IndexEventListener indexEventListener; private final IndexEventListener indexEventListener;
private final IndexSettings idxSettings;
private TimeValue refreshInterval; private TimeValue refreshInterval;
@ -197,6 +198,7 @@ public class IndexShard extends AbstractIndexShardComponent {
@Inject @Inject
public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexServicesProvider provider) { public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexServicesProvider provider) {
super(shardId, indexSettings); super(shardId, indexSettings);
this.idxSettings = indexSettings;
this.codecService = provider.getCodecService(); this.codecService = provider.getCodecService();
this.warmer = provider.getWarmer(); this.warmer = provider.getWarmer();
this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()); this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
@ -234,7 +236,7 @@ public class IndexShard extends AbstractIndexShardComponent {
final QueryCachingPolicy cachingPolicy; final QueryCachingPolicy cachingPolicy;
// the query cache is a node-level thing, however we want the most popular filters // the query cache is a node-level thing, however we want the most popular filters
// to be computed on a per-shard basis // to be computed on a per-shard basis
if (this.indexSettings.getAsBoolean(IndexCacheModule.QUERY_CACHE_EVERYTHING, false)) { if (this.indexSettings.getAsBoolean(IndexModule.QUERY_CACHE_EVERYTHING, false)) {
cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE; cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
} else { } else {
cachingPolicy = new UsageTrackingQueryCachingPolicy(); cachingPolicy = new UsageTrackingQueryCachingPolicy();
@ -261,6 +263,10 @@ public class IndexShard extends AbstractIndexShardComponent {
return this.store; return this.store;
} }
public IndexSettings getIndexSettings() {
return idxSettings;
}
/** returns true if this shard supports indexing (i.e., write) operations. */ /** returns true if this shard supports indexing (i.e., write) operations. */
public boolean canIndex() { public boolean canIndex() {
return true; return true;
@ -678,7 +684,7 @@ public class IndexShard extends AbstractIndexShardComponent {
luceneVersion = segment.getVersion(); luceneVersion = segment.getVersion();
} }
} }
return luceneVersion == null ? Version.indexCreated(indexSettings).luceneVersion : luceneVersion; return luceneVersion == null ? idxSettings.getIndexVersionCreated().luceneVersion : luceneVersion;
} }
/** /**
@ -1452,8 +1458,9 @@ public class IndexShard extends AbstractIndexShardComponent {
recoveryState.getTranslog().incrementRecoveredOperations(); recoveryState.getTranslog().incrementRecoveredOperations();
} }
}; };
final Engine.Warmer engineWarmer = (searcher, toLevel) -> warmer.warm(searcher, this, idxSettings, toLevel);
return new EngineConfig(shardId, return new EngineConfig(shardId,
threadPool, indexingService, indexSettings, warmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig, threadPool, indexingService, indexSettings, engineWarmer, store, deletionPolicy, mergePolicyConfig.getMergePolicy(), mergeSchedulerConfig,
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, indexingMemoryController.getInactiveTime()); mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, indexingMemoryController.getInactiveTime());
} }

View File

@ -44,7 +44,6 @@ import org.elasticsearch.index.*;
import org.elasticsearch.index.analysis.AnalysisModule; import org.elasticsearch.index.analysis.AnalysisModule;
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.IndexCacheModule;
import org.elasticsearch.index.fielddata.IndexFieldDataService; 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;
@ -61,6 +60,7 @@ import org.elasticsearch.index.shard.IndexEventListener;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.IndexStoreConfig; import org.elasticsearch.index.store.IndexStoreConfig;
import org.elasticsearch.indices.analysis.IndicesAnalysisService; import org.elasticsearch.indices.analysis.IndicesAnalysisService;
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
import org.elasticsearch.indices.recovery.RecoverySettings; 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;
@ -94,6 +94,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
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 volatile Map<String, IndexServiceInjectorPair> indices = emptyMap(); private volatile Map<String, IndexServiceInjectorPair> indices = emptyMap();
@ -121,12 +123,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private final IndexStoreConfig indexStoreConfig; private final IndexStoreConfig indexStoreConfig;
@Inject @Inject
public IndicesService(Settings settings, IndicesAnalysisService indicesAnalysisService, Injector injector, PluginsService pluginsService, NodeEnvironment nodeEnv, NodeSettingsService nodeSettingsService) { public IndicesService(Settings settings, IndicesAnalysisService indicesAnalysisService, Injector injector, PluginsService pluginsService, NodeEnvironment nodeEnv, NodeSettingsService nodeSettingsService, IndicesQueryCache indicesQueryCache, IndicesWarmer indicesWarmer) {
super(settings); super(settings);
this.indicesAnalysisService = indicesAnalysisService; this.indicesAnalysisService = indicesAnalysisService;
this.injector = injector; 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);
nodeSettingsService.addListener(indexStoreConfig); nodeSettingsService.addListener(indexStoreConfig);
@ -306,13 +310,12 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
for (Module pluginModule : pluginsService.indexModules(idxSettings.getSettings())) { for (Module pluginModule : pluginsService.indexModules(idxSettings.getSettings())) {
modules.add(pluginModule); modules.add(pluginModule);
} }
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig); final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, indicesQueryCache, indicesWarmer);
for (IndexEventListener listener : builtInListeners) { for (IndexEventListener listener : builtInListeners) {
indexModule.addIndexEventListener(listener); indexModule.addIndexEventListener(listener);
} }
indexModule.addIndexEventListener(oldShardsStats); indexModule.addIndexEventListener(oldShardsStats);
modules.add(new AnalysisModule(idxSettings.getSettings(), indicesAnalysisService)); modules.add(new AnalysisModule(idxSettings.getSettings(), indicesAnalysisService));
modules.add(new IndexCacheModule(idxSettings.getSettings()));
modules.add(indexModule); modules.add(indexModule);
pluginsService.processModules(modules); pluginsService.processModules(modules);
final IndexEventListener listener = indexModule.freeze(); final IndexEventListener listener = indexModule.freeze();

View File

@ -19,23 +19,21 @@
package org.elasticsearch.indices; package org.elasticsearch.indices;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
/** /**
@ -46,68 +44,46 @@ public final class IndicesWarmer extends AbstractComponent {
private final ThreadPool threadPool; private final ThreadPool threadPool;
private final ClusterService clusterService;
private final IndicesService indicesService;
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
@Inject @Inject
public IndicesWarmer(Settings settings, ThreadPool threadPool, ClusterService clusterService, IndicesService indicesService) { public IndicesWarmer(Settings settings, ThreadPool threadPool) {
super(settings); super(settings);
this.threadPool = threadPool; this.threadPool = threadPool;
this.clusterService = clusterService;
this.indicesService = indicesService;
} }
public void addListener(Listener listener) { public void addListener(Listener listener) {
listeners.add(listener); listeners.add(listener);
} }
public void removeListener(Listener listener) { public void removeListener(Listener listener) {
listeners.remove(listener); listeners.remove(listener);
} }
public void warmNewReaders(final WarmerContext context) { public void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings, boolean isTopReader) {
warmInternal(context, false); if (shard.state() == IndexShardState.CLOSED) {
}
public void warmTopReader(WarmerContext context) {
warmInternal(context, true);
}
private void warmInternal(final WarmerContext context, boolean topReader) {
final IndexMetaData indexMetaData = clusterService.state().metaData().index(context.shardId().index().name());
if (indexMetaData == null) {
return; return;
} }
if (!indexMetaData.getSettings().getAsBoolean(INDEX_WARMER_ENABLED, settings.getAsBoolean(INDEX_WARMER_ENABLED, true))) { final IndexMetaData indexMetaData = settings.getIndexMetaData();
return; final Settings indexSettings = settings.getSettings();
} if (!indexSettings.getAsBoolean(INDEX_WARMER_ENABLED, settings.getNodeSettings().getAsBoolean(INDEX_WARMER_ENABLED, true))) {
IndexService indexService = indicesService.indexService(context.shardId().index().name());
if (indexService == null) {
return;
}
final IndexShard indexShard = indexService.getShardOrNull(context.shardId().id());
if (indexShard == null) {
return; return;
} }
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
if (topReader) { if (isTopReader) {
logger.trace("[{}][{}] top warming [{}]", context.shardId().index().name(), context.shardId().id(), context); logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader());
} else { } else {
logger.trace("[{}][{}] warming [{}]", context.shardId().index().name(), context.shardId().id(), context); logger.trace("{} warming [{}]", shard.shardId(), searcher.reader());
} }
} }
indexShard.warmerService().onPreWarm(); shard.warmerService().onPreWarm();
long time = System.nanoTime(); long time = System.nanoTime();
final List<TerminationHandle> terminationHandles = new ArrayList<>(); final List<TerminationHandle> terminationHandles = new ArrayList<>();
// get a handle on pending tasks // get a handle on pending tasks
for (final Listener listener : listeners) { for (final Listener listener : listeners) {
if (topReader) { if (isTopReader) {
terminationHandles.add(listener.warmTopReader(indexShard, indexMetaData, context, threadPool)); terminationHandles.add(listener.warmTopReader(shard, searcher));
} else { } else {
terminationHandles.add(listener.warmNewReaders(indexShard, indexMetaData, context, threadPool)); terminationHandles.add(listener.warmNewReaders(shard, searcher));
} }
} }
// wait for termination // wait for termination
@ -116,7 +92,7 @@ public final class IndicesWarmer extends AbstractComponent {
terminationHandle.awaitTermination(); terminationHandle.awaitTermination();
} catch (InterruptedException e) { } catch (InterruptedException e) {
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
if (topReader) { if (isTopReader) {
logger.warn("top warming has been interrupted", e); logger.warn("top warming has been interrupted", e);
} else { } else {
logger.warn("warming has been interrupted", e); logger.warn("warming has been interrupted", e);
@ -125,69 +101,36 @@ public final class IndicesWarmer extends AbstractComponent {
} }
} }
long took = System.nanoTime() - time; long took = System.nanoTime() - time;
indexShard.warmerService().onPostWarm(took); shard.warmerService().onPostWarm(took);
if (indexShard.warmerService().logger().isTraceEnabled()) { if (shard.warmerService().logger().isTraceEnabled()) {
if (topReader) { if (isTopReader) {
indexShard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); shard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS));
} else { } else {
indexShard.warmerService().logger().trace("warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS)); shard.warmerService().logger().trace("warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS));
} }
} }
} }
/**
* Returns an executor for async warmer tasks
*/
public Executor getExecutor() {
return threadPool.executor(ThreadPool.Names.WARMER);
}
/** A handle on the execution of warm-up action. */ /** A handle on the execution of warm-up action. */
public interface TerminationHandle { public interface TerminationHandle {
public static TerminationHandle NO_WAIT = new TerminationHandle() { TerminationHandle NO_WAIT = () -> {};
@Override
public void awaitTermination() {}
};
/** Wait until execution of the warm-up action completes. */ /** Wait until execution of the warm-up action completes. */
void awaitTermination() throws InterruptedException; void awaitTermination() throws InterruptedException;
} }
public static abstract class Listener { public interface Listener {
public String executor() {
return ThreadPool.Names.WARMER;
}
/** Queue tasks to warm-up the given segments and return handles that allow to wait for termination of the execution of those tasks. */ /** Queue tasks to warm-up the given segments and return handles that allow to wait for termination of the execution of those tasks. */
public abstract TerminationHandle warmNewReaders(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool); TerminationHandle warmNewReaders(IndexShard indexShard, Engine.Searcher searcher);
public abstract TerminationHandle warmTopReader(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool); TerminationHandle warmTopReader(IndexShard indexShard, Engine.Searcher searcher);
} }
public static final class WarmerContext {
private final ShardId shardId;
private final Engine.Searcher searcher;
public WarmerContext(ShardId shardId, Engine.Searcher searcher) {
this.shardId = shardId;
this.searcher = searcher;
}
public ShardId shardId() {
return shardId;
}
/** Return a searcher instance that only wraps the segments to warm. */
public Engine.Searcher searcher() {
return searcher;
}
public IndexReader reader() {
return searcher.reader();
}
public DirectoryReader getDirectoryReader() {
return searcher.getDirectoryReader();
}
@Override
public String toString() {
return "WarmerContext: " + searcher.reader();
}
}
} }

View File

@ -68,7 +68,6 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.IndicesWarmer; import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.indices.IndicesWarmer.TerminationHandle; import org.elasticsearch.indices.IndicesWarmer.TerminationHandle;
import org.elasticsearch.indices.IndicesWarmer.WarmerContext;
import org.elasticsearch.indices.cache.request.IndicesRequestCache; import org.elasticsearch.indices.cache.request.IndicesRequestCache;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.ExecutableScript;
@ -180,8 +179,8 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval); this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval);
this.indicesWarmer.addListener(new NormsWarmer()); this.indicesWarmer.addListener(new NormsWarmer(indicesWarmer));
this.indicesWarmer.addListener(new FieldDataWarmer()); this.indicesWarmer.addListener(new FieldDataWarmer(indicesWarmer));
this.indicesWarmer.addListener(new SearchWarmer()); this.indicesWarmer.addListener(new SearchWarmer());
defaultSearchTimeout = settings.getAsTime(DEFAULT_SEARCH_TIMEOUT, NO_TIMEOUT); defaultSearchTimeout = settings.getAsTime(DEFAULT_SEARCH_TIMEOUT, NO_TIMEOUT);
@ -949,11 +948,15 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
return this.activeContexts.size(); return this.activeContexts.size();
} }
static class NormsWarmer extends IndicesWarmer.Listener { static class NormsWarmer implements IndicesWarmer.Listener {
private final IndicesWarmer indicesWarmer;
public NormsWarmer(IndicesWarmer indicesWarmer) {
this.indicesWarmer = indicesWarmer;
}
@Override @Override
public TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaData indexMetaData, final WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
final Loading defaultLoading = Loading.parse(indexMetaData.getSettings().get(NORMS_LOADING_KEY), Loading.LAZY); final Loading defaultLoading = Loading.parse(indexShard.getIndexSettings().getSettings().get(NORMS_LOADING_KEY), Loading.LAZY);
final MapperService mapperService = indexShard.mapperService(); final MapperService mapperService = indexShard.mapperService();
final ObjectSet<String> warmUp = new ObjectHashSet<>(); final ObjectSet<String> warmUp = new ObjectHashSet<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) { for (DocumentMapper docMapper : mapperService.docMappers(false)) {
@ -971,14 +974,14 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
final CountDownLatch latch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1);
// Norms loading may be I/O intensive but is not CPU intensive, so we execute it in a single task // Norms loading may be I/O intensive but is not CPU intensive, so we execute it in a single task
threadPool.executor(executor()).execute(new Runnable() { indicesWarmer.getExecutor().execute(new Runnable() {
@Override @Override
public void run() { public void run() {
try { try {
for (ObjectCursor<String> stringObjectCursor : warmUp) { for (ObjectCursor<String> stringObjectCursor : warmUp) {
final String indexName = stringObjectCursor.value; final String indexName = stringObjectCursor.value;
final long start = System.nanoTime(); final long start = System.nanoTime();
for (final LeafReaderContext ctx : context.searcher().reader().leaves()) { for (final LeafReaderContext ctx : searcher.reader().leaves()) {
final NumericDocValues values = ctx.reader().getNormValues(indexName); final NumericDocValues values = ctx.reader().getNormValues(indexName);
if (values != null) { if (values != null) {
values.get(0); values.get(0);
@ -1005,15 +1008,21 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
} }
@Override @Override
public TerminationHandle warmTopReader(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmTopReader(IndexShard indexShard, final Engine.Searcher searcher) {
return TerminationHandle.NO_WAIT; return TerminationHandle.NO_WAIT;
} }
} }
static class FieldDataWarmer extends IndicesWarmer.Listener { static class FieldDataWarmer implements IndicesWarmer.Listener {
private final IndicesWarmer indicesWarmer;
public FieldDataWarmer(IndicesWarmer indicesWarmer) {
this.indicesWarmer = indicesWarmer;
}
@Override @Override
public TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaData indexMetaData, final WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
final MapperService mapperService = indexShard.mapperService(); final MapperService mapperService = indexShard.mapperService();
final Map<String, MappedFieldType> warmUp = new HashMap<>(); final Map<String, MappedFieldType> warmUp = new HashMap<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) { for (DocumentMapper docMapper : mapperService.docMappers(false)) {
@ -1048,9 +1057,9 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
} }
} }
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
final Executor executor = threadPool.executor(executor()); final Executor executor = indicesWarmer.getExecutor();
final CountDownLatch latch = new CountDownLatch(context.searcher().reader().leaves().size() * warmUp.size()); final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size());
for (final LeafReaderContext ctx : context.searcher().reader().leaves()) { for (final LeafReaderContext ctx : searcher.reader().leaves()) {
for (final MappedFieldType fieldType : warmUp.values()) { for (final MappedFieldType fieldType : warmUp.values()) {
executor.execute(new Runnable() { executor.execute(new Runnable() {
@ -1081,7 +1090,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
} }
@Override @Override
public TerminationHandle warmTopReader(final IndexShard indexShard, IndexMetaData indexMetaData, final WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmTopReader(final IndexShard indexShard, final Engine.Searcher searcher) {
final MapperService mapperService = indexShard.mapperService(); final MapperService mapperService = indexShard.mapperService();
final Map<String, MappedFieldType> warmUpGlobalOrdinals = new HashMap<>(); final Map<String, MappedFieldType> warmUpGlobalOrdinals = new HashMap<>();
for (DocumentMapper docMapper : mapperService.docMappers(false)) { for (DocumentMapper docMapper : mapperService.docMappers(false)) {
@ -1114,7 +1123,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
} }
} }
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService(); final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
final Executor executor = threadPool.executor(executor()); final Executor executor = indicesWarmer.getExecutor();
final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size()); final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size());
for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) { for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) {
executor.execute(new Runnable() { executor.execute(new Runnable() {
@ -1123,7 +1132,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
try { try {
final long start = System.nanoTime(); final long start = System.nanoTime();
IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType); IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType);
ifd.loadGlobal(context.getDirectoryReader()); ifd.loadGlobal(searcher.getDirectoryReader());
if (indexShard.warmerService().logger().isTraceEnabled()) { if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.names().fullName(), TimeValue.timeValueNanos(System.nanoTime() - start)); indexShard.warmerService().logger().trace("warmed global ordinals for [{}], took [{}]", fieldType.names().fullName(), TimeValue.timeValueNanos(System.nanoTime() - start));
} }
@ -1144,83 +1153,73 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
} }
} }
class SearchWarmer extends IndicesWarmer.Listener { class SearchWarmer implements IndicesWarmer.Listener {
@Override @Override
public TerminationHandle warmNewReaders(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmNewReaders(IndexShard indexShard, final Engine.Searcher searcher) {
return internalWarm(indexShard, indexMetaData, context, threadPool, false); return internalWarm(indexShard, searcher, false);
} }
@Override @Override
public TerminationHandle warmTopReader(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) { public TerminationHandle warmTopReader(IndexShard indexShard, final Engine.Searcher searcher) {
return internalWarm(indexShard, indexMetaData, context, threadPool, true); return internalWarm(indexShard, searcher, true);
} }
public TerminationHandle internalWarm(final IndexShard indexShard, final IndexMetaData indexMetaData, final IndicesWarmer.WarmerContext warmerContext, ThreadPool threadPool, final boolean top) { public TerminationHandle internalWarm(final IndexShard indexShard, final Engine.Searcher searcher, final boolean top) {
IndexWarmersMetaData custom = indexMetaData.custom(IndexWarmersMetaData.TYPE); IndexWarmersMetaData custom = indexShard.getIndexSettings().getIndexMetaData().custom(IndexWarmersMetaData.TYPE);
if (custom == null) { if (custom == null) {
return TerminationHandle.NO_WAIT; return TerminationHandle.NO_WAIT;
} }
final Executor executor = threadPool.executor(executor()); final Executor executor = indicesWarmer.getExecutor();
final CountDownLatch latch = new CountDownLatch(custom.entries().size()); final CountDownLatch latch = new CountDownLatch(custom.entries().size());
for (final IndexWarmersMetaData.Entry entry : custom.entries()) { for (final IndexWarmersMetaData.Entry entry : custom.entries()) {
executor.execute(new Runnable() { executor.execute(() -> {
SearchContext context = null;
@Override try {
public void run() { long now = System.nanoTime();
SearchContext context = null; final IndexService indexService = indicesService.indexServiceSafe(indexShard.shardId().index().name());
QueryParseContext queryParseContext = new QueryParseContext(indexService.queryParserService().indicesQueriesRegistry());
queryParseContext.parseFieldMatcher(indexService.queryParserService().parseFieldMatcher());
ShardSearchRequest request = new ShardSearchLocalRequest(indexShard.shardId(), indexShard.getIndexSettings()
.getNumberOfShards(),
SearchType.QUERY_THEN_FETCH, entry.source().build(queryParseContext), entry.types(), entry.requestCache());
context = createContext(request, searcher);
// if we use sort, we need to do query to sort on
// it and load relevant field data
// if not, we might as well set size=0 (and cache
// if needed)
if (context.sort() == null) {
context.size(0);
}
boolean canCache = indicesQueryCache.canCache(request, context);
// early terminate when we can cache, since we
// can only do proper caching on top level searcher
// also, if we can't cache, and its top, we don't
// need to execute it, since we already did when its
// not top
if (canCache != top) {
return;
}
loadOrExecuteQueryPhase(request, context, queryPhase);
long took = System.nanoTime() - now;
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed [{}], took [{}]", entry.name(), TimeValue.timeValueNanos(took));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("warmer [{}] failed", t, entry.name());
} finally {
try { try {
long now = System.nanoTime(); if (context != null) {
final IndexService indexService = indicesService.indexServiceSafe(indexShard.shardId().index().name()); freeContext(context.id());
QueryParseContext queryParseContext = new QueryParseContext(indexService.queryParserService().indicesQueriesRegistry()); cleanContext(context);
queryParseContext.parseFieldMatcher(indexService.queryParserService().parseFieldMatcher());
ShardSearchRequest request = new ShardSearchLocalRequest(indexShard.shardId(), indexMetaData
.getNumberOfShards(),
SearchType.QUERY_THEN_FETCH, entry.source().build(queryParseContext), entry.types(), entry.requestCache());
context = createContext(request, warmerContext.searcher());
// if we use sort, we need to do query to sort on
// it and load relevant field data
// if not, we might as well set size=0 (and cache
// if needed)
if (context.sort() == null) {
context.size(0);
} }
boolean canCache = indicesQueryCache.canCache(request, context);
// early terminate when we can cache, since we
// can only do proper caching on top level searcher
// also, if we can't cache, and its top, we don't
// need to execute it, since we already did when its
// not top
if (canCache != top) {
return;
}
loadOrExecuteQueryPhase(request, context, queryPhase);
long took = System.nanoTime() - now;
if (indexShard.warmerService().logger().isTraceEnabled()) {
indexShard.warmerService().logger().trace("warmed [{}], took [{}]", entry.name(), TimeValue.timeValueNanos(took));
}
} catch (Throwable t) {
indexShard.warmerService().logger().warn("warmer [{}] failed", t, entry.name());
} finally { } finally {
try { latch.countDown();
if (context != null) {
freeContext(context.id());
cleanContext(context);
}
} finally {
latch.countDown();
}
} }
} }
}); });
} }
return new TerminationHandle() { return () -> latch.await();
@Override
public void awaitTermination() throws InterruptedException {
latch.await();
}
};
} }
} }

View File

@ -21,15 +21,16 @@ package org.elasticsearch.index;
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;
import org.apache.lucene.search.CollectionStatistics; import org.apache.lucene.search.*;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TermStatistics;
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.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.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.engine.EngineConfig; import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineException; import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.engine.EngineFactory;
@ -40,6 +41,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.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.test.engine.MockEngineFactory; import org.elasticsearch.test.engine.MockEngineFactory;
@ -49,13 +51,13 @@ 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);
public void testWrapperIsBound() { public void testWrapperIsBound() {
final Index index = new Index("foo"); final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).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); IndexModule module = new IndexModule(indexSettings, null, null, warmer);
assertInstanceBinding(module, IndexSearcherWrapper.class,(x) -> x == null); assertInstanceBinding(module, IndexSearcherWrapper.class, (x) -> x == null);
module.indexSearcherWrapper = Wrapper.class; module.indexSearcherWrapper = Wrapper.class;
assertBinding(module, IndexSearcherWrapper.class, Wrapper.class); assertBinding(module, IndexSearcherWrapper.class, Wrapper.class);
} }
@ -64,7 +66,7 @@ public class IndexModuleTests extends ModuleTestCase {
final Index index = new Index("foo"); final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).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); IndexModule module = new IndexModule(indexSettings, null, null, warmer);
assertBinding(module, EngineFactory.class, InternalEngineFactory.class); assertBinding(module, EngineFactory.class, InternalEngineFactory.class);
module.engineFactoryImpl = MockEngineFactory.class; module.engineFactoryImpl = MockEngineFactory.class;
assertBinding(module, EngineFactory.class, MockEngineFactory.class); assertBinding(module, EngineFactory.class, MockEngineFactory.class);
@ -74,7 +76,7 @@ public class IndexModuleTests extends ModuleTestCase {
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(IndexModule.STORE_TYPE, "foo_store").build(); final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).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); IndexModule module = new IndexModule(indexSettings, null, null, warmer);
module.addIndexStore("foo_store", FooStore::new); module.addIndexStore("foo_store", FooStore::new);
assertInstanceBinding(module, IndexStore.class, (x) -> x.getClass() == FooStore.class); assertInstanceBinding(module, IndexStore.class, (x) -> x.getClass() == FooStore.class);
try { try {
@ -96,7 +98,7 @@ public class IndexModuleTests extends ModuleTestCase {
final Index index = new Index("foo"); final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).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); IndexModule module = new IndexModule(indexSettings, null, null, warmer);
Consumer<Settings> listener = (s) -> {}; Consumer<Settings> listener = (s) -> {};
module.addIndexSettingsListener(listener); module.addIndexSettingsListener(listener);
module.addIndexEventListener(eventListener); module.addIndexEventListener(eventListener);
@ -117,7 +119,7 @@ public class IndexModuleTests extends ModuleTestCase {
final Index index = new Index("foo"); final Index index = new Index("foo");
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).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); IndexModule module = new IndexModule(indexSettings, null, null, warmer);
Consumer<Settings> listener = (s) -> { Consumer<Settings> listener = (s) -> {
}; };
module.addIndexSettingsListener(listener); module.addIndexSettingsListener(listener);
@ -145,7 +147,7 @@ public class IndexModuleTests extends ModuleTestCase {
.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")
.build(); .build();
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
module.addSimilarity("test_similarity", (string, settings) -> new SimilarityProvider() { module.addSimilarity("test_similarity", (string, settings) -> new SimilarityProvider() {
@Override @Override
public String name() { public String name() {
@ -174,7 +176,7 @@ public class IndexModuleTests extends ModuleTestCase {
.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)
.build(); .build();
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
try { try {
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService); assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
@ -188,7 +190,7 @@ public class IndexModuleTests extends ModuleTestCase {
.put("index.similarity.my_similarity.foo", "bar") .put("index.similarity.my_similarity.foo", "bar")
.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); IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
try { try {
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService); assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService);
} catch (IllegalArgumentException ex) { } catch (IllegalArgumentException ex) {
@ -196,6 +198,76 @@ public class IndexModuleTests extends ModuleTestCase {
} }
} }
public void testCannotRegisterProvidedImplementations() {
Settings indexSettings = Settings.settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
try {
module.registerQueryCache("index", IndexQueryCache::new);
fail("only once");
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [index]");
}
try {
module.registerQueryCache("none", (settings, x) -> new NoneQueryCache(settings));
fail("only once");
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [none]");
}
try {
module.registerQueryCache("index", null);
fail("must not be null");
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "provider must not be null");
}
}
public void testRegisterCustomQueryCache() {
Settings indexSettings = Settings.settingsBuilder()
.put(IndexModule.QUERY_CACHE_TYPE, "custom")
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
try {
module.registerQueryCache("custom", (a, b) -> new CustomQueryCache());
fail("only once");
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]");
}
assertInstanceBinding(module, QueryCache.class, (x) -> x instanceof CustomQueryCache);
}
public void testDefaultQueryCacheImplIsSelected() {
Settings indexSettings = Settings.settingsBuilder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexModule module = new IndexModule(IndexSettingsModule.newIndexSettings(new Index("foo"), indexSettings, Collections.EMPTY_LIST), null, null, warmer);
assertInstanceBinding(module, QueryCache.class, (x) -> x instanceof IndexQueryCache);
}
class CustomQueryCache implements QueryCache {
@Override
public void clear(String reason) {
}
@Override
public void close() throws IOException {
}
@Override
public Index index() {
return new Index("test");
}
@Override
public Weight doCache(Weight weight, QueryCachingPolicy policy) {
return weight;
}
}
private static class TestSimilarity extends Similarity { private static class TestSimilarity extends Similarity {
private final Similarity delegate = new BM25Similarity(); private final Similarity delegate = new BM25Similarity();

View File

@ -1,89 +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.cache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.Weight;
import org.elasticsearch.common.inject.ModuleTestCase;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.cache.query.QueryCache;
import org.elasticsearch.index.cache.query.index.IndexQueryCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import java.io.IOException;
public class IndexCacheModuleTests extends ModuleTestCase {
public void testCannotRegisterProvidedImplementations() {
IndexCacheModule module = new IndexCacheModule(Settings.EMPTY);
try {
module.registerQueryCache("index", IndexQueryCache.class);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [index]");
}
try {
module.registerQueryCache("none", NoneQueryCache.class);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [none]");
}
}
public void testRegisterCustomQueryCache() {
IndexCacheModule module = new IndexCacheModule(
Settings.builder().put(IndexCacheModule.QUERY_CACHE_TYPE, "custom").build()
);
module.registerQueryCache("custom", CustomQueryCache.class);
try {
module.registerQueryCache("custom", CustomQueryCache.class);
} catch (IllegalArgumentException e) {
assertEquals(e.getMessage(), "Can't register the same [query_cache] more than once for [custom]");
}
assertBinding(module, QueryCache.class, CustomQueryCache.class);
}
public void testDefaultQueryCacheImplIsSelected() {
IndexCacheModule module = new IndexCacheModule(Settings.EMPTY);
assertBinding(module, QueryCache.class, IndexQueryCache.class);
}
class CustomQueryCache implements QueryCache {
@Override
public void clear(String reason) {
}
@Override
public void close() throws IOException {
}
@Override
public Index index() {
return new Index("test");
}
@Override
public Weight doCache(Weight weight, QueryCachingPolicy policy) {
return weight;
}
}
}

View File

@ -37,12 +37,12 @@ import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BitSet; import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesWarmer;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.IndexSettingsModule;
@ -56,6 +56,8 @@ import static org.hamcrest.Matchers.equalTo;
public class BitSetFilterCacheTests extends ESTestCase { public class BitSetFilterCacheTests extends ESTestCase {
private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.emptyList()); private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.emptyList());
private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null);
private static int matchCount(BitSetProducer producer, IndexReader reader) throws IOException { private static int matchCount(BitSetProducer producer, IndexReader reader) throws IOException {
int count = 0; int count = 0;
@ -91,7 +93,7 @@ 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); BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer);
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));
@ -134,7 +136,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); final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer);
cache.setListener(new BitsetFilterCache.Listener() { cache.setListener(new BitsetFilterCache.Listener() {
@Override @Override
public void onCache(ShardId shardId, Accountable accountable) { public void onCache(ShardId shardId, Accountable accountable) {
@ -173,7 +175,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
} }
public void testSetListenerTwice() { public void testSetListenerTwice() {
final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS); final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer);
cache.setListener(new BitsetFilterCache.Listener() { cache.setListener(new BitsetFilterCache.Listener() {
@Override @Override

View File

@ -60,13 +60,16 @@ import org.elasticsearch.env.EnvironmentModule;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisModule; import org.elasticsearch.index.analysis.AnalysisModule;
import org.elasticsearch.index.cache.IndexCacheModule; import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
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.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.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.analysis.IndicesAnalysisService; import org.elasticsearch.indices.analysis.IndicesAnalysisService;
import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
@ -213,13 +216,16 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
} }
}, },
new IndexSettingsModule(index, indexSettings), new IndexSettingsModule(index, indexSettings),
new IndexCacheModule(indexSettings),
new AnalysisModule(indexSettings, new IndicesAnalysisService(indexSettings)), new AnalysisModule(indexSettings, new IndicesAnalysisService(indexSettings)),
new AbstractModule() { new AbstractModule() {
@Override @Override
protected void configure() { protected void configure() {
SimilarityService service = new SimilarityService(IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST), Collections.EMPTY_MAP); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings, Collections.EMPTY_LIST);
SimilarityService service = new SimilarityService(idxSettings, Collections.EMPTY_MAP);
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();

View File

@ -37,11 +37,15 @@ import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.Environment; import org.elasticsearch.env.Environment;
import org.elasticsearch.env.EnvironmentModule; import org.elasticsearch.env.EnvironmentModule;
import org.elasticsearch.index.Index; import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisModule; import org.elasticsearch.index.analysis.AnalysisModule;
import org.elasticsearch.index.cache.IndexCacheModule; import org.elasticsearch.index.cache.IndexCache;
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
import org.elasticsearch.index.cache.query.none.NoneQueryCache;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
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.analysis.IndicesAnalysisService; import org.elasticsearch.indices.analysis.IndicesAnalysisService;
import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
@ -95,12 +99,15 @@ public class TemplateQueryParserTests extends ESTestCase {
}, },
new ScriptModule(settings), new ScriptModule(settings),
new IndexSettingsModule(index, settings), new IndexSettingsModule(index, settings),
new IndexCacheModule(settings),
new AnalysisModule(settings, new IndicesAnalysisService(settings)), new AnalysisModule(settings, new IndicesAnalysisService(settings)),
new AbstractModule() { new AbstractModule() {
@Override @Override
protected void configure() { protected void configure() {
SimilarityService service = new SimilarityService(IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST), Collections.EMPTY_MAP); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
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);

View File

@ -38,8 +38,8 @@ import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.VersionType; import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.cache.query.QueryCacheStats; import org.elasticsearch.index.cache.query.QueryCacheStats;
import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
@ -77,8 +77,8 @@ public class IndexStatsIT extends ESIntegTestCase {
//Filter/Query cache is cleaned periodically, default is 60s, so make sure it runs often. Thread.sleep for 60s is bad //Filter/Query cache is cleaned periodically, default is 60s, so make sure it runs often. Thread.sleep for 60s is bad
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)) return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
.put(IndicesRequestCache.INDICES_CACHE_REQUEST_CLEAN_INTERVAL, "1ms") .put(IndicesRequestCache.INDICES_CACHE_REQUEST_CLEAN_INTERVAL, "1ms")
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true) .put(IndexModule.QUERY_CACHE_EVERYTHING, true)
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE) .put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
.build(); .build();
} }

View File

@ -30,7 +30,7 @@ import org.elasticsearch.common.lucene.search.function.CombineFunction;
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery; import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.cache.IndexCacheModule; import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.mapper.MergeMappingException; import org.elasticsearch.index.mapper.MergeMappingException;
import org.elasticsearch.index.query.HasChildQueryBuilder; import org.elasticsearch.index.query.HasChildQueryBuilder;
import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.IdsQueryBuilder;
@ -97,8 +97,8 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)) return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
// aggressive filter caching so that we can assert on the filter cache size // aggressive filter caching so that we can assert on the filter cache size
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE) .put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true) .put(IndexModule.QUERY_CACHE_EVERYTHING, true)
.build(); .build();
} }

View File

@ -62,8 +62,8 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.cache.IndexCacheModule;
import org.elasticsearch.index.engine.CommitStats; import org.elasticsearch.index.engine.CommitStats;
import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
@ -446,11 +446,11 @@ public final class InternalTestCluster extends TestCluster {
} }
if (random.nextBoolean()) { if (random.nextBoolean()) {
builder.put(IndexCacheModule.QUERY_CACHE_TYPE, random.nextBoolean() ? IndexCacheModule.INDEX_QUERY_CACHE : IndexCacheModule.NONE_QUERY_CACHE); builder.put(IndexModule.QUERY_CACHE_TYPE, random.nextBoolean() ? IndexModule.INDEX_QUERY_CACHE : IndexModule.NONE_QUERY_CACHE);
} }
if (random.nextBoolean()) { if (random.nextBoolean()) {
builder.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, random.nextBoolean()); builder.put(IndexModule.QUERY_CACHE_EVERYTHING, random.nextBoolean());
} }
if (random.nextBoolean()) { if (random.nextBoolean()) {

View File

@ -21,7 +21,7 @@ package org.elasticsearch.messy.tests;
import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.cache.IndexCacheModule; import org.elasticsearch.index.IndexModule;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
@ -53,8 +53,8 @@ public class ScriptQuerySearchTests extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)) return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
// aggressive filter caching so that we can assert on the number of iterations of the script filters // aggressive filter caching so that we can assert on the number of iterations of the script filters
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE) .put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true) .put(IndexModule.QUERY_CACHE_EVERYTHING, true)
.build(); .build();
} }