Merge pull request #14293 from s1monw/fold_in_index_cache_module
Fold IndexCacheModule into IndexModule
This commit is contained in:
commit
59a71b0441
|
@ -22,6 +22,12 @@ package org.elasticsearch.index;
|
|||
import org.elasticsearch.common.inject.AbstractModule;
|
||||
import org.elasticsearch.common.inject.util.Providers;
|
||||
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.InternalEngineFactory;
|
||||
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.store.IndexStore;
|
||||
import org.elasticsearch.index.store.IndexStoreConfig;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
|
||||
|
||||
import java.util.*;
|
||||
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 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 IndexStoreConfig indexStoreConfig;
|
||||
private final IndicesQueryCache indicesQueryCache;
|
||||
// pkg private so tests can mock
|
||||
Class<? extends EngineFactory> engineFactoryImpl = InternalEngineFactory.class;
|
||||
Class<? extends IndexSearcherWrapper> indexSearcherWrapper = null;
|
||||
|
@ -64,11 +78,17 @@ public class IndexModule extends AbstractModule {
|
|||
private IndexEventListener listener;
|
||||
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, 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.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);
|
||||
}
|
||||
|
||||
/**
|
||||
* 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() {
|
||||
// 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");
|
||||
}
|
||||
}
|
||||
|
||||
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));
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@
|
|||
package org.elasticsearch.index.cache;
|
||||
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.cache.bitset.BitsetFilterCache;
|
||||
|
@ -37,7 +36,6 @@ public class IndexCache extends AbstractIndexComponent implements Closeable {
|
|||
private final QueryCache queryCache;
|
||||
private final BitsetFilterCache bitsetFilterCache;
|
||||
|
||||
@Inject
|
||||
public IndexCache(IndexSettings indexSettings, QueryCache queryCache, BitsetFilterCache bitsetFilterCache) {
|
||||
super(indexSettings);
|
||||
this.queryCache = queryCache;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -32,16 +32,15 @@ import org.apache.lucene.util.Accountable;
|
|||
import org.apache.lucene.util.BitDocIdSet;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
import org.elasticsearch.ExceptionsHelper;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.cache.Cache;
|
||||
import org.elasticsearch.common.cache.CacheBuilder;
|
||||
import org.elasticsearch.common.cache.RemovalListener;
|
||||
import org.elasticsearch.common.cache.RemovalNotification;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.lucene.search.Queries;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.index.AbstractIndexComponent;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.mapper.DocumentMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
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.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.IndicesWarmer.TerminationHandle;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.io.Closeable;
|
||||
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
|
||||
* {@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";
|
||||
private static final Listener DEFAULT_NOOP_LISTENER = new Listener() {
|
||||
|
@ -85,24 +83,17 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
|
|||
private final Cache<Object, Cache<Query, Value>> loadedFilters;
|
||||
private volatile Listener listener = DEFAULT_NOOP_LISTENER;
|
||||
private final BitSetProducerWarmer warmer;
|
||||
private final IndicesWarmer indicesWarmer;
|
||||
|
||||
private IndicesWarmer indicesWarmer;
|
||||
|
||||
@Inject
|
||||
public BitsetFilterCache(IndexSettings indexSettings) {
|
||||
public BitsetFilterCache(IndexSettings indexSettings, IndicesWarmer indicesWarmer) {
|
||||
super(indexSettings);
|
||||
this.loadRandomAccessFiltersEagerly = this.indexSettings.getSettings().getAsBoolean(LOAD_RANDOM_ACCESS_FILTERS_EAGERLY, true);
|
||||
this.loadedFilters = CacheBuilder.<Object, Cache<Query, Value>>builder().removalListener(this).build();
|
||||
this.warmer = new BitSetProducerWarmer();
|
||||
}
|
||||
|
||||
|
||||
@Inject(optional = true)
|
||||
public void setIndicesWarmer(IndicesWarmer indicesWarmer) {
|
||||
this.indicesWarmer = indicesWarmer;
|
||||
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
|
||||
|
@ -129,10 +120,11 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
|
|||
|
||||
@Override
|
||||
public void close() {
|
||||
if (indicesWarmer != null) {
|
||||
try {
|
||||
indicesWarmer.removeListener(warmer);
|
||||
} finally {
|
||||
clear("close");
|
||||
}
|
||||
clear("close");
|
||||
}
|
||||
|
||||
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
|
||||
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) {
|
||||
return TerminationHandle.NO_WAIT;
|
||||
}
|
||||
|
@ -258,9 +250,9 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
|
|||
warmUp.add(Queries.newNonNestedFilter());
|
||||
}
|
||||
|
||||
final Executor executor = threadPool.executor(executor());
|
||||
final CountDownLatch latch = new CountDownLatch(context.searcher().reader().leaves().size() * warmUp.size());
|
||||
for (final LeafReaderContext ctx : context.searcher().reader().leaves()) {
|
||||
final Executor executor = indicesWarmer.getExecutor();
|
||||
final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size());
|
||||
for (final LeafReaderContext ctx : searcher.reader().leaves()) {
|
||||
for (final Query filterToWarm : warmUp) {
|
||||
executor.execute(() -> {
|
||||
try {
|
||||
|
@ -281,7 +273,7 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
|
||||
|
|
|
@ -1056,4 +1056,18 @@ public abstract class Engine implements Closeable {
|
|||
public long getLastWriteNanos() {
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -61,8 +61,7 @@ public final class EngineConfig {
|
|||
private final String codecName;
|
||||
private final ThreadPool threadPool;
|
||||
private final ShardIndexingService indexingService;
|
||||
@Nullable
|
||||
private final IndicesWarmer warmer;
|
||||
private final Engine.Warmer warmer;
|
||||
private final Store store;
|
||||
private final SnapshotDeletionPolicy deletionPolicy;
|
||||
private final MergePolicy mergePolicy;
|
||||
|
@ -116,7 +115,7 @@ public final class EngineConfig {
|
|||
* Creates a new {@link org.elasticsearch.index.engine.EngineConfig}
|
||||
*/
|
||||
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,
|
||||
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
|
||||
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy, TranslogConfig translogConfig, TimeValue flushMergesAfter) {
|
||||
|
@ -124,7 +123,7 @@ public final class EngineConfig {
|
|||
this.indexSettings = indexSettings;
|
||||
this.threadPool = threadPool;
|
||||
this.indexingService = indexingService;
|
||||
this.warmer = warmer;
|
||||
this.warmer = warmer == null ? (a,b) -> {} : warmer;
|
||||
this.store = store;
|
||||
this.deletionPolicy = deletionPolicy;
|
||||
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.
|
||||
* Note: This method might retrun <code>null</code>
|
||||
* Returns an {@link org.elasticsearch.index.engine.Engine.Warmer} used to warm new searchers before they are used for searching.
|
||||
*/
|
||||
@Nullable
|
||||
public IndicesWarmer getWarmer() {
|
||||
public Engine.Warmer getWarmer() {
|
||||
return warmer;
|
||||
}
|
||||
|
||||
|
|
|
@ -79,8 +79,7 @@ public class InternalEngine extends Engine {
|
|||
private volatile long lastDeleteVersionPruneTimeMSec;
|
||||
|
||||
private final ShardIndexingService indexingService;
|
||||
@Nullable
|
||||
private final IndicesWarmer warmer;
|
||||
private final Engine.Warmer warmer;
|
||||
private final Translog translog;
|
||||
private final ElasticsearchConcurrentMergeScheduler mergeScheduler;
|
||||
|
||||
|
@ -930,8 +929,7 @@ public class InternalEngine extends Engine {
|
|||
assert isMergedSegment(esLeafReader);
|
||||
if (warmer != null) {
|
||||
final Engine.Searcher searcher = new Searcher("warmer", searcherFactory.newSearcher(esLeafReader, null));
|
||||
final IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, searcher);
|
||||
warmer.warmNewReaders(context);
|
||||
warmer.warm(searcher, false);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
// 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 */
|
||||
final static class SearchFactory extends EngineSearcherFactory {
|
||||
private final IndicesWarmer warmer;
|
||||
private final Engine.Warmer warmer;
|
||||
private final ShardId shardId;
|
||||
private final ESLogger logger;
|
||||
private final AtomicBoolean isEngineClosed;
|
||||
|
@ -1014,11 +1012,10 @@ public class InternalEngine extends Engine {
|
|||
}
|
||||
|
||||
if (newSearcher != null) {
|
||||
IndicesWarmer.WarmerContext context = new IndicesWarmer.WarmerContext(shardId, new Searcher("new_reader_warming", newSearcher));
|
||||
warmer.warmNewReaders(context);
|
||||
warmer.warm(new Searcher("new_reader_warming", newSearcher), false);
|
||||
}
|
||||
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) {
|
||||
if (isEngineClosed.get() == false) {
|
||||
logger.warn("failed to prepare/warm", e);
|
||||
|
|
|
@ -55,11 +55,11 @@ import org.elasticsearch.common.util.concurrent.AbstractRefCounted;
|
|||
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
|
||||
import org.elasticsearch.common.util.concurrent.FutureUtils;
|
||||
import org.elasticsearch.gateway.MetaDataStateFormat;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.IndexServicesProvider;
|
||||
import org.elasticsearch.index.VersionType;
|
||||
import org.elasticsearch.index.cache.IndexCache;
|
||||
import org.elasticsearch.index.cache.IndexCacheModule;
|
||||
import org.elasticsearch.index.cache.bitset.ShardBitsetFilterCache;
|
||||
import org.elasticsearch.index.cache.query.QueryCacheStats;
|
||||
import org.elasticsearch.index.cache.request.ShardRequestCache;
|
||||
|
@ -147,6 +147,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
private final MergePolicyConfig mergePolicyConfig;
|
||||
private final IndicesQueryCache indicesQueryCache;
|
||||
private final IndexEventListener indexEventListener;
|
||||
private final IndexSettings idxSettings;
|
||||
|
||||
private TimeValue refreshInterval;
|
||||
|
||||
|
@ -197,6 +198,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
@Inject
|
||||
public IndexShard(ShardId shardId, IndexSettings indexSettings, ShardPath path, Store store, IndexServicesProvider provider) {
|
||||
super(shardId, indexSettings);
|
||||
this.idxSettings = indexSettings;
|
||||
this.codecService = provider.getCodecService();
|
||||
this.warmer = provider.getWarmer();
|
||||
this.deletionPolicy = new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy());
|
||||
|
@ -234,7 +236,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
final QueryCachingPolicy cachingPolicy;
|
||||
// the query cache is a node-level thing, however we want the most popular filters
|
||||
// to be computed on a per-shard basis
|
||||
if (this.indexSettings.getAsBoolean(IndexCacheModule.QUERY_CACHE_EVERYTHING, false)) {
|
||||
if (this.indexSettings.getAsBoolean(IndexModule.QUERY_CACHE_EVERYTHING, false)) {
|
||||
cachingPolicy = QueryCachingPolicy.ALWAYS_CACHE;
|
||||
} else {
|
||||
cachingPolicy = new UsageTrackingQueryCachingPolicy();
|
||||
|
@ -261,6 +263,10 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
return this.store;
|
||||
}
|
||||
|
||||
public IndexSettings getIndexSettings() {
|
||||
return idxSettings;
|
||||
}
|
||||
|
||||
/** returns true if this shard supports indexing (i.e., write) operations. */
|
||||
public boolean canIndex() {
|
||||
return true;
|
||||
|
@ -678,7 +684,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
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();
|
||||
}
|
||||
};
|
||||
final Engine.Warmer engineWarmer = (searcher, toLevel) -> warmer.warm(searcher, this, idxSettings, toLevel);
|
||||
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());
|
||||
}
|
||||
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.elasticsearch.index.*;
|
|||
import org.elasticsearch.index.analysis.AnalysisModule;
|
||||
import org.elasticsearch.index.analysis.AnalysisService;
|
||||
import org.elasticsearch.index.cache.IndexCache;
|
||||
import org.elasticsearch.index.cache.IndexCacheModule;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
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.store.IndexStoreConfig;
|
||||
import org.elasticsearch.indices.analysis.IndicesAnalysisService;
|
||||
import org.elasticsearch.indices.cache.query.IndicesQueryCache;
|
||||
import org.elasticsearch.indices.recovery.RecoverySettings;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.plugins.PluginsService;
|
||||
|
@ -94,6 +94,8 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
private final PluginsService pluginsService;
|
||||
private final NodeEnvironment nodeEnv;
|
||||
private final TimeValue shardsClosedTimeout;
|
||||
private final IndicesWarmer indicesWarmer;
|
||||
private final IndicesQueryCache indicesQueryCache;
|
||||
|
||||
private volatile Map<String, IndexServiceInjectorPair> indices = emptyMap();
|
||||
|
||||
|
@ -121,12 +123,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
private final IndexStoreConfig indexStoreConfig;
|
||||
|
||||
@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);
|
||||
this.indicesAnalysisService = indicesAnalysisService;
|
||||
this.injector = injector;
|
||||
this.pluginsService = pluginsService;
|
||||
this.nodeEnv = nodeEnv;
|
||||
this.indicesWarmer = indicesWarmer;
|
||||
this.indicesQueryCache = indicesQueryCache;
|
||||
this.shardsClosedTimeout = settings.getAsTime(INDICES_SHARDS_CLOSED_TIMEOUT, new TimeValue(1, TimeUnit.DAYS));
|
||||
this.indexStoreConfig = new IndexStoreConfig(settings);
|
||||
nodeSettingsService.addListener(indexStoreConfig);
|
||||
|
@ -306,13 +310,12 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
|
|||
for (Module pluginModule : pluginsService.indexModules(idxSettings.getSettings())) {
|
||||
modules.add(pluginModule);
|
||||
}
|
||||
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig);
|
||||
final IndexModule indexModule = new IndexModule(idxSettings, indexStoreConfig, indicesQueryCache, indicesWarmer);
|
||||
for (IndexEventListener listener : builtInListeners) {
|
||||
indexModule.addIndexEventListener(listener);
|
||||
}
|
||||
indexModule.addIndexEventListener(oldShardsStats);
|
||||
modules.add(new AnalysisModule(idxSettings.getSettings(), indicesAnalysisService));
|
||||
modules.add(new IndexCacheModule(idxSettings.getSettings()));
|
||||
modules.add(indexModule);
|
||||
pluginsService.processModules(modules);
|
||||
final IndexEventListener listener = indexModule.freeze();
|
||||
|
|
|
@ -19,23 +19,21 @@
|
|||
|
||||
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.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
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.shard.IndexShard;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.index.shard.IndexShardState;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
|
@ -46,68 +44,46 @@ public final class IndicesWarmer extends AbstractComponent {
|
|||
|
||||
private final ThreadPool threadPool;
|
||||
|
||||
private final ClusterService clusterService;
|
||||
|
||||
private final IndicesService indicesService;
|
||||
|
||||
private final CopyOnWriteArrayList<Listener> listeners = new CopyOnWriteArrayList<>();
|
||||
|
||||
@Inject
|
||||
public IndicesWarmer(Settings settings, ThreadPool threadPool, ClusterService clusterService, IndicesService indicesService) {
|
||||
public IndicesWarmer(Settings settings, ThreadPool threadPool) {
|
||||
super(settings);
|
||||
this.threadPool = threadPool;
|
||||
this.clusterService = clusterService;
|
||||
this.indicesService = indicesService;
|
||||
}
|
||||
|
||||
public void addListener(Listener listener) {
|
||||
listeners.add(listener);
|
||||
}
|
||||
|
||||
public void removeListener(Listener listener) {
|
||||
listeners.remove(listener);
|
||||
}
|
||||
|
||||
public void warmNewReaders(final WarmerContext context) {
|
||||
warmInternal(context, false);
|
||||
}
|
||||
|
||||
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) {
|
||||
public void warm(Engine.Searcher searcher, IndexShard shard, IndexSettings settings, boolean isTopReader) {
|
||||
if (shard.state() == IndexShardState.CLOSED) {
|
||||
return;
|
||||
}
|
||||
if (!indexMetaData.getSettings().getAsBoolean(INDEX_WARMER_ENABLED, settings.getAsBoolean(INDEX_WARMER_ENABLED, true))) {
|
||||
return;
|
||||
}
|
||||
IndexService indexService = indicesService.indexService(context.shardId().index().name());
|
||||
if (indexService == null) {
|
||||
return;
|
||||
}
|
||||
final IndexShard indexShard = indexService.getShardOrNull(context.shardId().id());
|
||||
if (indexShard == null) {
|
||||
final IndexMetaData indexMetaData = settings.getIndexMetaData();
|
||||
final Settings indexSettings = settings.getSettings();
|
||||
if (!indexSettings.getAsBoolean(INDEX_WARMER_ENABLED, settings.getNodeSettings().getAsBoolean(INDEX_WARMER_ENABLED, true))) {
|
||||
return;
|
||||
}
|
||||
if (logger.isTraceEnabled()) {
|
||||
if (topReader) {
|
||||
logger.trace("[{}][{}] top warming [{}]", context.shardId().index().name(), context.shardId().id(), context);
|
||||
if (isTopReader) {
|
||||
logger.trace("{} top warming [{}]", shard.shardId(), searcher.reader());
|
||||
} 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();
|
||||
final List<TerminationHandle> terminationHandles = new ArrayList<>();
|
||||
// get a handle on pending tasks
|
||||
for (final Listener listener : listeners) {
|
||||
if (topReader) {
|
||||
terminationHandles.add(listener.warmTopReader(indexShard, indexMetaData, context, threadPool));
|
||||
if (isTopReader) {
|
||||
terminationHandles.add(listener.warmTopReader(shard, searcher));
|
||||
} else {
|
||||
terminationHandles.add(listener.warmNewReaders(indexShard, indexMetaData, context, threadPool));
|
||||
terminationHandles.add(listener.warmNewReaders(shard, searcher));
|
||||
}
|
||||
}
|
||||
// wait for termination
|
||||
|
@ -116,7 +92,7 @@ public final class IndicesWarmer extends AbstractComponent {
|
|||
terminationHandle.awaitTermination();
|
||||
} catch (InterruptedException e) {
|
||||
Thread.currentThread().interrupt();
|
||||
if (topReader) {
|
||||
if (isTopReader) {
|
||||
logger.warn("top warming has been interrupted", e);
|
||||
} else {
|
||||
logger.warn("warming has been interrupted", e);
|
||||
|
@ -125,69 +101,36 @@ public final class IndicesWarmer extends AbstractComponent {
|
|||
}
|
||||
}
|
||||
long took = System.nanoTime() - time;
|
||||
indexShard.warmerService().onPostWarm(took);
|
||||
if (indexShard.warmerService().logger().isTraceEnabled()) {
|
||||
if (topReader) {
|
||||
indexShard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS));
|
||||
shard.warmerService().onPostWarm(took);
|
||||
if (shard.warmerService().logger().isTraceEnabled()) {
|
||||
if (isTopReader) {
|
||||
shard.warmerService().logger().trace("top warming took [{}]", new TimeValue(took, TimeUnit.NANOSECONDS));
|
||||
} 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. */
|
||||
public interface TerminationHandle {
|
||||
|
||||
public static TerminationHandle NO_WAIT = new TerminationHandle() {
|
||||
@Override
|
||||
public void awaitTermination() {}
|
||||
};
|
||||
TerminationHandle NO_WAIT = () -> {};
|
||||
|
||||
/** Wait until execution of the warm-up action completes. */
|
||||
void awaitTermination() throws InterruptedException;
|
||||
}
|
||||
public static abstract class Listener {
|
||||
|
||||
public String executor() {
|
||||
return ThreadPool.Names.WARMER;
|
||||
}
|
||||
|
||||
public interface Listener {
|
||||
/** 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -68,7 +68,6 @@ import org.elasticsearch.index.shard.IndexShard;
|
|||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.IndicesWarmer.TerminationHandle;
|
||||
import org.elasticsearch.indices.IndicesWarmer.WarmerContext;
|
||||
import org.elasticsearch.indices.cache.request.IndicesRequestCache;
|
||||
import org.elasticsearch.node.settings.NodeSettingsService;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
|
@ -180,8 +179,8 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
|
||||
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval);
|
||||
|
||||
this.indicesWarmer.addListener(new NormsWarmer());
|
||||
this.indicesWarmer.addListener(new FieldDataWarmer());
|
||||
this.indicesWarmer.addListener(new NormsWarmer(indicesWarmer));
|
||||
this.indicesWarmer.addListener(new FieldDataWarmer(indicesWarmer));
|
||||
this.indicesWarmer.addListener(new SearchWarmer());
|
||||
|
||||
defaultSearchTimeout = settings.getAsTime(DEFAULT_SEARCH_TIMEOUT, NO_TIMEOUT);
|
||||
|
@ -949,11 +948,15 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
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
|
||||
public TerminationHandle warmNewReaders(final IndexShard indexShard, IndexMetaData indexMetaData, final WarmerContext context, ThreadPool threadPool) {
|
||||
final Loading defaultLoading = Loading.parse(indexMetaData.getSettings().get(NORMS_LOADING_KEY), Loading.LAZY);
|
||||
public TerminationHandle warmNewReaders(final IndexShard indexShard, final Engine.Searcher searcher) {
|
||||
final Loading defaultLoading = Loading.parse(indexShard.getIndexSettings().getSettings().get(NORMS_LOADING_KEY), Loading.LAZY);
|
||||
final MapperService mapperService = indexShard.mapperService();
|
||||
final ObjectSet<String> warmUp = new ObjectHashSet<>();
|
||||
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
|
||||
|
@ -971,14 +974,14 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
|
||||
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
|
||||
threadPool.executor(executor()).execute(new Runnable() {
|
||||
indicesWarmer.getExecutor().execute(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
for (ObjectCursor<String> stringObjectCursor : warmUp) {
|
||||
final String indexName = stringObjectCursor.value;
|
||||
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);
|
||||
if (values != null) {
|
||||
values.get(0);
|
||||
|
@ -1005,15 +1008,21 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
}
|
||||
|
||||
@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;
|
||||
}
|
||||
}
|
||||
|
||||
static class FieldDataWarmer extends IndicesWarmer.Listener {
|
||||
static class FieldDataWarmer implements IndicesWarmer.Listener {
|
||||
|
||||
private final IndicesWarmer indicesWarmer;
|
||||
|
||||
public FieldDataWarmer(IndicesWarmer indicesWarmer) {
|
||||
this.indicesWarmer = indicesWarmer;
|
||||
}
|
||||
|
||||
@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 Map<String, MappedFieldType> warmUp = new HashMap<>();
|
||||
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
|
||||
|
@ -1048,9 +1057,9 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
}
|
||||
}
|
||||
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
|
||||
final Executor executor = threadPool.executor(executor());
|
||||
final CountDownLatch latch = new CountDownLatch(context.searcher().reader().leaves().size() * warmUp.size());
|
||||
for (final LeafReaderContext ctx : context.searcher().reader().leaves()) {
|
||||
final Executor executor = indicesWarmer.getExecutor();
|
||||
final CountDownLatch latch = new CountDownLatch(searcher.reader().leaves().size() * warmUp.size());
|
||||
for (final LeafReaderContext ctx : searcher.reader().leaves()) {
|
||||
for (final MappedFieldType fieldType : warmUp.values()) {
|
||||
executor.execute(new Runnable() {
|
||||
|
||||
|
@ -1081,7 +1090,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
}
|
||||
|
||||
@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 Map<String, MappedFieldType> warmUpGlobalOrdinals = new HashMap<>();
|
||||
for (DocumentMapper docMapper : mapperService.docMappers(false)) {
|
||||
|
@ -1114,7 +1123,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
}
|
||||
}
|
||||
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
|
||||
final Executor executor = threadPool.executor(executor());
|
||||
final Executor executor = indicesWarmer.getExecutor();
|
||||
final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size());
|
||||
for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) {
|
||||
executor.execute(new Runnable() {
|
||||
|
@ -1123,7 +1132,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
try {
|
||||
final long start = System.nanoTime();
|
||||
IndexFieldData.Global ifd = indexFieldDataService.getForField(fieldType);
|
||||
ifd.loadGlobal(context.getDirectoryReader());
|
||||
ifd.loadGlobal(searcher.getDirectoryReader());
|
||||
if (indexShard.warmerService().logger().isTraceEnabled()) {
|
||||
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
|
||||
public TerminationHandle warmNewReaders(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) {
|
||||
return internalWarm(indexShard, indexMetaData, context, threadPool, false);
|
||||
public TerminationHandle warmNewReaders(IndexShard indexShard, final Engine.Searcher searcher) {
|
||||
return internalWarm(indexShard, searcher, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public TerminationHandle warmTopReader(IndexShard indexShard, IndexMetaData indexMetaData, WarmerContext context, ThreadPool threadPool) {
|
||||
return internalWarm(indexShard, indexMetaData, context, threadPool, true);
|
||||
public TerminationHandle warmTopReader(IndexShard indexShard, final Engine.Searcher searcher) {
|
||||
return internalWarm(indexShard, searcher, true);
|
||||
}
|
||||
|
||||
public TerminationHandle internalWarm(final IndexShard indexShard, final IndexMetaData indexMetaData, final IndicesWarmer.WarmerContext warmerContext, ThreadPool threadPool, final boolean top) {
|
||||
IndexWarmersMetaData custom = indexMetaData.custom(IndexWarmersMetaData.TYPE);
|
||||
public TerminationHandle internalWarm(final IndexShard indexShard, final Engine.Searcher searcher, final boolean top) {
|
||||
IndexWarmersMetaData custom = indexShard.getIndexSettings().getIndexMetaData().custom(IndexWarmersMetaData.TYPE);
|
||||
if (custom == null) {
|
||||
return TerminationHandle.NO_WAIT;
|
||||
}
|
||||
final Executor executor = threadPool.executor(executor());
|
||||
final Executor executor = indicesWarmer.getExecutor();
|
||||
final CountDownLatch latch = new CountDownLatch(custom.entries().size());
|
||||
for (final IndexWarmersMetaData.Entry entry : custom.entries()) {
|
||||
executor.execute(new Runnable() {
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
SearchContext context = null;
|
||||
executor.execute(() -> {
|
||||
SearchContext context = null;
|
||||
try {
|
||||
long now = System.nanoTime();
|
||||
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 {
|
||||
long now = System.nanoTime();
|
||||
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(), 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);
|
||||
if (context != null) {
|
||||
freeContext(context.id());
|
||||
cleanContext(context);
|
||||
}
|
||||
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 {
|
||||
if (context != null) {
|
||||
freeContext(context.id());
|
||||
cleanContext(context);
|
||||
}
|
||||
} finally {
|
||||
latch.countDown();
|
||||
}
|
||||
latch.countDown();
|
||||
}
|
||||
}
|
||||
|
||||
});
|
||||
}
|
||||
return new TerminationHandle() {
|
||||
@Override
|
||||
public void awaitTermination() throws InterruptedException {
|
||||
latch.await();
|
||||
}
|
||||
};
|
||||
return () -> latch.await();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -21,15 +21,16 @@ package org.elasticsearch.index;
|
|||
import org.apache.lucene.index.DirectoryReader;
|
||||
import org.apache.lucene.index.FieldInvertState;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.search.CollectionStatistics;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermStatistics;
|
||||
import org.apache.lucene.search.*;
|
||||
import org.apache.lucene.search.similarities.BM25Similarity;
|
||||
import org.apache.lucene.search.similarities.Similarity;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.inject.ModuleTestCase;
|
||||
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.EngineException;
|
||||
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.store.IndexStore;
|
||||
import org.elasticsearch.index.store.IndexStoreConfig;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.test.IndexSettingsModule;
|
||||
import org.elasticsearch.test.engine.MockEngineFactory;
|
||||
|
||||
|
@ -49,13 +51,13 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.function.Consumer;
|
||||
|
||||
public class IndexModuleTests extends ModuleTestCase {
|
||||
|
||||
private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null);
|
||||
public void testWrapperIsBound() {
|
||||
final Index index = new Index("foo");
|
||||
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(index, settings, Collections.EMPTY_LIST);
|
||||
IndexModule module = new IndexModule(indexSettings, null);
|
||||
assertInstanceBinding(module, IndexSearcherWrapper.class,(x) -> x == null);
|
||||
IndexModule module = new IndexModule(indexSettings, null, null, warmer);
|
||||
assertInstanceBinding(module, IndexSearcherWrapper.class, (x) -> x == null);
|
||||
module.indexSearcherWrapper = Wrapper.class;
|
||||
assertBinding(module, IndexSearcherWrapper.class, Wrapper.class);
|
||||
}
|
||||
|
@ -64,7 +66,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
final Index index = new Index("foo");
|
||||
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
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);
|
||||
module.engineFactoryImpl = MockEngineFactory.class;
|
||||
assertBinding(module, EngineFactory.class, MockEngineFactory.class);
|
||||
|
@ -74,7 +76,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
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();
|
||||
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);
|
||||
assertInstanceBinding(module, IndexStore.class, (x) -> x.getClass() == FooStore.class);
|
||||
try {
|
||||
|
@ -96,7 +98,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
final Index index = new Index("foo");
|
||||
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
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) -> {};
|
||||
module.addIndexSettingsListener(listener);
|
||||
module.addIndexEventListener(eventListener);
|
||||
|
@ -117,7 +119,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
final Index index = new Index("foo");
|
||||
final Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
|
||||
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) -> {
|
||||
};
|
||||
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.key", "there is a key")
|
||||
.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() {
|
||||
@Override
|
||||
public String name() {
|
||||
|
@ -174,7 +176,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
.put("index.similarity.my_similarity.type", "test_similarity")
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
.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 {
|
||||
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService);
|
||||
} catch (IllegalArgumentException ex) {
|
||||
|
@ -188,7 +190,7 @@ public class IndexModuleTests extends ModuleTestCase {
|
|||
.put("index.similarity.my_similarity.foo", "bar")
|
||||
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
|
||||
.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 {
|
||||
assertInstanceBinding(module, SimilarityService.class, (inst) -> inst instanceof SimilarityService);
|
||||
} 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 final Similarity delegate = new BM25Similarity();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -37,12 +37,12 @@ import org.apache.lucene.store.RAMDirectory;
|
|||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.BitSet;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.IndexSettingsModule;
|
||||
|
||||
|
@ -56,6 +56,8 @@ import static org.hamcrest.Matchers.equalTo;
|
|||
public class BitSetFilterCacheTests extends ESTestCase {
|
||||
|
||||
private static final IndexSettings INDEX_SETTINGS = IndexSettingsModule.newIndexSettings(new Index("test"), Settings.EMPTY, Collections.emptyList());
|
||||
private final IndicesWarmer warmer = new IndicesWarmer(Settings.EMPTY, null);
|
||||
|
||||
|
||||
private static int matchCount(BitSetProducer producer, IndexReader reader) throws IOException {
|
||||
int count = 0;
|
||||
|
@ -91,7 +93,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
IndexReader reader = DirectoryReader.open(writer, false);
|
||||
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")));
|
||||
assertThat(matchCount(filter, reader), equalTo(3));
|
||||
|
||||
|
@ -134,7 +136,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
final AtomicInteger onCacheCalls = 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() {
|
||||
@Override
|
||||
public void onCache(ShardId shardId, Accountable accountable) {
|
||||
|
@ -173,7 +175,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testSetListenerTwice() {
|
||||
final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS);
|
||||
final BitsetFilterCache cache = new BitsetFilterCache(INDEX_SETTINGS, warmer);
|
||||
cache.setListener(new BitsetFilterCache.Listener() {
|
||||
|
||||
@Override
|
||||
|
|
|
@ -60,13 +60,16 @@ import org.elasticsearch.env.EnvironmentModule;
|
|||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
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.query.functionscore.ScoreFunctionParser;
|
||||
import org.elasticsearch.index.query.functionscore.ScoreFunctionParserMapper;
|
||||
import org.elasticsearch.index.query.support.QueryParsers;
|
||||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.IndicesModule;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.analysis.IndicesAnalysisService;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
|
@ -213,13 +216,16 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
|
|||
}
|
||||
},
|
||||
new IndexSettingsModule(index, indexSettings),
|
||||
new IndexCacheModule(indexSettings),
|
||||
new AnalysisModule(indexSettings, new IndicesAnalysisService(indexSettings)),
|
||||
new AbstractModule() {
|
||||
@Override
|
||||
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);
|
||||
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);
|
||||
Multibinder.newSetBinder(binder(), ScoreFunctionParser.class);
|
||||
bind(ScoreFunctionParserMapper.class).asEagerSingleton();
|
||||
|
|
|
@ -37,11 +37,15 @@ import org.elasticsearch.common.xcontent.XContentParser;
|
|||
import org.elasticsearch.env.Environment;
|
||||
import org.elasticsearch.env.EnvironmentModule;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
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.similarity.SimilarityService;
|
||||
import org.elasticsearch.indices.IndicesModule;
|
||||
import org.elasticsearch.indices.IndicesWarmer;
|
||||
import org.elasticsearch.indices.analysis.IndicesAnalysisService;
|
||||
import org.elasticsearch.indices.breaker.CircuitBreakerService;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
|
@ -95,12 +99,15 @@ public class TemplateQueryParserTests extends ESTestCase {
|
|||
},
|
||||
new ScriptModule(settings),
|
||||
new IndexSettingsModule(index, settings),
|
||||
new IndexCacheModule(settings),
|
||||
new AnalysisModule(settings, new IndicesAnalysisService(settings)),
|
||||
new AbstractModule() {
|
||||
@Override
|
||||
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(Client.class).toInstance(proxy); // not needed here
|
||||
Multibinder.newSetBinder(binder(), ScoreFunctionParser.class);
|
||||
|
|
|
@ -38,8 +38,8 @@ import org.elasticsearch.common.bytes.BytesReference;
|
|||
import org.elasticsearch.common.io.stream.BytesStreamOutput;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.index.VersionType;
|
||||
import org.elasticsearch.index.cache.IndexCacheModule;
|
||||
import org.elasticsearch.index.cache.query.QueryCacheStats;
|
||||
import org.elasticsearch.index.engine.VersionConflictEngineException;
|
||||
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
|
||||
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
|
||||
.put(IndicesRequestCache.INDICES_CACHE_REQUEST_CLEAN_INTERVAL, "1ms")
|
||||
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
|
||||
.put(IndexModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
|
||||
.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -30,7 +30,7 @@ import org.elasticsearch.common.lucene.search.function.CombineFunction;
|
|||
import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
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.query.HasChildQueryBuilder;
|
||||
import org.elasticsearch.index.query.IdsQueryBuilder;
|
||||
|
@ -97,8 +97,8 @@ public class ChildQuerySearchIT extends ESIntegTestCase {
|
|||
protected Settings nodeSettings(int nodeOrdinal) {
|
||||
return Settings.settingsBuilder().put(super.nodeSettings(nodeOrdinal))
|
||||
// aggressive filter caching so that we can assert on the filter cache size
|
||||
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
|
||||
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
|
||||
.put(IndexModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.build();
|
||||
}
|
||||
|
||||
|
|
|
@ -62,8 +62,8 @@ import org.elasticsearch.common.unit.TimeValue;
|
|||
import org.elasticsearch.common.util.concurrent.EsExecutors;
|
||||
import org.elasticsearch.env.NodeEnvironment;
|
||||
import org.elasticsearch.http.HttpServerTransport;
|
||||
import org.elasticsearch.index.IndexModule;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.cache.IndexCacheModule;
|
||||
import org.elasticsearch.index.engine.CommitStats;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
|
@ -446,11 +446,11 @@ public final class InternalTestCluster extends TestCluster {
|
|||
}
|
||||
|
||||
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()) {
|
||||
builder.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, random.nextBoolean());
|
||||
builder.put(IndexModule.QUERY_CACHE_EVERYTHING, random.nextBoolean());
|
||||
}
|
||||
|
||||
if (random.nextBoolean()) {
|
||||
|
|
|
@ -21,7 +21,7 @@ package org.elasticsearch.messy.tests;
|
|||
|
||||
import org.elasticsearch.action.search.SearchResponse;
|
||||
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.script.Script;
|
||||
import org.elasticsearch.script.ScriptService.ScriptType;
|
||||
|
@ -53,8 +53,8 @@ public class ScriptQuerySearchTests extends ESIntegTestCase {
|
|||
protected Settings nodeSettings(int 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
|
||||
.put(IndexCacheModule.QUERY_CACHE_TYPE, IndexCacheModule.INDEX_QUERY_CACHE)
|
||||
.put(IndexCacheModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.put(IndexModule.QUERY_CACHE_TYPE, IndexModule.INDEX_QUERY_CACHE)
|
||||
.put(IndexModule.QUERY_CACHE_EVERYTHING, true)
|
||||
.build();
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue