Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Increase Size and lower TTL on DLS BitSet Cache #50535

Merged
merged 6 commits into from
Jan 13, 2020
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -38,44 +38,69 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicLong;

/**
* This is a cache for {@link BitSet} instances that are used with the {@link DocumentSubsetReader}.
* It is bounded by memory size and access time.
*
* DLS uses {@link BitSet} instances to track which documents should be visible to the user ("live") and which should not ("dead").
* This means that there is a bit for each document in a Lucene index (ES shard).
* Consequently, an index with 10 million document will use more than 1Mb of bitset memory for every unique DLS query, and an index
* with 1 billion documents will use more than 100Mb of memory per DLS query.
* Because DLS supports templating queries based on user metadata, there may be many distinct queries in use for each index, even if
* there is only a single active role.
*
* The primary benefit of the cache is to avoid recalculating the "live docs" (visible documents) when a user performs multiple
* consecutive queries across one or more large indices. Given the memory examples above, the cache is only useful if it can hold at
* least 1 large (100Mb or more ) {@code BitSet} during a user's active session, and ideally should be capable of support multiple
* simultaneous users with distinct DLS queries.
*
* For this reason the default memory usage (weight) for the cache set to 10% of JVM heap ({@link #CACHE_SIZE_SETTING}), so that it
* automatically scales with the size of the Elasticsearch deployment, and can provide benefit to most use cases without needing
* customisation. On a 32Gb heap, a 10% cache would be 3.2Gb which is large enough to store BitSets representing 25 billion docs.
*
* However, because queries can be templated by user metadata and that metadata can change frequently, it is common for the
* effetively lifetime of a single DLS query to be relatively short. We do not want to sacrifice 10% of heap to a cache that is storing
* BitSets that are not longer needed, so we set the TTL on this cache to be 2 hours ({@link #CACHE_TTL_SETTING}). This time has been
* chosen so that it will retain BitSets that are in active use during a user's session, but not be an ongoing drain on memory.
*
* @see org.elasticsearch.index.cache.bitset.BitsetFilterCache
*/
public final class DocumentSubsetBitsetCache implements IndexReader.ClosedListener, Closeable, Accountable {

/**
* The TTL defaults to 1 week. We depend on the {@code max_bytes} setting to keep the cache to a sensible size, by evicting LRU
* entries, however there is benefit in reclaiming memory by expiring bitsets that have not be used for some period of time.
* Because {@link org.elasticsearch.xpack.core.security.authz.permission.IndicesPermission.Group#query} can be templated, it is
* not uncommon for a query to only be used for a relatively short period of time (e.g. because a user's metadata changed, or because
* that user is an infrequent user of Elasticsearch). This access time expiry helps free up memory in those circumstances even if the
* cache is never filled.
* The TTL defaults to 2 hours. We default to a large cache size ({@link #CACHE_SIZE_SETTING}), and aggressively
* expire unused entries so that the cache does not hold on to memory unnecessarily.
*/
static final Setting<TimeValue> CACHE_TTL_SETTING =
Setting.timeSetting("xpack.security.dls.bitset.cache.ttl", TimeValue.timeValueHours(24 * 7), Property.NodeScope);
Setting.timeSetting("xpack.security.dls.bitset.cache.ttl", TimeValue.timeValueHours(2), Property.NodeScope);

static final Setting<ByteSizeValue> CACHE_SIZE_SETTING = Setting.byteSizeSetting("xpack.security.dls.bitset.cache.size",
new ByteSizeValue(50, ByteSizeUnit.MB), Property.NodeScope);
/**
* The size defaults to 10% of heap so that it automatically scales up with larger node size
*/
static final Setting<ByteSizeValue> CACHE_SIZE_SETTING = Setting.memorySizeSetting("xpack.security.dls.bitset.cache.size",
"10%", Property.NodeScope);

private static final BitSet NULL_MARKER = new FixedBitSet(0);

private final Logger logger;
private final long maxWeightBytes;
private final Cache<BitsetCacheKey, BitSet> bitsetCache;
private final Map<IndexReader.CacheKey, Set<BitsetCacheKey>> keysByIndex;
private final AtomicLong cacheFullWarningTime;

public DocumentSubsetBitsetCache(Settings settings) {
this.logger = LogManager.getLogger(getClass());
final TimeValue ttl = CACHE_TTL_SETTING.get(settings);
final ByteSizeValue size = CACHE_SIZE_SETTING.get(settings);
this.maxWeightBytes = CACHE_SIZE_SETTING.get(settings).getBytes();
this.bitsetCache = CacheBuilder.<BitsetCacheKey, BitSet>builder()
.setExpireAfterAccess(ttl)
.setMaximumWeight(size.getBytes())
.setMaximumWeight(maxWeightBytes)
.weigher((key, bitSet) -> bitSet == NULL_MARKER ? 0 : bitSet.ramBytesUsed()).build();
this.keysByIndex = new ConcurrentHashMap<>();
this.cacheFullWarningTime = new AtomicLong(0);
}

@Override
Expand Down Expand Up @@ -148,7 +173,16 @@ public BitSet getBitSet(final Query query, final LeafReaderContext context) thro
// A cache loader is not allowed to return null, return a marker object instead.
return NULL_MARKER;
} else {
return BitSet.of(s.iterator(), context.reader().maxDoc());
final BitSet bs = BitSet.of(s.iterator(), context.reader().maxDoc());
final long bitSetBytes = bs.ramBytesUsed();
if (bitSetBytes > this.maxWeightBytes) {
logger.warn("built a DLS BitSet that uses [{}] bytes; the BitSet cache has a maximum size of [{}] bytes;" +
tvernum marked this conversation as resolved.
Show resolved Hide resolved
" 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 bs;
}
});
if (bitSet == NULL_MARKER) {
Expand All @@ -158,6 +192,20 @@ public BitSet getBitSet(final Query query, final LeafReaderContext context) thro
}
}

private void maybeLogCacheFullWarning() {
final long nextLogTime = cacheFullWarningTime.get();
final long now = System.currentTimeMillis();
if (nextLogTime > now) {
return;
}
final long nextCheck = now + TimeUnit.MINUTES.toMillis(30);
if (cacheFullWarningTime.compareAndSet(nextLogTime, nextCheck)) {
logger.info(
"the Document Level Security BitSet cache is full which may impact performance; consider increasing the value of [{}]",
CACHE_SIZE_SETTING.getKey());
}
}

public static List<Setting<?>> getSettings() {
return List.of(CACHE_TTL_SETTING, CACHE_SIZE_SETTING);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,9 @@

package org.elasticsearch.xpack.core.security.authz.accesscontrol;

import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
Expand All @@ -21,6 +24,7 @@
import org.apache.lucene.util.BitSet;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.CheckedBiConsumer;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
Expand All @@ -31,6 +35,7 @@
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.IndexSettingsModule;
import org.elasticsearch.test.MockLogAppender;
import org.hamcrest.Matchers;

import java.util.concurrent.TimeUnit;
Expand Down Expand Up @@ -138,6 +143,93 @@ public void testCacheRespectsMemoryLimit() throws Exception {
});
}

public void testLogWarningIfBitSetExceedsCacheSize() throws Exception {
// 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
final long expectedBytesPerBitSet = 56;

// Enough to hold less than 1 bit-sets in the cache
final long maxCacheBytes = expectedBytesPerBitSet - expectedBytesPerBitSet/3;
final Settings settings = Settings.builder()
.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 Logger cacheLogger = LogManager.getLogger(cache.getClass());
final MockLogAppender mockAppender = new MockLogAppender();
mockAppender.start();
try {
Loggers.addAppender(cacheLogger, mockAppender);
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation(
"[bitset too big]",
cache.getClass().getName(),
Level.WARN,
"built a DLS BitSet that uses [" + expectedBytesPerBitSet + "] bytes; the BitSet cache has a maximum size of [" +
maxCacheBytes + "] bytes; this object cannot be cached and will need to be rebuilt for each use;" +
" consider increasing the value of [xpack.security.dls.bitset.cache.size]"
));

runTestOnIndex((shardContext, leafContext) -> {
final TermQueryBuilder queryBuilder = QueryBuilders.termQuery("field-1", "value-1");
final Query query = queryBuilder.toQuery(shardContext);
final BitSet bitSet = cache.getBitSet(query, leafContext);
assertThat(bitSet, notNullValue());
assertThat(bitSet.ramBytesUsed(), equalTo(expectedBytesPerBitSet));
});

mockAppender.assertAllExpectationsMatched();
} finally {
Loggers.removeAppender(cacheLogger, mockAppender);
mockAppender.stop();
}
}

public void testLogMessageIfCacheFull() throws Exception {
// 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
final long expectedBytesPerBitSet = 56;

// Enough to hold slightly more than 1 bit-sets in the cache
final long maxCacheBytes = expectedBytesPerBitSet + expectedBytesPerBitSet/3;
final Settings settings = Settings.builder()
.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 Logger cacheLogger = LogManager.getLogger(cache.getClass());
final MockLogAppender mockAppender = new MockLogAppender();
mockAppender.start();
try {
Loggers.addAppender(cacheLogger, mockAppender);
mockAppender.addExpectation(new MockLogAppender.SeenEventExpectation(
"[cache full]",
cache.getClass().getName(),
Level.INFO,
"the Document Level Security BitSet cache is full which may impact performance;" +
" consider increasing the value of [xpack.security.dls.bitset.cache.size]"
));

runTestOnIndex((shardContext, leafContext) -> {
for (int i = 1; i <= 3; i++) {
final TermQueryBuilder queryBuilder = QueryBuilders.termQuery("field-" + i, "value-" + i);
final Query query = queryBuilder.toQuery(shardContext);
final BitSet bitSet = cache.getBitSet(query, leafContext);
assertThat(bitSet, notNullValue());
assertThat(bitSet.ramBytesUsed(), equalTo(expectedBytesPerBitSet));
}
});

mockAppender.assertAllExpectationsMatched();
} finally {
Loggers.removeAppender(cacheLogger, mockAppender);
mockAppender.stop();
}
}

public void testCacheRespectsAccessTimeExpiry() throws Exception {
final Settings settings = Settings.builder()
.put(DocumentSubsetBitsetCache.CACHE_TTL_SETTING.getKey(), "10ms")
Expand Down