diff --git a/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java b/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java index 30c0905dc0f..5630faf5546 100644 --- a/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java +++ b/core/src/main/java/org/elasticsearch/index/cache/bitset/BitsetFilterCache.java @@ -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.>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 filterToFbs = loadedFilters.get(coreCacheReader, new Callable>() { - @Override - public Cache call() throws Exception { - context.reader().addCoreClosedListener(BitsetFilterCache.this); - return CacheBuilder.newBuilder().build(); - } + Cache filterToFbs = loadedFilters.computeIfAbsent(coreCacheReader, key -> { + context.reader().addCoreClosedListener(BitsetFilterCache.this); + return CacheBuilder.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> 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 } diff --git a/core/src/main/java/org/elasticsearch/index/cache/request/ShardRequestCache.java b/core/src/main/java/org/elasticsearch/index/cache/request/ShardRequestCache.java index ef82e73dc4d..0f594d2faca 100644 --- a/core/src/main/java/org/elasticsearch/index/cache/request/ShardRequestCache.java +++ b/core/src/main/java/org/elasticsearch/index/cache/request/ShardRequestCache.java @@ -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 removalNotification) { - if (removalNotification.wasEvicted()) { + if (removalNotification.getRemovalReason() == RemovalNotification.RemovalReason.EVICTED) { evictionsMetric.inc(); } long dec = 0; diff --git a/core/src/main/java/org/elasticsearch/indices/cache/request/IndicesRequestCache.java b/core/src/main/java/org/elasticsearch/indices/cache/request/IndicesRequestCache.java index 4ab4691c166..21fb77ad65e 100644 --- a/core/src/main/java/org/elasticsearch/indices/cache/request/IndicesRequestCache.java +++ b/core/src/main/java/org/elasticsearch/indices/cache/request/IndicesRequestCache.java @@ -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 cacheBuilder = CacheBuilder.newBuilder() - .maximumWeight(sizeInBytes).weigher(new QueryCacheWeigher()).removalListener(this); - cacheBuilder.concurrencyLevel(concurrencyLevel); + CacheBuilder cacheBuilder = CacheBuilder.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 { - - @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 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 { + private static class Loader implements Function { 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 iterator = cache.asMap().keySet().iterator(); iterator.hasNext(); ) { + for (Iterator 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(); } diff --git a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java index 2a2aef4d691..84454544033 100644 --- a/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java +++ b/core/src/main/java/org/elasticsearch/indices/fielddata/cache/IndicesFieldDataCache.java @@ -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 cacheBuilder = CacheBuilder.newBuilder() + CacheBuilder cacheBuilder = CacheBuilder.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 { - + public static class FieldDataWeigher implements ToLongBiFunction { @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); } diff --git a/core/src/main/java/org/elasticsearch/script/ScriptService.java b/core/src/main/java/org/elasticsearch/script/ScriptService.java index 85769bcc8f1..25d019ebdd8 100644 --- a/core/src/main/java/org/elasticsearch/script/ScriptService.java +++ b/core/src/main/java/org/elasticsearch/script/ScriptService.java @@ -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 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 { - @Override public void onRemoval(RemovalNotification notification) { - if (logger.isDebugEnabled()) { - logger.debug("notifying script services of script removal due to: [{}]", notification.getCause()); - } scriptMetrics.onCacheEviction(); for (ScriptEngineService service : scriptEngines) { try { diff --git a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java index 6a9608619ff..c781a58b905 100644 --- a/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java +++ b/core/src/test/java/org/elasticsearch/index/cache/bitset/BitSetFilterCacheTests.java @@ -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 { diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractStringFieldDataTestCase.java b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractStringFieldDataTestCase.java index 6c3054f379f..117ef2f4993 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/AbstractStringFieldDataTestCase.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/AbstractStringFieldDataTestCase.java @@ -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)); } } diff --git a/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java b/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java index 234f3eada9a..4f0bd600be7 100644 --- a/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java +++ b/core/src/test/java/org/elasticsearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java @@ -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(); diff --git a/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java b/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java index 53733bca3e3..999c27eedc3 100644 --- a/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java +++ b/core/src/test/java/org/elasticsearch/test/InternalTestCluster.java @@ -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);