Cleanup IndexFieldData visibility (#25900)
Today we expose `IndexFieldDataService` outside of IndexService to do maintenance or lookup field data in different ways. Yet, we have a streamlined way to access IndexFieldData via `QueryShardContext` that should encapsulate all access to it. This also ensures that we control all other functionality like cache clearing etc. This change also removes the `recycler` option from `ClearIndicesCacheRequest` this option is a no-op and should have been removed long ago.
This commit is contained in:
parent
6d02b45f10
commit
b72c71083c
|
@ -544,7 +544,6 @@
|
|||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]deps[/\\]joda[/\\]SimpleJodaTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]BlockingClusterStatePublishResponseHandlerTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]discovery[/\\]zen[/\\]ZenDiscoveryUnitTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]document[/\\]DocumentActionsIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]env[/\\]EnvironmentTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]env[/\\]NodeEnvironmentTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]explain[/\\]ExplainActionIT.java" checks="LineLength" />
|
||||
|
|
|
@ -19,7 +19,9 @@
|
|||
|
||||
package org.elasticsearch.action.admin.indices.cache.clear;
|
||||
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
|
||||
|
@ -29,9 +31,8 @@ public class ClearIndicesCacheRequest extends BroadcastRequest<ClearIndicesCache
|
|||
|
||||
private boolean queryCache = false;
|
||||
private boolean fieldDataCache = false;
|
||||
private boolean recycler = false;
|
||||
private boolean requestCache = false;
|
||||
private String[] fields = null;
|
||||
private String[] fields = Strings.EMPTY_ARRAY;
|
||||
|
||||
|
||||
public ClearIndicesCacheRequest() {
|
||||
|
@ -69,7 +70,7 @@ public class ClearIndicesCacheRequest extends BroadcastRequest<ClearIndicesCache
|
|||
}
|
||||
|
||||
public ClearIndicesCacheRequest fields(String... fields) {
|
||||
this.fields = fields;
|
||||
this.fields = fields == null ? Strings.EMPTY_ARRAY : fields;
|
||||
return this;
|
||||
}
|
||||
|
||||
|
@ -77,21 +78,14 @@ public class ClearIndicesCacheRequest extends BroadcastRequest<ClearIndicesCache
|
|||
return this.fields;
|
||||
}
|
||||
|
||||
public ClearIndicesCacheRequest recycler(boolean recycler) {
|
||||
this.recycler = recycler;
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean recycler() {
|
||||
return this.recycler;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
super.readFrom(in);
|
||||
queryCache = in.readBoolean();
|
||||
fieldDataCache = in.readBoolean();
|
||||
recycler = in.readBoolean();
|
||||
if (in.getVersion().before(Version.V_6_0_0_beta1)) {
|
||||
in.readBoolean(); // recycler
|
||||
}
|
||||
fields = in.readStringArray();
|
||||
requestCache = in.readBoolean();
|
||||
}
|
||||
|
@ -101,7 +95,9 @@ public class ClearIndicesCacheRequest extends BroadcastRequest<ClearIndicesCache
|
|||
super.writeTo(out);
|
||||
out.writeBoolean(queryCache);
|
||||
out.writeBoolean(fieldDataCache);
|
||||
out.writeBoolean(recycler);
|
||||
if (out.getVersion().before(Version.V_6_0_0_beta1)) {
|
||||
out.writeBoolean(false); // recycler
|
||||
}
|
||||
out.writeStringArrayNullable(fields);
|
||||
out.writeBoolean(requestCache);
|
||||
}
|
||||
|
|
|
@ -32,8 +32,6 @@ import org.elasticsearch.cluster.service.ClusterService;
|
|||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
@ -45,7 +43,8 @@ import java.util.List;
|
|||
/**
|
||||
* Indices clear cache action.
|
||||
*/
|
||||
public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAction<ClearIndicesCacheRequest, ClearIndicesCacheResponse, TransportBroadcastByNodeAction.EmptyResult> {
|
||||
public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAction<ClearIndicesCacheRequest, ClearIndicesCacheResponse,
|
||||
TransportBroadcastByNodeAction.EmptyResult> {
|
||||
|
||||
private final IndicesService indicesService;
|
||||
|
||||
|
@ -53,8 +52,8 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
|
|||
public TransportClearIndicesCacheAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, IndicesService indicesService, ActionFilters actionFilters,
|
||||
IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
super(settings, ClearIndicesCacheAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
|
||||
ClearIndicesCacheRequest::new, ThreadPool.Names.MANAGEMENT, false);
|
||||
super(settings, ClearIndicesCacheAction.NAME, threadPool, clusterService, transportService, actionFilters,
|
||||
indexNameExpressionResolver, ClearIndicesCacheRequest::new, ThreadPool.Names.MANAGEMENT, false);
|
||||
this.indicesService = indicesService;
|
||||
}
|
||||
|
||||
|
@ -64,7 +63,9 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
|
|||
}
|
||||
|
||||
@Override
|
||||
protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
|
||||
protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, int totalShards, int successfulShards,
|
||||
int failedShards, List<EmptyResult> responses,
|
||||
List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
|
||||
return new ClearIndicesCacheResponse(totalShards, successfulShards, failedShards, shardFailures);
|
||||
}
|
||||
|
||||
|
@ -77,46 +78,8 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
|
|||
|
||||
@Override
|
||||
protected EmptyResult shardOperation(ClearIndicesCacheRequest request, ShardRouting shardRouting) {
|
||||
IndexService service = indicesService.indexService(shardRouting.index());
|
||||
if (service != null) {
|
||||
IndexShard shard = service.getShardOrNull(shardRouting.id());
|
||||
boolean clearedAtLeastOne = false;
|
||||
if (request.queryCache()) {
|
||||
clearedAtLeastOne = true;
|
||||
service.cache().query().clear("api");
|
||||
}
|
||||
if (request.fieldDataCache()) {
|
||||
clearedAtLeastOne = true;
|
||||
if (request.fields() == null || request.fields().length == 0) {
|
||||
service.fieldData().clear();
|
||||
} else {
|
||||
for (String field : request.fields()) {
|
||||
service.fieldData().clearField(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (request.requestCache()) {
|
||||
clearedAtLeastOne = true;
|
||||
indicesService.clearRequestCache(shard);
|
||||
}
|
||||
if (request.recycler()) {
|
||||
logger.debug("Clear CacheRecycler on index [{}]", service.index());
|
||||
clearedAtLeastOne = true;
|
||||
// cacheRecycler.clear();
|
||||
}
|
||||
if (!clearedAtLeastOne) {
|
||||
if (request.fields() != null && request.fields().length > 0) {
|
||||
// only clear caches relating to the specified fields
|
||||
for (String field : request.fields()) {
|
||||
service.fieldData().clearField(field);
|
||||
}
|
||||
} else {
|
||||
service.cache().clear("api");
|
||||
service.fieldData().clear();
|
||||
indicesService.clearRequestCache(shard);
|
||||
}
|
||||
}
|
||||
}
|
||||
indicesService.clearIndexShardCache(shardRouting.shardId(), request.queryCache(), request.fieldDataCache(), request.requestCache(),
|
||||
request.fields());
|
||||
return EmptyResult.INSTANCE;
|
||||
}
|
||||
|
||||
|
|
|
@ -172,7 +172,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
|||
this.indexStore = indexStore;
|
||||
indexFieldData.setListener(new FieldDataCacheListener(this));
|
||||
this.bitsetFilterCache = new BitsetFilterCache(indexSettings, new BitsetCacheListener(this));
|
||||
this.warmer = new IndexWarmer(indexSettings.getSettings(), threadPool,
|
||||
this.warmer = new IndexWarmer(indexSettings.getSettings(), threadPool, indexFieldData,
|
||||
bitsetFilterCache.createListener(threadPool));
|
||||
this.indexCache = new IndexCache(indexSettings, queryCache, bitsetFilterCache);
|
||||
this.engineFactory = engineFactory;
|
||||
|
@ -231,10 +231,6 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
|||
return indexCache;
|
||||
}
|
||||
|
||||
public IndexFieldDataService fieldData() {
|
||||
return indexFieldData;
|
||||
}
|
||||
|
||||
public IndexAnalyzers getIndexAnalyzers() {
|
||||
return this.mapperService.getIndexAnalyzers();
|
||||
}
|
||||
|
@ -363,7 +359,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
|||
store = new Store(shardId, this.indexSettings, indexStore.newDirectoryService(path), lock,
|
||||
new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)));
|
||||
indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier,
|
||||
indexCache, mapperService, similarityService, indexFieldData, engineFactory,
|
||||
indexCache, mapperService, similarityService, engineFactory,
|
||||
eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer,
|
||||
searchOperationListeners, indexingOperationListeners);
|
||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||
|
@ -892,4 +888,37 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
|||
return fsyncTask;
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the caches for the given shard id if the shard is still allocated on this node
|
||||
*/
|
||||
public boolean clearCaches(boolean queryCache, boolean fieldDataCache, String...fields) {
|
||||
boolean clearedAtLeastOne = false;
|
||||
if (queryCache) {
|
||||
clearedAtLeastOne = true;
|
||||
indexCache.query().clear("api");
|
||||
}
|
||||
if (fieldDataCache) {
|
||||
clearedAtLeastOne = true;
|
||||
if (fields.length == 0) {
|
||||
indexFieldData.clear();
|
||||
} else {
|
||||
for (String field : fields) {
|
||||
indexFieldData.clearField(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
if (clearedAtLeastOne == false) {
|
||||
if (fields.length == 0) {
|
||||
indexCache.clear("api");
|
||||
indexFieldData.clear();
|
||||
} else {
|
||||
// only clear caches relating to the specified fields
|
||||
for (String field : fields) {
|
||||
indexFieldData.clearField(field);
|
||||
}
|
||||
}
|
||||
}
|
||||
return clearedAtLeastOne;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -49,11 +49,12 @@ public final class IndexWarmer extends AbstractComponent {
|
|||
|
||||
private final List<Listener> listeners;
|
||||
|
||||
IndexWarmer(Settings settings, ThreadPool threadPool, Listener... listeners) {
|
||||
IndexWarmer(Settings settings, ThreadPool threadPool, IndexFieldDataService indexFieldDataService,
|
||||
Listener... listeners) {
|
||||
super(settings);
|
||||
ArrayList<Listener> list = new ArrayList<>();
|
||||
final Executor executor = threadPool.executor(ThreadPool.Names.WARMER);
|
||||
list.add(new FieldDataWarmer(executor));
|
||||
list.add(new FieldDataWarmer(executor, indexFieldDataService));
|
||||
|
||||
Collections.addAll(list, listeners);
|
||||
this.listeners = Collections.unmodifiableList(list);
|
||||
|
@ -110,8 +111,11 @@ public final class IndexWarmer extends AbstractComponent {
|
|||
private static class FieldDataWarmer implements IndexWarmer.Listener {
|
||||
|
||||
private final Executor executor;
|
||||
FieldDataWarmer(Executor executor) {
|
||||
private final IndexFieldDataService indexFieldDataService;
|
||||
|
||||
FieldDataWarmer(Executor executor, IndexFieldDataService indexFieldDataService) {
|
||||
this.executor = executor;
|
||||
this.indexFieldDataService = indexFieldDataService;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -128,7 +132,6 @@ public final class IndexWarmer extends AbstractComponent {
|
|||
warmUpGlobalOrdinals.put(indexName, fieldType);
|
||||
}
|
||||
}
|
||||
final IndexFieldDataService indexFieldDataService = indexShard.indexFieldDataService();
|
||||
final CountDownLatch latch = new CountDownLatch(warmUpGlobalOrdinals.size());
|
||||
for (final MappedFieldType fieldType : warmUpGlobalOrdinals.values()) {
|
||||
executor.execute(() -> {
|
||||
|
|
|
@ -85,7 +85,6 @@ import org.elasticsearch.index.engine.RefreshFailedEngineException;
|
|||
import org.elasticsearch.index.engine.Segment;
|
||||
import org.elasticsearch.index.engine.SegmentsStats;
|
||||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.fielddata.ShardFieldData;
|
||||
import org.elasticsearch.index.flush.FlushStats;
|
||||
import org.elasticsearch.index.get.GetStats;
|
||||
|
@ -170,7 +169,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
private final ShardIndexWarmerService shardWarmerService;
|
||||
private final ShardRequestCache requestCacheStats;
|
||||
private final ShardFieldData shardFieldData;
|
||||
private final IndexFieldDataService indexFieldDataService;
|
||||
private final ShardBitsetFilterCache shardBitsetFilterCache;
|
||||
private final Object mutex = new Object();
|
||||
private final String checkIndexOnStartup;
|
||||
|
@ -235,7 +233,7 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
|
||||
public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store,
|
||||
Supplier<Sort> indexSortSupplier, IndexCache indexCache, MapperService mapperService, SimilarityService similarityService,
|
||||
IndexFieldDataService indexFieldDataService, @Nullable EngineFactory engineFactory,
|
||||
@Nullable EngineFactory engineFactory,
|
||||
IndexEventListener indexEventListener, IndexSearcherWrapper indexSearcherWrapper, ThreadPool threadPool, BigArrays bigArrays,
|
||||
Engine.Warmer warmer, List<SearchOperationListener> searchOperationListener, List<IndexingOperationListener> listeners) throws IOException {
|
||||
super(shardRouting.shardId(), indexSettings);
|
||||
|
@ -264,7 +262,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
this.shardWarmerService = new ShardIndexWarmerService(shardId, indexSettings);
|
||||
this.requestCacheStats = new ShardRequestCache();
|
||||
this.shardFieldData = new ShardFieldData();
|
||||
this.indexFieldDataService = indexFieldDataService;
|
||||
this.shardBitsetFilterCache = new ShardBitsetFilterCache(shardId, indexSettings);
|
||||
state = IndexShardState.CREATED;
|
||||
this.path = path;
|
||||
|
@ -320,10 +317,6 @@ public class IndexShard extends AbstractIndexShardComponent implements IndicesCl
|
|||
return shardBitsetFilterCache;
|
||||
}
|
||||
|
||||
public IndexFieldDataService indexFieldDataService() {
|
||||
return indexFieldDataService;
|
||||
}
|
||||
|
||||
public MapperService mapperService() {
|
||||
return mapperService;
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ import org.elasticsearch.cluster.routing.RecoverySource;
|
|||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.CheckedFunction;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.breaker.CircuitBreaker;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.component.AbstractLifecycleComponent;
|
||||
|
@ -1099,13 +1100,6 @@ public class IndicesService extends AbstractLifecycleComponent
|
|||
|
||||
}
|
||||
|
||||
public void clearRequestCache(IndexShard shard) {
|
||||
if (shard == null) {
|
||||
return;
|
||||
}
|
||||
indicesRequestCache.clear(new IndexShardCacheEntity(shard));
|
||||
logger.trace("{} explicit cache clear", shard.shardId());
|
||||
}
|
||||
|
||||
/**
|
||||
* Loads the cache result, computing it if needed by executing the query phase and otherwise deserializing the cached
|
||||
|
@ -1240,4 +1234,19 @@ public class IndicesService extends AbstractLifecycleComponent
|
|||
public QueryRewriteContext getRewriteContext(LongSupplier nowInMillis) {
|
||||
return new QueryRewriteContext(xContentRegistry, client, nowInMillis);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clears the caches for the given shard id if the shard is still allocated on this node
|
||||
*/
|
||||
public void clearIndexShardCache(ShardId shardId, boolean queryCache, boolean fieldDataCache, boolean requestCache,
|
||||
String...fields) {
|
||||
final IndexService service = indexService(shardId.getIndex());
|
||||
if (service != null) {
|
||||
IndexShard shard = service.getShardOrNull(shardId.id());
|
||||
final boolean clearedAtLeastOne = service.clearCaches(queryCache, fieldDataCache, fields);
|
||||
if ((requestCache || (clearedAtLeastOne == false && fields.length == 0)) && shard != null) {
|
||||
indicesRequestCache.clear(new IndexShardCacheEntity(shard));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -85,17 +85,11 @@ public class RestClearIndicesCacheAction extends BaseRestHandler {
|
|||
for (Map.Entry<String, String> entry : request.params().entrySet()) {
|
||||
if (Fields.QUERY.match(entry.getKey())) {
|
||||
clearIndicesCacheRequest.queryCache(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.queryCache()));
|
||||
}
|
||||
if (Fields.REQUEST.match(entry.getKey())) {
|
||||
} else if (Fields.REQUEST.match(entry.getKey())) {
|
||||
clearIndicesCacheRequest.requestCache(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.requestCache()));
|
||||
}
|
||||
if (Fields.FIELD_DATA.match(entry.getKey())) {
|
||||
} else if (Fields.FIELD_DATA.match(entry.getKey())) {
|
||||
clearIndicesCacheRequest.fieldDataCache(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.fieldDataCache()));
|
||||
}
|
||||
if (Fields.RECYCLER.match(entry.getKey())) {
|
||||
clearIndicesCacheRequest.recycler(request.paramAsBoolean(entry.getKey(), clearIndicesCacheRequest.recycler()));
|
||||
}
|
||||
if (Fields.FIELDS.match(entry.getKey())) {
|
||||
} else if (Fields.FIELDS.match(entry.getKey())) {
|
||||
clearIndicesCacheRequest.fields(request.paramAsStringArray(entry.getKey(), clearIndicesCacheRequest.fields()));
|
||||
}
|
||||
}
|
||||
|
@ -107,7 +101,6 @@ public class RestClearIndicesCacheAction extends BaseRestHandler {
|
|||
public static final ParseField QUERY = new ParseField("query", "filter", "filter_cache");
|
||||
public static final ParseField REQUEST = new ParseField("request", "request_cache");
|
||||
public static final ParseField FIELD_DATA = new ParseField("field_data", "fielddata");
|
||||
public static final ParseField RECYCLER = new ParseField("recycler");
|
||||
public static final ParseField FIELDS = new ParseField("fields");
|
||||
}
|
||||
|
||||
|
|
|
@ -493,7 +493,6 @@ final class DefaultSearchContext extends SearchContext {
|
|||
return indexService.cache().bitsetFilterCache();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public <IFD extends IndexFieldData<?>> IFD getForField(MappedFieldType fieldType) {
|
||||
return queryShardContext.getForField(fieldType);
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.apache.lucene.util.LuceneTestCase.AwaitsFix;
|
||||
import org.elasticsearch.action.admin.indices.alias.Alias;
|
||||
import org.elasticsearch.action.admin.indices.analyze.AnalyzeAction;
|
||||
import org.elasticsearch.action.admin.indices.analyze.AnalyzeRequest;
|
||||
|
|
|
@ -81,7 +81,8 @@ public class DocumentActionsIT extends ESIntegTestCase {
|
|||
assertThat(indexExists("test1234565"), equalTo(false));
|
||||
|
||||
logger.info("Clearing cache");
|
||||
ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().clearCache(clearIndicesCacheRequest("test").recycler(true).fieldDataCache(true).queryCache(true)).actionGet();
|
||||
ClearIndicesCacheResponse clearIndicesCacheResponse = client().admin().indices().clearCache(clearIndicesCacheRequest("test")
|
||||
.fieldDataCache(true).queryCache(true)).actionGet();
|
||||
assertNoFailures(clearIndicesCacheResponse);
|
||||
assertThat(clearIndicesCacheResponse.getSuccessfulShards(), equalTo(numShards.totalNumShards));
|
||||
|
||||
|
@ -160,7 +161,8 @@ public class DocumentActionsIT extends ESIntegTestCase {
|
|||
// check count
|
||||
for (int i = 0; i < 5; i++) {
|
||||
// test successful
|
||||
SearchResponse countResponse = client().prepareSearch("test").setSize(0).setQuery(termQuery("_type", "type1")).execute().actionGet();
|
||||
SearchResponse countResponse = client().prepareSearch("test").setSize(0).setQuery(termQuery("_type", "type1"))
|
||||
.execute().actionGet();
|
||||
assertNoFailures(countResponse);
|
||||
assertThat(countResponse.getHits().getTotalHits(), equalTo(2L));
|
||||
assertThat(countResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries));
|
||||
|
@ -168,7 +170,8 @@ public class DocumentActionsIT extends ESIntegTestCase {
|
|||
|
||||
// count with no query is a match all one
|
||||
countResponse = client().prepareSearch("test").setSize(0).execute().actionGet();
|
||||
assertThat("Failures " + countResponse.getShardFailures(), countResponse.getShardFailures() == null ? 0 : countResponse.getShardFailures().length, equalTo(0));
|
||||
assertThat("Failures " + countResponse.getShardFailures(), countResponse.getShardFailures() == null ? 0
|
||||
: countResponse.getShardFailures().length, equalTo(0));
|
||||
assertThat(countResponse.getHits().getTotalHits(), equalTo(2L));
|
||||
assertThat(countResponse.getSuccessfulShards(), equalTo(numShards.numPrimaries));
|
||||
assertThat(countResponse.getFailedShards(), equalTo(0));
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.elasticsearch.index.mapper.MapperService;
|
|||
import org.elasticsearch.index.mapper.NumberFieldMapper;
|
||||
import org.elasticsearch.index.mapper.ParentFieldMapper;
|
||||
import org.elasticsearch.index.mapper.TextFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
|
||||
|
@ -68,12 +69,13 @@ import static org.hamcrest.Matchers.sameInstance;
|
|||
public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
|
||||
|
||||
protected IndexService indexService;
|
||||
protected IndexFieldDataService ifdService;
|
||||
protected MapperService mapperService;
|
||||
protected IndexWriter writer;
|
||||
protected List<LeafReaderContext> readerContexts = null;
|
||||
protected DirectoryReader topLevelReader = null;
|
||||
protected IndicesFieldDataCache indicesFieldDataCache;
|
||||
protected QueryShardContext shardContext;
|
||||
|
||||
protected abstract String getFieldDataType();
|
||||
|
||||
protected boolean hasDocValues() {
|
||||
|
@ -129,7 +131,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
|
|||
} else {
|
||||
throw new UnsupportedOperationException(type);
|
||||
}
|
||||
return ifdService.getForField(fieldType);
|
||||
return shardContext.getForField(fieldType);
|
||||
}
|
||||
|
||||
@Before
|
||||
|
@ -137,9 +139,9 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
|
|||
indexService = createIndex("test", Settings.builder().build());
|
||||
mapperService = indexService.mapperService();
|
||||
indicesFieldDataCache = getInstanceFromNode(IndicesService.class).getIndicesFieldDataCache();
|
||||
ifdService = indexService.fieldData();
|
||||
// LogByteSizeMP to preserve doc ID order
|
||||
writer = new IndexWriter(new RAMDirectory(), new IndexWriterConfig(new StandardAnalyzer()).setMergePolicy(new LogByteSizeMergePolicy()));
|
||||
shardContext = indexService.newQueryShardContext(0, null, () -> 0, null);
|
||||
}
|
||||
|
||||
protected final List<LeafReaderContext> refreshReader() throws Exception {
|
||||
|
@ -159,6 +161,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
|
|||
topLevelReader.close();
|
||||
}
|
||||
writer.close();
|
||||
shardContext = null;
|
||||
}
|
||||
|
||||
protected Nested createNested(IndexSearcher searcher, Query parentFilter, Query childFilter) throws IOException {
|
||||
|
|
|
@ -606,7 +606,7 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
|
|||
refreshReader();
|
||||
assertThat(ifd.loadGlobal(topLevelReader), not(sameInstance(globalOrdinals)));
|
||||
|
||||
ifdService.clear();
|
||||
indexService.clearCaches(false, true);
|
||||
assertThat(indicesFieldDataCache.getCache().weight(), equalTo(0L));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -63,12 +63,12 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
|
|||
final BuilderContext builderCtx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
|
||||
|
||||
{
|
||||
ifdService.clear();
|
||||
indexService.clearCaches(false, true);
|
||||
MappedFieldType ft = new TextFieldMapper.Builder("high_freq")
|
||||
.fielddata(true)
|
||||
.fielddataFrequencyFilter(0, random.nextBoolean() ? 100 : 0.5d, 0)
|
||||
.build(builderCtx).fieldType();
|
||||
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
|
||||
IndexOrdinalsFieldData fieldData = shardContext.getForField(ft);
|
||||
for (LeafReaderContext context : contexts) {
|
||||
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
|
||||
SortedSetDocValues bytesValues = loadDirect.getOrdinalsValues();
|
||||
|
@ -78,12 +78,12 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
|
|||
}
|
||||
}
|
||||
{
|
||||
ifdService.clear();
|
||||
indexService.clearCaches(false, true);
|
||||
MappedFieldType ft = new TextFieldMapper.Builder("high_freq")
|
||||
.fielddata(true)
|
||||
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, 201, 100)
|
||||
.build(builderCtx).fieldType();
|
||||
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
|
||||
IndexOrdinalsFieldData fieldData = shardContext.getForField(ft);
|
||||
for (LeafReaderContext context : contexts) {
|
||||
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
|
||||
SortedSetDocValues bytesValues = loadDirect.getOrdinalsValues();
|
||||
|
@ -93,12 +93,12 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
|
|||
}
|
||||
|
||||
{
|
||||
ifdService.clear(); // test # docs with value
|
||||
indexService.clearCaches(false, true);// test # docs with value
|
||||
MappedFieldType ft = new TextFieldMapper.Builder("med_freq")
|
||||
.fielddata(true)
|
||||
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
|
||||
.build(builderCtx).fieldType();
|
||||
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
|
||||
IndexOrdinalsFieldData fieldData = shardContext.getForField(ft);
|
||||
for (LeafReaderContext context : contexts) {
|
||||
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
|
||||
SortedSetDocValues bytesValues = loadDirect.getOrdinalsValues();
|
||||
|
@ -109,12 +109,12 @@ public class FilterFieldDataTests extends AbstractFieldDataTestCase {
|
|||
}
|
||||
|
||||
{
|
||||
ifdService.clear();
|
||||
indexService.clearCaches(false, true);
|
||||
MappedFieldType ft = new TextFieldMapper.Builder("med_freq")
|
||||
.fielddata(true)
|
||||
.fielddataFrequencyFilter(random.nextBoolean() ? 101 : 101d/200.0d, Integer.MAX_VALUE, 101)
|
||||
.build(builderCtx).fieldType();
|
||||
IndexOrdinalsFieldData fieldData = ifdService.getForField(ft);
|
||||
IndexOrdinalsFieldData fieldData = shardContext.getForField(ft);
|
||||
for (LeafReaderContext context : contexts) {
|
||||
AtomicOrdinalsFieldData loadDirect = fieldData.loadDirect(context);
|
||||
SortedSetDocValues bytesValues = loadDirect.getOrdinalsValues();
|
||||
|
|
|
@ -67,7 +67,9 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
|
|||
|
||||
public void testGetForFieldDefaults() {
|
||||
final IndexService indexService = createIndex("test");
|
||||
final IndexFieldDataService ifdService = indexService.fieldData();
|
||||
final IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
final IndexFieldDataService ifdService = new IndexFieldDataService(indexService.getIndexSettings(),
|
||||
indicesService.getIndicesFieldDataCache(), indicesService.getCircuitBreakerService(), indexService.mapperService());
|
||||
final BuilderContext ctx = new BuilderContext(indexService.getIndexSettings().getSettings(), new ContentPath(1));
|
||||
final MappedFieldType stringMapper = new KeywordFieldMapper.Builder("string").build(ctx).fieldType();
|
||||
ifdService.clear();
|
||||
|
@ -152,7 +154,22 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase {
|
|||
|
||||
public void testSetCacheListenerTwice() {
|
||||
final IndexService indexService = createIndex("test");
|
||||
IndexFieldDataService shardPrivateService = indexService.fieldData();
|
||||
final IndicesService indicesService = getInstanceFromNode(IndicesService.class);
|
||||
final IndexFieldDataService shardPrivateService = new IndexFieldDataService(indexService.getIndexSettings(),
|
||||
indicesService.getIndicesFieldDataCache(), indicesService.getCircuitBreakerService(), indexService.mapperService());
|
||||
// set it the first time...
|
||||
shardPrivateService.setListener(new IndexFieldDataCache.Listener() {
|
||||
@Override
|
||||
public void onCache(ShardId shardId, String fieldName, Accountable ramUsage) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, long sizeInBytes) {
|
||||
|
||||
}
|
||||
});
|
||||
// now set it again and make sure we fail
|
||||
try {
|
||||
shardPrivateService.setListener(new IndexFieldDataCache.Listener() {
|
||||
@Override
|
||||
|
|
|
@ -538,7 +538,7 @@ public class IndexShardIT extends ESSingleNodeTestCase {
|
|||
ShardRouting initializingShardRouting = getInitializingShardRouting(shard.routingEntry());
|
||||
IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(),
|
||||
shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), indexService.similarityService(),
|
||||
indexService.fieldData(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
||||
shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper,
|
||||
indexService.getThreadPool(), indexService.getBigArrays(), null, Collections.emptyList(), Arrays.asList(listeners));
|
||||
return newShard;
|
||||
}
|
||||
|
|
|
@ -74,6 +74,8 @@ import org.elasticsearch.index.engine.Engine;
|
|||
import org.elasticsearch.index.engine.EngineException;
|
||||
import org.elasticsearch.index.fielddata.FieldDataStats;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.mapper.IdFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.Mapping;
|
||||
|
@ -88,6 +90,8 @@ import org.elasticsearch.index.store.Store;
|
|||
import org.elasticsearch.index.translog.Translog;
|
||||
import org.elasticsearch.index.translog.TranslogTests;
|
||||
import org.elasticsearch.indices.IndicesQueryCache;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
|
||||
import org.elasticsearch.indices.recovery.RecoveryState;
|
||||
import org.elasticsearch.indices.recovery.RecoveryTarget;
|
||||
import org.elasticsearch.repositories.IndexId;
|
||||
|
@ -1721,7 +1725,11 @@ public class IndexShardTests extends IndexShardTestCase {
|
|||
|
||||
// test global ordinals are evicted
|
||||
MappedFieldType foo = shard.mapperService().fullName("foo");
|
||||
IndexFieldData.Global ifd = shard.indexFieldDataService().getForField(foo);
|
||||
IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(shard.indexSettings.getNodeSettings(),
|
||||
new IndexFieldDataCache.Listener() {});
|
||||
IndexFieldDataService indexFieldDataService = new IndexFieldDataService(shard.indexSettings, indicesFieldDataCache,
|
||||
new NoneCircuitBreakerService(), shard.mapperService());
|
||||
IndexFieldData.Global ifd = indexFieldDataService.getForField(foo);
|
||||
FieldDataStats before = shard.fieldData().stats("foo");
|
||||
assertThat(before.getMemorySizeInBytes(), equalTo(0L));
|
||||
FieldDataStats after = null;
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.elasticsearch.script.expression;
|
|||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.script.ScriptException;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.lookup.SearchLookup;
|
||||
|
@ -38,7 +39,8 @@ public class ExpressionTests extends ESSingleNodeTestCase {
|
|||
super.setUp();
|
||||
IndexService index = createIndex("test", Settings.EMPTY, "type", "d", "type=double");
|
||||
service = new ExpressionScriptEngine(Settings.EMPTY);
|
||||
lookup = new SearchLookup(index.mapperService(), index.fieldData()::getForField, null);
|
||||
QueryShardContext shardContext = index.newQueryShardContext(0, null, () -> 0, null);
|
||||
lookup = new SearchLookup(index.mapperService(), shardContext::getForField, null);
|
||||
}
|
||||
|
||||
private SearchScript.LeafFactory compile(String expression) {
|
||||
|
|
|
@ -21,6 +21,8 @@ package org.elasticsearch.painless;
|
|||
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexService;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.IndexShard;
|
||||
import org.elasticsearch.script.ExecutableScript;
|
||||
import org.elasticsearch.script.SearchScript;
|
||||
import org.elasticsearch.search.lookup.SearchLookup;
|
||||
|
@ -40,7 +42,9 @@ public class NeedsScoreTests extends ESSingleNodeTestCase {
|
|||
|
||||
PainlessScriptEngine service = new PainlessScriptEngine(Settings.EMPTY,
|
||||
Arrays.asList(SearchScript.CONTEXT, ExecutableScript.CONTEXT));
|
||||
SearchLookup lookup = new SearchLookup(index.mapperService(), index.fieldData()::getForField, null);
|
||||
|
||||
QueryShardContext shardContext = index.newQueryShardContext(0, null, () -> 0, null);
|
||||
SearchLookup lookup = new SearchLookup(index.mapperService(), shardContext::getForField, null);
|
||||
|
||||
SearchScript.Factory factory = service.compile(null, "1.2", SearchScript.CONTEXT, Collections.emptyMap());
|
||||
SearchScript.LeafFactory ss = factory.newFactory(Collections.emptyMap(), lookup);
|
||||
|
|
|
@ -53,8 +53,6 @@ import org.elasticsearch.index.cache.IndexCache;
|
|||
import org.elasticsearch.index.cache.query.DisabledQueryCache;
|
||||
import org.elasticsearch.index.engine.Engine;
|
||||
import org.elasticsearch.index.engine.EngineFactory;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataCache;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldDataService;
|
||||
import org.elasticsearch.index.mapper.IdFieldMapper;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.mapper.Mapping;
|
||||
|
@ -64,8 +62,6 @@ import org.elasticsearch.index.seqno.SequenceNumbersService;
|
|||
import org.elasticsearch.index.similarity.SimilarityService;
|
||||
import org.elasticsearch.index.store.DirectoryService;
|
||||
import org.elasticsearch.index.store.Store;
|
||||
import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
|
||||
import org.elasticsearch.indices.fielddata.cache.IndicesFieldDataCache;
|
||||
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
|
||||
import org.elasticsearch.indices.recovery.RecoveryFailedException;
|
||||
import org.elasticsearch.indices.recovery.RecoverySourceHandler;
|
||||
|
@ -77,7 +73,6 @@ import org.elasticsearch.test.DummyShardLock;
|
|||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.threadpool.TestThreadPool;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.junit.Before;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
|
@ -284,12 +279,8 @@ public abstract class IndexShardTestCase extends ESTestCase {
|
|||
};
|
||||
final Engine.Warmer warmer = searcher -> {
|
||||
};
|
||||
IndicesFieldDataCache indicesFieldDataCache = new IndicesFieldDataCache(nodeSettings, new IndexFieldDataCache.Listener() {
|
||||
});
|
||||
IndexFieldDataService indexFieldDataService = new IndexFieldDataService(indexSettings, indicesFieldDataCache,
|
||||
new NoneCircuitBreakerService(), mapperService);
|
||||
indexShard = new IndexShard(routing, indexSettings, shardPath, store, () ->null, indexCache, mapperService, similarityService,
|
||||
indexFieldDataService, engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
||||
engineFactory, indexEventListener, indexSearcherWrapper, threadPool,
|
||||
BigArrays.NON_RECYCLING_INSTANCE, warmer, Collections.emptyList(), Arrays.asList(listeners));
|
||||
success = true;
|
||||
} finally {
|
||||
|
|
|
@ -72,7 +72,6 @@ public class TestSearchContext extends SearchContext {
|
|||
|
||||
final BigArrays bigArrays;
|
||||
final IndexService indexService;
|
||||
final IndexFieldDataService indexFieldDataService;
|
||||
final BitsetFilterCache fixedBitSetFilterCache;
|
||||
final ThreadPool threadPool;
|
||||
final Map<Class<?>, Collector> queryCollectors = new HashMap<>();
|
||||
|
@ -101,7 +100,6 @@ public class TestSearchContext extends SearchContext {
|
|||
public TestSearchContext(ThreadPool threadPool, BigArrays bigArrays, IndexService indexService) {
|
||||
this.bigArrays = bigArrays.withCircuitBreaking();
|
||||
this.indexService = indexService;
|
||||
this.indexFieldDataService = indexService.fieldData();
|
||||
this.fixedBitSetFilterCache = indexService.cache().bitsetFilterCache();
|
||||
this.threadPool = threadPool;
|
||||
this.indexShard = indexService.getShardOrNull(0);
|
||||
|
@ -115,7 +113,6 @@ public class TestSearchContext extends SearchContext {
|
|||
public TestSearchContext(QueryShardContext queryShardContext, IndexShard indexShard) {
|
||||
this.bigArrays = null;
|
||||
this.indexService = null;
|
||||
this.indexFieldDataService = null;
|
||||
this.threadPool = null;
|
||||
this.fixedBitSetFilterCache = null;
|
||||
this.indexShard = indexShard;
|
||||
|
@ -309,7 +306,7 @@ public class TestSearchContext extends SearchContext {
|
|||
|
||||
@Override
|
||||
public <IFD extends IndexFieldData<?>> IFD getForField(MappedFieldType fieldType) {
|
||||
return indexFieldDataService.getForField(fieldType);
|
||||
return queryShardContext.getForField(fieldType);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue