Remove and forbid use of com.google.common.cache.*

This commit removes and now forbids all uses of
com.google.common.cache.Cache, com.google.common.cache.CacheBuilder,
com.google.common.cache.RemovalListener,
com.google.common.cache.RemovalNotification,
com.google.common.cache.Weigher across the codebase. This is a major
step in the eventual removal of Guava as a dependency.

Relates #13224
This commit is contained in:
Jason Tedor 2015-09-30 21:45:49 +02:00
parent aa8bfeb88c
commit 5d340f5e6e
9 changed files with 111 additions and 136 deletions

View File

@ -19,11 +19,6 @@
package org.elasticsearch.index.cache.bitset;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReader;
import org.apache.lucene.index.LeafReaderContext;
@ -38,6 +33,10 @@ 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.settings.Settings;
@ -58,10 +57,11 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.io.Closeable;
import java.io.IOException;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executor;
/**
* This is a cache for {@link BitDocIdSet} based filters and is unbounded by size or time.
@ -94,10 +94,11 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
public BitsetFilterCache(Index index, @IndexSettings Settings indexSettings) {
super(index, indexSettings);
this.loadRandomAccessFiltersEagerly = indexSettings.getAsBoolean(LOAD_RANDOM_ACCESS_FILTERS_EAGERLY, true);
this.loadedFilters = CacheBuilder.newBuilder().removalListener(this).build();
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;
@ -144,36 +145,37 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
private BitSet getAndLoadIfNotPresent(final Query query, final LeafReaderContext context) throws IOException, ExecutionException {
final Object coreCacheReader = context.reader().getCoreCacheKey();
final ShardId shardId = ShardUtils.extractShardId(context.reader());
Cache<Query, Value> filterToFbs = loadedFilters.get(coreCacheReader, new Callable<Cache<Query, Value>>() {
@Override
public Cache<Query, Value> call() throws Exception {
context.reader().addCoreClosedListener(BitsetFilterCache.this);
return CacheBuilder.newBuilder().build();
}
Cache<Query, Value> filterToFbs = loadedFilters.computeIfAbsent(coreCacheReader, key -> {
context.reader().addCoreClosedListener(BitsetFilterCache.this);
return CacheBuilder.<Query, Value>builder().build();
});
return filterToFbs.get(query, () -> {
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
searcher.setQueryCache(null);
final Weight weight = searcher.createNormalizedWeight(query, false);
final DocIdSetIterator it = weight.scorer(context);
final BitSet bitSet;
if (it == null) {
bitSet = null;
} else {
bitSet = BitSet.of(it, context.reader().maxDoc());
}
Value value = new Value(bitSet, shardId);
listener.onCache(shardId, value.bitset);
return value;
return filterToFbs.computeIfAbsent(query, key -> {
try {
final IndexReaderContext topLevelContext = ReaderUtil.getTopLevelContext(context);
final IndexSearcher searcher = new IndexSearcher(topLevelContext);
searcher.setQueryCache(null);
final Weight weight = searcher.createNormalizedWeight(query, false);
final DocIdSetIterator it = weight.scorer(context);
final BitSet bitSet;
if (it == null) {
bitSet = null;
} else {
bitSet = BitSet.of(it, context.reader().maxDoc());
}
Value value = new Value(bitSet, shardId);
listener.onCache(shardId, value.bitset);
return value;
} catch (IOException e) {
throw new RuntimeException(e);
}
}).bitset;
}
@Override
public void onRemoval(RemovalNotification<Object, Cache<Query, Value>> notification) {
Object key = notification.getKey();
if (key == null) {
if (notification.getKey() == null) {
return;
}
@ -182,7 +184,7 @@ public class BitsetFilterCache extends AbstractIndexComponent implements LeafRea
return;
}
for (Value value : valueCache.asMap().values()) {
for (Value value : valueCache.values()) {
listener.onRemoval(value.shardId, value.bitset);
// if null then this means the shard has already been removed and the stats are 0 anyway for the shard this key belongs to
}

View File

@ -19,10 +19,8 @@
package org.elasticsearch.index.cache.request;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.cache.RemovalListener;
import org.elasticsearch.common.cache.RemovalNotification;
import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
@ -61,7 +59,7 @@ public class ShardRequestCache extends AbstractIndexShardComponent implements Re
@Override
public void onRemoval(RemovalNotification<IndicesRequestCache.Key, IndicesRequestCache.Value> removalNotification) {
if (removalNotification.wasEvicted()) {
if (removalNotification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED) {
evictionsMetric.inc();
}
long dec = 0;

View File

@ -21,12 +21,6 @@ package org.elasticsearch.indices.cache.request;
import com.carrotsearch.hppc.ObjectHashSet;
import com.carrotsearch.hppc.ObjectSet;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import com.google.common.cache.Weigher;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.util.Accountable;
@ -35,6 +29,10 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.bytes.BytesReference;
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.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -51,14 +49,11 @@ import org.elasticsearch.search.query.QueryPhase;
import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collection;
import java.util.Collections;
import java.util.EnumSet;
import java.util.Iterator;
import java.util.Set;
import java.util.concurrent.Callable;
import java.io.IOException;
import java.util.*;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;
import static org.elasticsearch.common.Strings.hasLength;
@ -162,25 +157,17 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
private void buildCache() {
long sizeInBytes = MemorySizeValue.parseBytesSizeValueOrHeapRatio(size, INDICES_CACHE_QUERY_SIZE).bytes();
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.newBuilder()
.maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this);
cacheBuilder.concurrencyLevel(concurrencyLevel);
CacheBuilder<Key, Value> cacheBuilder = CacheBuilder.<Key, Value>builder()
.setMaximumWeight(sizeInBytes).weigher((k, v) -> k.ramBytesUsed() + v.ramBytesUsed()).removalListener(this);
// cacheBuilder.concurrencyLevel(concurrencyLevel);
if (expire != null) {
cacheBuilder.expireAfterAccess(expire.millis(), TimeUnit.MILLISECONDS);
cacheBuilder.setExpireAfter(TimeUnit.MILLISECONDS.toNanos(expire.millis()));
}
cache = cacheBuilder.build();
}
private static class QueryCacheWeigher implements Weigher<Key, Value> {
@Override
public int weigh(Key key, Value value) {
return (int) (key.ramBytesUsed() + value.ramBytesUsed());
}
}
public void close() {
reaper.close();
cache.invalidateAll();
@ -197,9 +184,6 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
@Override
public void onRemoval(RemovalNotification<Key, Value> notification) {
if (notification.getKey() == null) {
return;
}
notification.getKey().shard.requestCache().onRemoval(notification);
}
@ -258,8 +242,8 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
public void loadIntoContext(final ShardSearchRequest request, final SearchContext context, final QueryPhase queryPhase) throws Exception {
assert canCache(request, context);
Key key = buildKey(request, context);
Loader loader = new Loader(queryPhase, context, key);
Value value = cache.get(key, loader);
Loader loader = new Loader(queryPhase, context);
Value value = cache.computeIfAbsent(key, loader);
if (loader.isLoaded()) {
key.shard.requestCache().onMiss();
// see if its the first time we see this reader, and make sure to register a cleanup key
@ -279,17 +263,15 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
}
}
private static class Loader implements Callable<Value> {
private static class Loader implements Function<Key, Value> {
private final QueryPhase queryPhase;
private final SearchContext context;
private final IndicesRequestCache.Key key;
private boolean loaded;
Loader(QueryPhase queryPhase, SearchContext context, IndicesRequestCache.Key key) {
Loader(QueryPhase queryPhase, SearchContext context) {
this.queryPhase = queryPhase;
this.context = context;
this.key = key;
}
public boolean isLoaded() {
@ -297,7 +279,7 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
}
@Override
public Value call() throws Exception {
public Value apply(Key key) {
queryPhase.execute(context);
/* BytesStreamOutput allows to pass the expected size but by default uses
@ -317,6 +299,8 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
Value value = new Value(reference, out.ramBytesUsed());
key.shard.requestCache().onCached(key, value);
return value;
} catch (IOException e) {
throw new RuntimeException(e);
}
}
}
@ -473,7 +457,7 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
if (!currentKeysToClean.isEmpty() || !currentFullClean.isEmpty()) {
CleanupKey lookupKey = new CleanupKey(null, -1);
for (Iterator<Key> iterator = cache.asMap().keySet().iterator(); iterator.hasNext(); ) {
for (Iterator<Key> iterator = cache.keys().iterator(); iterator.hasNext(); ) {
Key key = iterator.next();
if (currentFullClean.contains(key.shard)) {
iterator.remove();
@ -487,7 +471,7 @@ public class IndicesRequestCache extends AbstractComponent implements RemovalLis
}
}
cache.cleanUp();
cache.refresh();
currentKeysToClean.clear();
currentFullClean.clear();
}

View File

@ -19,12 +19,15 @@
package org.elasticsearch.indices.fielddata.cache;
import com.google.common.cache.*;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.SegmentReader;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.common.Nullable;
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.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
@ -43,6 +46,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import java.util.ArrayList;
import java.util.List;
import java.util.function.ToLongBiFunction;
/**
*/
@ -66,17 +70,11 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
this.indicesFieldDataCacheListener = indicesFieldDataCacheListener;
final String size = settings.get(INDICES_FIELDDATA_CACHE_SIZE_KEY, "-1");
final long sizeInBytes = settings.getAsMemory(INDICES_FIELDDATA_CACHE_SIZE_KEY, "-1").bytes();
CacheBuilder<Key, Accountable> cacheBuilder = CacheBuilder.newBuilder()
CacheBuilder<Key, Accountable> cacheBuilder = CacheBuilder.<Key, Accountable>builder()
.removalListener(this);
if (sizeInBytes > 0) {
cacheBuilder.maximumWeight(sizeInBytes).weigher(new FieldDataWeigher());
cacheBuilder.setMaximumWeight(sizeInBytes).weigher(new FieldDataWeigher());
}
// defaults to 4, but this is a busy map for all indices, increase it a bit by default
final int concurrencyLevel = settings.getAsInt(FIELDDATA_CACHE_CONCURRENCY_LEVEL, 16);
if (concurrencyLevel <= 0) {
throw new IllegalArgumentException("concurrency_level must be > 0 but was: " + concurrencyLevel);
}
cacheBuilder.concurrencyLevel(concurrencyLevel);
logger.debug("using size [{}] [{}]", size, new ByteSizeValue(sizeInBytes));
cache = cacheBuilder.build();
@ -108,7 +106,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final Accountable value = notification.getValue();
for (IndexFieldDataCache.Listener listener : key.listeners) {
try {
listener.onRemoval(key.shardId, indexCache.fieldNames, indexCache.fieldDataType, notification.wasEvicted(), value.ramBytesUsed());
listener.onRemoval(key.shardId, indexCache.fieldNames, indexCache.fieldDataType, notification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED, value.ramBytesUsed());
} catch (Throwable e) {
// load anyway since listeners should not throw exceptions
logger.error("Failed to call listener on field data cache unloading", e);
@ -116,10 +114,9 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
}
}
public static class FieldDataWeigher implements Weigher<Key, Accountable> {
public static class FieldDataWeigher implements ToLongBiFunction<Key, Accountable> {
@Override
public int weigh(Key key, Accountable ramUsage) {
public long applyAsLong(Key key, Accountable ramUsage) {
int weight = (int) Math.min(ramUsage.ramBytesUsed(), Integer.MAX_VALUE);
return weight == 0 ? 1 : weight;
}
@ -150,13 +147,18 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final ShardId shardId = ShardUtils.extractShardId(context.reader());
final Key key = new Key(this, context.reader().getCoreCacheKey(), shardId);
//noinspection unchecked
final Accountable accountable = cache.get(key, () -> {
final Accountable accountable = cache.computeIfAbsent(key, k -> {
context.reader().addCoreClosedListener(IndexFieldCache.this);
for (Listener listener : this.listeners) {
key.listeners.add(listener);
k.listeners.add(listener);
}
final AtomicFieldData fieldData = indexFieldData.loadDirect(context);
for (Listener listener : key.listeners) {
final AtomicFieldData fieldData;
try {
fieldData = indexFieldData.loadDirect(context);
} catch (Exception e) {
throw new RuntimeException(e);
}
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldNames, fieldDataType, fieldData);
} catch (Throwable e) {
@ -174,13 +176,18 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
final ShardId shardId = ShardUtils.extractShardId(indexReader);
final Key key = new Key(this, indexReader.getCoreCacheKey(), shardId);
//noinspection unchecked
final Accountable accountable = cache.get(key, () -> {
final Accountable accountable = cache.computeIfAbsent(key, k -> {
indexReader.addReaderClosedListener(IndexFieldCache.this);
for (Listener listener : this.listeners) {
key.listeners.add(listener);
k.listeners.add(listener);
}
final Accountable ifd = (Accountable) indexFieldData.localGlobalDirect(indexReader);
for (Listener listener : key.listeners) {
final Accountable ifd;
try {
ifd = (Accountable) indexFieldData.localGlobalDirect(indexReader);
} catch (Exception e) {
throw new RuntimeException(e);
}
for (Listener listener : k.listeners) {
try {
listener.onCache(shardId, fieldNames, fieldDataType, ifd);
} catch (Throwable e) {
@ -207,38 +214,28 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
@Override
public void clear() {
for (Key key : cache.asMap().keySet()) {
for (Key key : cache.keys()) {
if (key.indexCache.index.equals(index)) {
cache.invalidate(key);
}
}
// Note that cache invalidation in Guava does not immediately remove
// values from the cache. In the case of a cache with a rare write or
// read rate, it's possible for values to persist longer than desired.
//
// Note this is intended by the Guava developers, see:
// https://code.google.com/p/guava-libraries/wiki/CachesExplained#Eviction
// (the "When Does Cleanup Happen" section)
// We call it explicitly here since it should be a "rare" operation, and
// if a user runs it he probably wants to see memory returned as soon as
// possible
cache.cleanUp();
// force eviction
cache.refresh();
}
@Override
public void clear(String fieldName) {
for (Key key : cache.asMap().keySet()) {
for (Key key : cache.keys()) {
if (key.indexCache.index.equals(index)) {
if (key.indexCache.fieldNames.fullName().equals(fieldName)) {
cache.invalidate(key);
}
}
}
// we call cleanUp() because this is a manual operation, should happen
// we call refresh because this is a manual operation, should happen
// rarely and probably means the user wants to see memory returned as
// soon as possible
cache.cleanUp();
cache.refresh();
}
@Override
@ -305,7 +302,7 @@ public class IndicesFieldDataCache extends AbstractComponent implements RemovalL
logger.trace("running periodic field data cache cleanup");
}
try {
this.cache.cleanUp();
this.cache.refresh();
} catch (Exception e) {
logger.warn("Exception during periodic field data cache cleanup:", e);
}

View File

@ -19,13 +19,7 @@
package org.elasticsearch.script;
import java.nio.charset.StandardCharsets;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import com.google.common.collect.ImmutableMap;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.delete.DeleteRequest;
@ -43,6 +37,10 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
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.collect.Tuple;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
@ -67,13 +65,13 @@ import org.elasticsearch.watcher.ResourceWatcherService;
import java.io.Closeable;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Locale;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
/**
*
@ -153,12 +151,12 @@ public class ScriptService extends AbstractComponent implements Closeable {
this.defaultLang = settings.get(DEFAULT_SCRIPTING_LANGUAGE_SETTING, DEFAULT_LANG);
CacheBuilder cacheBuilder = CacheBuilder.newBuilder();
CacheBuilder<String, CompiledScript> cacheBuilder = CacheBuilder.builder();
if (cacheMaxSize >= 0) {
cacheBuilder.maximumSize(cacheMaxSize);
cacheBuilder.setMaximumWeight(cacheMaxSize);
}
if (cacheExpire != null) {
cacheBuilder.expireAfterAccess(cacheExpire.nanos(), TimeUnit.NANOSECONDS);
cacheBuilder.setExpireAfter(cacheExpire.nanos());
}
this.cache = cacheBuilder.removalListener(new ScriptCacheRemovalListener()).build();
@ -301,7 +299,7 @@ public class ScriptService extends AbstractComponent implements Closeable {
}
String cacheKey = getCacheKey(scriptEngineService, type == ScriptType.INLINE ? null : name, code);
CompiledScript compiledScript = cache.getIfPresent(cacheKey);
CompiledScript compiledScript = cache.get(cacheKey);
if (compiledScript == null) {
//Either an un-cached inline script or indexed script
@ -493,12 +491,8 @@ public class ScriptService extends AbstractComponent implements Closeable {
* script has been removed from the cache
*/
private class ScriptCacheRemovalListener implements RemovalListener<String, CompiledScript> {
@Override
public void onRemoval(RemovalNotification<String, CompiledScript> notification) {
if (logger.isDebugEnabled()) {
logger.debug("notifying script services of script removal due to: [{}]", notification.getCause());
}
scriptMetrics.onCacheEviction();
for (ScriptEngineService service : scriptEngines) {
try {

View File

@ -96,7 +96,7 @@ public class BitSetFilterCacheTests extends ESTestCase {
// now cached
assertThat(matchCount(filter, reader), equalTo(3));
// There are 3 segments
assertThat(cache.getLoadedFilters().size(), equalTo(3l));
assertThat(cache.getLoadedFilters().weight(), equalTo(3L));
writer.forceMerge(1);
reader.close();
@ -108,12 +108,12 @@ public class BitSetFilterCacheTests extends ESTestCase {
// now cached
assertThat(matchCount(filter, reader), equalTo(3));
// Only one segment now, so the size must be 1
assertThat(cache.getLoadedFilters().size(), equalTo(1l));
assertThat(cache.getLoadedFilters().weight(), equalTo(1L));
reader.close();
writer.close();
// There is no reference from readers and writer to any segment in the test index, so the size in the fbs cache must be 0
assertThat(cache.getLoadedFilters().size(), equalTo(0l));
assertThat(cache.getLoadedFilters().weight(), equalTo(0L));
}
public void testListener() throws IOException {

View File

@ -601,10 +601,10 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
assertThat(ifd.loadGlobal(topLevelReader), sameInstance(globalOrdinals));
// 3 b/c 1 segment level caches and 1 top level cache
// in case of doc values, we don't cache atomic FD, so only the top-level cache is there
assertThat(indicesFieldDataCache.getCache().size(), equalTo(hasDocValues() ? 1L : 4L));
assertThat(indicesFieldDataCache.getCache().weight(), equalTo(hasDocValues() ? 1L : 4L));
IndexOrdinalsFieldData cachedInstance = null;
for (Accountable ramUsage : indicesFieldDataCache.getCache().asMap().values()) {
for (Accountable ramUsage : indicesFieldDataCache.getCache().values()) {
if (ramUsage instanceof IndexOrdinalsFieldData) {
cachedInstance = (IndexOrdinalsFieldData) ramUsage;
break;
@ -613,12 +613,12 @@ public abstract class AbstractStringFieldDataTestCase extends AbstractFieldDataI
assertThat(cachedInstance, sameInstance(globalOrdinals));
topLevelReader.close();
// Now only 3 segment level entries, only the toplevel reader has been closed, but the segment readers are still used by IW
assertThat(indicesFieldDataCache.getCache().size(), equalTo(hasDocValues() ? 0L : 3L));
assertThat(indicesFieldDataCache.getCache().weight(), equalTo(hasDocValues() ? 0L : 3L));
refreshReader();
assertThat(ifd.loadGlobal(topLevelReader), not(sameInstance(globalOrdinals)));
ifdService.clear();
assertThat(indicesFieldDataCache.getCache().size(), equalTo(0l));
assertThat(indicesFieldDataCache.getCache().weight(), equalTo(0l));
}
}

View File

@ -188,7 +188,7 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
for (String node : internalCluster().getNodeNames()) {
final IndicesFieldDataCache fdCache = internalCluster().getInstance(IndicesFieldDataCache.class, node);
// Clean up the cache, ensuring that entries' listeners have been called
fdCache.getCache().cleanUp();
fdCache.getCache().refresh();
}
NodesStatsResponse nodeStats = client().admin().cluster().prepareNodesStats()
.clear().setBreaker(true).execute().actionGet();

View File

@ -1854,7 +1854,7 @@ public final class InternalTestCluster extends TestCluster {
for (NodeAndClient nodeAndClient : nodes.values()) {
final IndicesFieldDataCache fdCache = getInstanceFromNode(IndicesFieldDataCache.class, nodeAndClient.node);
// Clean up the cache, ensuring that entries' listeners have been called
fdCache.getCache().cleanUp();
fdCache.getCache().refresh();
final String name = nodeAndClient.name;
final CircuitBreakerService breakerService = getInstanceFromNode(CircuitBreakerService.class, nodeAndClient.node);