diff --git a/docs/changelog/133681.yaml b/docs/changelog/133681.yaml new file mode 100644 index 0000000000000..6231eec2a5235 --- /dev/null +++ b/docs/changelog/133681.yaml @@ -0,0 +1,6 @@ +pr: 133681 +summary: Remove `DocumentSubsetBitsetCache` locking +area: Authorization +type: bug +issues: + - 132842 diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCache.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCache.java index 0208c3f34941e..5507234cddf60 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCache.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCache.java @@ -31,12 +31,10 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.common.util.concurrent.ReleasableLock; import org.elasticsearch.core.Nullable; import org.elasticsearch.core.TimeValue; import org.elasticsearch.lucene.util.BitSets; import org.elasticsearch.lucene.util.MatchAllBitSet; -import org.elasticsearch.threadpool.ThreadPool; import java.io.Closeable; import java.io.IOException; @@ -45,15 +43,12 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.LongSupplier; /** @@ -108,18 +103,6 @@ public final class DocumentSubsetBitsetCache implements IndexReader.ClosedListen private static final BitSet NULL_MARKER = new FixedBitSet(0); - /** - * When a {@link BitSet} is evicted from {@link #bitsetCache}, we need to also remove it from {@link #keysByIndex}. - * We use a {@link ReentrantReadWriteLock} to control atomicity here - the "read" side represents potential insertions to the - * {@link #bitsetCache}, the "write" side represents removals from {@link #keysByIndex}. - * The risk (that {@link Cache} does not provide protection for) is that an entry is removed from the cache, and then immediately - * re-populated, before we process the removal event. To protect against that we need to check the state of the {@link #bitsetCache} - * but we need exclusive ("write") access while performing that check and updating the values in {@link #keysByIndex}. - */ - private final ReleasableLock cacheEvictionLock; - private final ReleasableLock cacheModificationLock; - private final ExecutorService cleanupExecutor; - private final long maxWeightBytes; private final Cache bitsetCache; private final Map> keysByIndex; @@ -128,28 +111,16 @@ public final class DocumentSubsetBitsetCache implements IndexReader.ClosedListen private final LongAdder hitsTimeInNanos = new LongAdder(); private final LongAdder missesTimeInNanos = new LongAdder(); - public DocumentSubsetBitsetCache(Settings settings, ThreadPool threadPool) { - this(settings, threadPool.executor(ThreadPool.Names.GENERIC)); - } - - // visible for testing - DocumentSubsetBitsetCache(Settings settings, ExecutorService cleanupExecutor) { - this(settings, cleanupExecutor, System::nanoTime); + public DocumentSubsetBitsetCache(Settings settings) { + this(settings, System::nanoTime); } /** * @param settings The global settings object for this node - * @param cleanupExecutor An executor on which the cache cleanup tasks can be run. Due to the way the cache is structured internally, - * it is sometimes necessary to run an asynchronous task to synchronize the internal state. * @param relativeNanoTimeProvider Provider of nanos for code that needs to measure relative time. */ // visible for testing - DocumentSubsetBitsetCache(Settings settings, ExecutorService cleanupExecutor, LongSupplier relativeNanoTimeProvider) { - final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); - this.cacheEvictionLock = new ReleasableLock(readWriteLock.writeLock()); - this.cacheModificationLock = new ReleasableLock(readWriteLock.readLock()); - this.cleanupExecutor = cleanupExecutor; - + DocumentSubsetBitsetCache(Settings settings, LongSupplier relativeNanoTimeProvider) { final TimeValue ttl = CACHE_TTL_SETTING.get(settings); this.maxWeightBytes = CACHE_SIZE_SETTING.get(settings).getBytes(); this.bitsetCache = CacheBuilder.builder() @@ -180,22 +151,15 @@ public void onClose(IndexReader.CacheKey indexKey) { private void onCacheEviction(RemovalNotification notification) { final BitsetCacheKey cacheKey = notification.getKey(); final IndexReader.CacheKey indexKey = cacheKey.indexKey; - if (keysByIndex.getOrDefault(indexKey, Set.of()).contains(cacheKey) == false) { - // If the cacheKey isn't in the lookup map, then there's nothing to synchronize - return; - } - // We push this to a background thread, so that it reduces the risk of blocking searches, but also so that the lock management is - // simpler - this callback is likely to take place on a thread that is actively adding something to the cache, and is therefore - // holding the read ("update") side of the lock. It is not possible to upgrade a read lock to a write lock ("eviction"), but we - // need to acquire that lock here. - cleanupExecutor.submit(() -> { - try (ReleasableLock ignored = cacheEvictionLock.acquire()) { - // it's possible for the key to be back in the cache if it was immediately repopulated after it was evicted, so check - if (bitsetCache.get(cacheKey) == null) { - // key is no longer in the cache, make sure it is no longer in the lookup map either. - Optional.ofNullable(keysByIndex.get(indexKey)).ifPresent(set -> set.remove(cacheKey)); - } - } + // the key is *probably* no longer in the cache, so make sure it is no longer in the lookup map. + // note: rather than locking (which destroys our throughput), we're erring on the side of tidying the keysByIndex + // structure even if some other racing thread has already added a new bitset into the cache for this same key. + // the keysByIndex structure is used in onClose (our notification from lucene that a segment has become inaccessible), + // so we might end up failing to *eagerly* invalidate a bitset -- the consequence of that would be temporarily higher + // memory use (the bitset will not be accessed, and it will still be invalidated eventually for size or ttl reasons). + keysByIndex.computeIfPresent(indexKey, (ignored, keys) -> { + keys.remove(cacheKey); + return keys.isEmpty() ? null : keys; }); } @@ -248,48 +212,46 @@ public BitSet getBitSet(final Query query, final LeafReaderContext context) thro final IndexReader.CacheKey indexKey = coreCacheHelper.getKey(); final BitsetCacheKey cacheKey = new BitsetCacheKey(indexKey, query); - try (ReleasableLock ignored = cacheModificationLock.acquire()) { - final boolean[] cacheKeyWasPresent = new boolean[] { true }; - final BitSet bitSet = bitsetCache.computeIfAbsent(cacheKey, ignore1 -> { - cacheKeyWasPresent[0] = false; - // This ensures all insertions into the set are guarded by ConcurrentHashMap's atomicity guarantees. - keysByIndex.compute(indexKey, (ignore2, set) -> { - if (set == null) { - set = ConcurrentCollections.newConcurrentSet(); - } - set.add(cacheKey); - return set; - }); - final BitSet result = computeBitSet(query, context); - if (result == null) { - // A cache loader is not allowed to return null, return a marker object instead. - return NULL_MARKER; + final boolean[] cacheKeyWasPresent = new boolean[] { true }; + final BitSet bitSet = bitsetCache.computeIfAbsent(cacheKey, ignore1 -> { + cacheKeyWasPresent[0] = false; + // This ensures all insertions into the set are guarded by ConcurrentHashMap's atomicity guarantees. + keysByIndex.compute(indexKey, (ignore2, keys) -> { + if (keys == null) { + keys = ConcurrentCollections.newConcurrentSet(); } - final long bitSetBytes = result.ramBytesUsed(); - if (bitSetBytes > this.maxWeightBytes) { - logger.warn( - "built a DLS BitSet that uses [{}] bytes; the DLS BitSet cache has a maximum size of [{}] bytes;" - + " this object cannot be cached and will need to be rebuilt for each use;" - + " consider increasing the value of [{}]", - bitSetBytes, - maxWeightBytes, - CACHE_SIZE_SETTING.getKey() - ); - } else if (bitSetBytes + bitsetCache.weight() > maxWeightBytes) { - maybeLogCacheFullWarning(); - } - return result; + keys.add(cacheKey); + return keys; }); - if (cacheKeyWasPresent[0]) { - hitsTimeInNanos.add(relativeNanoTimeProvider.getAsLong() - cacheStart); - } else { - missesTimeInNanos.add(relativeNanoTimeProvider.getAsLong() - cacheStart); + final BitSet result = computeBitSet(query, context); + if (result == null) { + // A cache loader is not allowed to return null, return a marker object instead. + return NULL_MARKER; } - if (bitSet == NULL_MARKER) { - return null; - } else { - return bitSet; + final long bitSetBytes = result.ramBytesUsed(); + if (bitSetBytes > this.maxWeightBytes) { + logger.warn( + "built a DLS BitSet that uses [{}] bytes; the DLS BitSet cache has a maximum size of [{}] bytes;" + + " this object cannot be cached and will need to be rebuilt for each use;" + + " consider increasing the value of [{}]", + bitSetBytes, + maxWeightBytes, + CACHE_SIZE_SETTING.getKey() + ); + } else if (bitSetBytes + bitsetCache.weight() > maxWeightBytes) { + maybeLogCacheFullWarning(); } + return result; + }); + if (cacheKeyWasPresent[0]) { + hitsTimeInNanos.add(relativeNanoTimeProvider.getAsLong() - cacheStart); + } else { + missesTimeInNanos.add(relativeNanoTimeProvider.getAsLong() - cacheStart); + } + if (bitSet == NULL_MARKER) { + return null; + } else { + return bitSet; } } @@ -402,26 +364,44 @@ public String toString() { } /** - * This method verifies that the two internal data structures ({@link #bitsetCache} and {@link #keysByIndex}) are consistent with one - * another. This method is only called by tests. + * This test-only method verifies that the two internal data structures ({@link #bitsetCache} and {@link #keysByIndex}) are consistent + * with one another. */ + // visible for testing void verifyInternalConsistency() { - this.bitsetCache.keys().forEach(bck -> { - final Set set = this.keysByIndex.get(bck.indexKey); - if (set == null) { - throw new IllegalStateException( - "Key [" + bck + "] is in the cache, but there is no entry for [" + bck.indexKey + "] in the lookup map" - ); - } - if (set.contains(bck) == false) { + verifyInternalConsistencyCacheToKeys(); + verifyInternalConsistencyKeysToCache(); + } + + /** + * This test-only method iterates over the {@link #bitsetCache} and checks that {@link #keysByIndex} is consistent with it. + */ + // visible for testing + void verifyInternalConsistencyCacheToKeys() { + bitsetCache.keys().forEach(cacheKey -> { + final Set keys = keysByIndex.get(cacheKey.indexKey); + if (keys == null || keys.contains(cacheKey) == false) { throw new IllegalStateException( - "Key [" + bck + "] is in the cache, but the lookup entry for [" + bck.indexKey + "] does not contain that key" + "Key [" + cacheKey + "] is in the cache, but the lookup entry for [" + cacheKey.indexKey + "] does not contain that key" ); } }); - this.keysByIndex.values().stream().flatMap(Set::stream).forEach(bck -> { - if (this.bitsetCache.get(bck) == null) { - throw new IllegalStateException("Key [" + bck + "] is in the lookup map, but is not in the cache"); + } + + /** + * This test-only method iterates over the {@link #keysByIndex} and checks that {@link #bitsetCache} is consistent with it. + */ + // visible for testing + void verifyInternalConsistencyKeysToCache() { + keysByIndex.forEach((indexKey, keys) -> { + if (keys == null || keys.isEmpty()) { + throw new IllegalStateException("The lookup entry for [" + indexKey + "] is null or empty"); + } else { + keys.forEach(cacheKey -> { + if (bitsetCache.get(cacheKey) == null) { + throw new IllegalStateException("Key [" + cacheKey + "] is in the lookup map, but is not in the cache"); + } + }); } }); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index c0da20407d2fc..22465865270f2 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -43,10 +43,6 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.MockLog; -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.Before; -import org.mockito.Mockito; import java.io.Closeable; import java.io.IOException; @@ -70,11 +66,11 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -84,17 +80,6 @@ public class DocumentSubsetBitsetCacheTests extends ESTestCase { // This value is based on the internal implementation details of lucene's FixedBitSet // If the implementation changes, this can be safely updated to match the new ram usage for a single bitset private static final long EXPECTED_BYTES_PER_BIT_SET = 56; - private ExecutorService singleThreadExecutor; - - @Before - public void setUpExecutor() { - singleThreadExecutor = Executors.newSingleThreadExecutor(); - } - - @After - public void cleanUpExecutor() { - singleThreadExecutor.shutdown(); - } public void testSameBitSetIsReturnedForIdenticalQuery() throws Exception { final DocumentSubsetBitsetCache cache = newCache(Settings.EMPTY); @@ -107,7 +92,7 @@ public void testSameBitSetIsReturnedForIdenticalQuery() throws Exception { final BitSet bitSet2 = cache.getBitSet(query2, leafContext); assertThat(bitSet2, notNullValue()); - assertThat(bitSet2, Matchers.sameInstance(bitSet1)); + assertThat(bitSet2, sameInstance(bitSet1)); }); } @@ -276,7 +261,7 @@ public void testCacheRespectsAccessTimeExpiry() throws Exception { assertThat(bitSet2, notNullValue()); // Loop until the cache has less than 2 items, which mean that something we evicted - assertThat(cache.entryCount(), Matchers.lessThan(2)); + assertThat(cache.entryCount(), lessThan(2)); }, 100, TimeUnit.MILLISECONDS); @@ -292,42 +277,28 @@ public void testIndexLookupIsClearedWhenBitSetIsEvicted() throws Exception { .put(DocumentSubsetBitsetCache.CACHE_SIZE_SETTING.getKey(), maxCacheBytes + "b") .build(); - final ExecutorService executor = mock(ExecutorService.class); - final AtomicReference runnableRef = new AtomicReference<>(); - when(executor.submit(any(Runnable.class))).thenAnswer(inv -> { - final Runnable r = (Runnable) inv.getArguments()[0]; - runnableRef.set(r); - return null; - }); - - final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings, executor); + final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings); assertThat(cache.entryCount(), equalTo(0)); assertThat(cache.ramBytesUsed(), equalTo(0L)); runTestOnIndex((searchExecutionContext, leafContext) -> { + cache.verifyInternalConsistency(); + final Query query1 = QueryBuilders.termQuery("field-1", "value-1").toQuery(searchExecutionContext); final BitSet bitSet1 = cache.getBitSet(query1, leafContext); assertThat(bitSet1, notNullValue()); + cache.verifyInternalConsistency(); final Query query2 = QueryBuilders.termQuery("field-2", "value-2").toQuery(searchExecutionContext); final BitSet bitSet2 = cache.getBitSet(query2, leafContext); assertThat(bitSet2, notNullValue()); - - // BitSet1 has been evicted now, run the cleanup... - final Runnable runnable1 = runnableRef.get(); - assertThat(runnable1, notNullValue()); - runnable1.run(); cache.verifyInternalConsistency(); - // Check that the original bitset is no longer in the cache (a new instance is returned) assertThat(cache.getBitSet(query1, leafContext), not(sameInstance(bitSet1))); - - // BitSet2 has been evicted now, run the cleanup... - final Runnable runnable2 = runnableRef.get(); - assertThat(runnable2, not(sameInstance(runnable1))); - runnable2.run(); cache.verifyInternalConsistency(); }); + + cache.verifyInternalConsistency(); } public void testCacheUnderConcurrentAccess() throws Exception { @@ -341,23 +312,12 @@ public void testCacheUnderConcurrentAccess() throws Exception { .put(DocumentSubsetBitsetCache.CACHE_SIZE_SETTING.getKey(), maxCacheBytes + "b") .build(); + final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings); + assertThat(cache.entryCount(), equalTo(0)); + assertThat(cache.ramBytesUsed(), equalTo(0L)); + final ExecutorService threads = Executors.newFixedThreadPool(concurrentThreads + 1); - final ExecutorService cleanupExecutor = Mockito.mock(ExecutorService.class); - when(cleanupExecutor.submit(any(Runnable.class))).thenAnswer(inv -> { - final Runnable runnable = (Runnable) inv.getArguments()[0]; - return threads.submit(() -> { - // Sleep for a small (random) length of time. - // This increases the likelihood that cache could have been modified between the eviction & the cleanup - Thread.sleep(randomIntBetween(1, 10)); - runnable.run(); - return null; - }); - }); try { - final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings, cleanupExecutor); - assertThat(cache.entryCount(), equalTo(0)); - assertThat(cache.ramBytesUsed(), equalTo(0L)); - runTestOnIndices(numberOfIndices, contexts -> { final CountDownLatch start = new CountDownLatch(concurrentThreads); final CountDownLatch end = new CountDownLatch(concurrentThreads); @@ -398,7 +358,7 @@ public void testCacheUnderConcurrentAccess() throws Exception { threads.shutdown(); assertTrue("Cleanup thread did not complete in expected time", threads.awaitTermination(3, TimeUnit.SECONDS)); - cache.verifyInternalConsistency(); + cache.verifyInternalConsistencyKeysToCache(); // Due to cache evictions, we must get more bitsets than fields assertThat(uniqueBitSets.size(), greaterThan(FIELD_COUNT)); @@ -411,62 +371,41 @@ public void testCacheUnderConcurrentAccess() throws Exception { } finally { threads.shutdown(); } + + cache.verifyInternalConsistencyKeysToCache(); } - public void testCleanupWorksWhenIndexIsClosing() throws Exception { + public void testCleanupWorksWhenIndexIsClosed() throws Exception { // Enough to hold slightly more than 1 bit-set in the cache final long maxCacheBytes = EXPECTED_BYTES_PER_BIT_SET + EXPECTED_BYTES_PER_BIT_SET / 2; final Settings settings = Settings.builder() .put(DocumentSubsetBitsetCache.CACHE_SIZE_SETTING.getKey(), maxCacheBytes + "b") .build(); - final ExecutorService threads = Executors.newFixedThreadPool(1); - final ExecutorService cleanupExecutor = Mockito.mock(ExecutorService.class); - final CountDownLatch cleanupReadyLatch = new CountDownLatch(1); - final CountDownLatch cleanupCompleteLatch = new CountDownLatch(1); - final CountDownLatch indexCloseLatch = new CountDownLatch(1); - final AtomicReference cleanupException = new AtomicReference<>(); - when(cleanupExecutor.submit(any(Runnable.class))).thenAnswer(inv -> { - final Runnable runnable = (Runnable) inv.getArguments()[0]; - return threads.submit(() -> { - try { - cleanupReadyLatch.countDown(); - assertTrue("index close did not completed in expected time", indexCloseLatch.await(1, TimeUnit.SECONDS)); - runnable.run(); - } catch (Throwable e) { - logger.warn("caught error in cleanup thread", e); - cleanupException.compareAndSet(null, e); - } finally { - cleanupCompleteLatch.countDown(); - } - return null; - }); - }); - final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings, cleanupExecutor); + final DocumentSubsetBitsetCache cache = new DocumentSubsetBitsetCache(settings); assertThat(cache.entryCount(), equalTo(0)); assertThat(cache.ramBytesUsed(), equalTo(0L)); - try { - runTestOnIndex((searchExecutionContext, leafContext) -> { - final Query query1 = QueryBuilders.termQuery("field-1", "value-1").toQuery(searchExecutionContext); - final BitSet bitSet1 = cache.getBitSet(query1, leafContext); - assertThat(bitSet1, notNullValue()); + runTestOnIndex((searchExecutionContext, leafContext) -> { + final Query query1 = QueryBuilders.termQuery("field-1", "value-1").toQuery(searchExecutionContext); + final BitSet bitSet1 = cache.getBitSet(query1, leafContext); + assertThat(bitSet1, notNullValue()); + cache.verifyInternalConsistency(); - // Second query should trigger a cache eviction - final Query query2 = QueryBuilders.termQuery("field-2", "value-2").toQuery(searchExecutionContext); - final BitSet bitSet2 = cache.getBitSet(query2, leafContext); - assertThat(bitSet2, notNullValue()); + // Second query should trigger a cache eviction + final Query query2 = QueryBuilders.termQuery("field-2", "value-2").toQuery(searchExecutionContext); + final BitSet bitSet2 = cache.getBitSet(query2, leafContext); + assertThat(bitSet2, notNullValue()); + cache.verifyInternalConsistency(); - final IndexReader.CacheKey indexKey = leafContext.reader().getCoreCacheHelper().getKey(); - assertTrue("cleanup did not trigger in expected time", cleanupReadyLatch.await(1, TimeUnit.SECONDS)); - cache.onClose(indexKey); - indexCloseLatch.countDown(); - assertTrue("cleanup did not complete in expected time", cleanupCompleteLatch.await(1, TimeUnit.SECONDS)); - assertThat("caught error in cleanup thread: " + cleanupException.get(), cleanupException.get(), nullValue()); - }); - } finally { - threads.shutdown(); - } + final IndexReader.CacheKey indexKey = leafContext.reader().getCoreCacheHelper().getKey(); + cache.onClose(indexKey); + cache.verifyInternalConsistency(); + + // closing an index results in the associated entries being removed from the cache (at least when single threaded) + assertThat(cache.entryCount(), equalTo(0)); + assertThat(cache.ramBytesUsed(), equalTo(0L)); + }); } public void testCacheIsPerIndex() throws Exception { @@ -496,7 +435,7 @@ public void accept(SearchExecutionContext searchExecutionContext, LeafReaderCont runTestOnIndex(consumer); } - public void testCacheClearEntriesWhenIndexIsClosed() throws Exception { + public void testCacheClearsEntriesWhenIndexIsClosed() throws Exception { final DocumentSubsetBitsetCache cache = newCache(Settings.EMPTY); assertThat(cache.entryCount(), equalTo(0)); assertThat(cache.ramBytesUsed(), equalTo(0L)); @@ -508,9 +447,13 @@ public void testCacheClearEntriesWhenIndexIsClosed() throws Exception { final BitSet bitSet = cache.getBitSet(query, leafContext); assertThat(bitSet, notNullValue()); } + cache.verifyInternalConsistency(); assertThat(cache.entryCount(), not(equalTo(0))); assertThat(cache.ramBytesUsed(), not(equalTo(0L))); }); + cache.verifyInternalConsistency(); + + // closing an index results in the associated entries being removed from the cache (at least when single threaded) assertThat(cache.entryCount(), equalTo(0)); assertThat(cache.ramBytesUsed(), equalTo(0L)); } @@ -569,20 +512,17 @@ public void testHitsMissesAndEvictionsStats() throws Exception { final Query query2 = QueryBuilders.termQuery("field-2", "value-2").toQuery(searchExecutionContext); final BitSet bitSet2 = cache.getBitSet(query2, leafContext); assertThat(bitSet2, notNullValue()); - // surprisingly, the eviction callback can call `get` on the cache (asynchronously) which causes another miss (or hit) - // so this assertion is about the current state of the code, rather than the expected or desired state. - // see https://github.com/elastic/elasticsearch/issues/132842 - expectedStats.put("misses", 3L); + expectedStats.put("misses", 2L); expectedStats.put("evictions", 1L); // underlying Cache class tracks hits/misses, but timing is in DLS cache, which is why we have `2L` here, // because DLS cache is only hit once expectedStats.put("misses_time_in_millis", 2L); - assertBusy(() -> { assertThat(cache.usageStats(), equalTo(expectedStats)); }, 200, TimeUnit.MILLISECONDS); + assertThat(cache.usageStats(), equalTo(expectedStats)); }); final Map finalStats = emptyStatsSupplier.get(); finalStats.put("hits", 1L); - finalStats.put("misses", 3L); + finalStats.put("misses", 2L); finalStats.put("evictions", 2L); finalStats.put("hits_time_in_millis", 1L); finalStats.put("misses_time_in_millis", 2L); @@ -710,6 +650,6 @@ private void runTestOnIndices(int numberIndices, CheckedConsumer TimeUnit.MILLISECONDS.toNanos(increasingMillisTime.getAndIncrement()); - return new DocumentSubsetBitsetCache(settings, singleThreadExecutor, relativeNanoTimeProvider); + return new DocumentSubsetBitsetCache(settings, relativeNanoTimeProvider); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetReaderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetReaderTests.java index 6fe271d1b05e3..cfa23bcedf143 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetReaderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetReaderTests.java @@ -31,14 +31,14 @@ import org.elasticsearch.core.IOUtils; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; -import org.hamcrest.Matchers; import org.junit.After; import org.junit.Before; import java.io.IOException; -import java.util.concurrent.Executors; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; public class DocumentSubsetReaderTests extends ESTestCase { @@ -54,7 +54,7 @@ public void setUpDirectory() { // this test and garbage not cleaned up by other tests. assertTrue(DocumentSubsetReader.NUM_DOCS_CACHE.toString(), DocumentSubsetReader.NUM_DOCS_CACHE.isEmpty()); directory = newDirectory(); - bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); + bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); } @After @@ -242,9 +242,9 @@ public void testProducesStoredFieldsReader() throws Exception { assertEquals(2, reader.leaves().size()); TestUtil.checkReader(reader); - assertThat(reader.leaves().size(), Matchers.greaterThanOrEqualTo(1)); + assertThat(reader.leaves().size(), greaterThanOrEqualTo(1)); for (LeafReaderContext context : reader.leaves()) { - assertThat(context.reader(), Matchers.instanceOf(SequentialStoredFieldsLeafReader.class)); + assertThat(context.reader(), instanceOf(SequentialStoredFieldsLeafReader.class)); SequentialStoredFieldsLeafReader lf = (SequentialStoredFieldsLeafReader) context.reader(); assertNotNull(lf.getSequentialStoredFieldsReader()); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/FieldSubsetReaderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/FieldSubsetReaderTests.java index 0a072a685fd1a..76bb146b55fcc 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/FieldSubsetReaderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/FieldSubsetReaderTests.java @@ -76,7 +76,6 @@ import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissions; import org.elasticsearch.xpack.core.security.authz.permission.FieldPermissionsDefinition; import org.elasticsearch.xpack.core.security.support.Automatons; -import org.hamcrest.Matchers; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -91,6 +90,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; /** Simple tests for this filterreader */ public class FieldSubsetReaderTests extends MapperServiceTestCase { @@ -1418,9 +1419,9 @@ public void testProducesStoredFieldsReader() throws Exception { DirectoryReader ir = FieldSubsetReader.wrap(DirectoryReader.open(iw), new CharacterRunAutomaton(automaton)); TestUtil.checkReader(ir); - assertThat(ir.leaves().size(), Matchers.greaterThanOrEqualTo(1)); + assertThat(ir.leaves().size(), greaterThanOrEqualTo(1)); for (LeafReaderContext context : ir.leaves()) { - assertThat(context.reader(), Matchers.instanceOf(SequentialStoredFieldsLeafReader.class)); + assertThat(context.reader(), instanceOf(SequentialStoredFieldsLeafReader.class)); SequentialStoredFieldsLeafReader lf = (SequentialStoredFieldsLeafReader) context.reader(); assertNotNull(lf.getSequentialStoredFieldsReader()); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java index 8461daeef5d8c..72af50b23d6d5 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java @@ -66,7 +66,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Executors; import java.util.stream.Collectors; import static org.elasticsearch.xpack.core.security.SecurityField.DOCUMENT_LEVEL_SECURITY_FEATURE; @@ -116,7 +115,7 @@ public void testDLS() throws Exception { MapperMetrics.NOOP ); SearchExecutionContext searchExecutionContext = spy(realSearchExecutionContext); - DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); + DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final MockLicenseState licenseState = mock(MockLicenseState.class); when(licenseState.isAllowed(DOCUMENT_LEVEL_SECURITY_FEATURE)).thenReturn(true); @@ -272,7 +271,7 @@ public void testDLSWithLimitedPermissions() throws Exception { MapperMetrics.NOOP ); SearchExecutionContext searchExecutionContext = spy(realSearchExecutionContext); - DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); + DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final MockLicenseState licenseState = mock(MockLicenseState.class); when(licenseState.isAllowed(DOCUMENT_LEVEL_SECURITY_FEATURE)).thenReturn(true); @@ -474,7 +473,7 @@ public void testDLSWithNestedDocs() throws Exception { DocumentPermissions.filteredBy(queries) ); - DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY, Executors.newSingleThreadExecutor()); + DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); final MockLicenseState licenseState = mock(MockLicenseState.class); when(licenseState.isAllowed(DOCUMENT_LEVEL_SECURITY_FEATURE)).thenReturn(true); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index 5fe15a857951a..0571ee831fe91 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -924,7 +924,7 @@ Collection createComponents( components.add(privilegeStore); final ReservedRolesStore reservedRolesStore = new ReservedRolesStore(Set.copyOf(INCLUDED_RESERVED_ROLES_SETTING.get(settings))); - dlsBitsetCache.set(new DocumentSubsetBitsetCache(settings, threadPool)); + dlsBitsetCache.set(new DocumentSubsetBitsetCache(settings)); final FieldPermissionsCache fieldPermissionsCache = new FieldPermissionsCache(settings); RoleDescriptor.setFieldPermissionsCache(fieldPermissionsCache); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java index 6ca47dd9807e1..530b012cb55c3 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/IndicesAndAliasesResolverTests.java @@ -61,7 +61,6 @@ import org.elasticsearch.protocol.xpack.graph.GraphExploreRequest; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.EmptyRequest; import org.elasticsearch.transport.NoSuchRemoteClusterException; import org.elasticsearch.transport.TransportRequest; @@ -256,7 +255,7 @@ public void setup() { mock(ApiKeyService.class), mock(ServiceAccountService.class), TestProjectResolvers.DEFAULT_PROJECT_ONLY, - new DocumentSubsetBitsetCache(Settings.EMPTY, mock(ThreadPool.class)), + new DocumentSubsetBitsetCache(Settings.EMPTY), RESTRICTED_INDICES, EsExecutors.DIRECT_EXECUTOR_SERVICE, rds -> {} diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java index 7a97a74cf44b8..ba0b28171bf23 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authz/store/CompositeRolesStoreTests.java @@ -3972,7 +3972,7 @@ private RoleProviders buildRolesProvider( } private DocumentSubsetBitsetCache buildBitsetCache() { - return new DocumentSubsetBitsetCache(Settings.EMPTY, mock(ThreadPool.class)); + return new DocumentSubsetBitsetCache(Settings.EMPTY); } private static class InMemoryRolesProvider implements BiConsumer, ActionListener> {