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:
Simon Willnauer 2017-07-26 20:03:42 +02:00 committed by GitHub
parent 6d02b45f10
commit b72c71083c
21 changed files with 142 additions and 134 deletions

View File

@ -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" />

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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;
}
}

View File

@ -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(() -> {

View File

@ -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;
}

View File

@ -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));
}
}
}
}

View File

@ -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");
}

View File

@ -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);

View File

@ -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;

View File

@ -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));

View File

@ -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 {

View File

@ -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));
}
}

View File

@ -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();

View File

@ -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

View File

@ -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;
}

View File

@ -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;

View File

@ -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) {

View File

@ -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);

View File

@ -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 {

View File

@ -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