Skip to content

Commit c2b5e5e

Browse files
Stuart McLeanb-slim
authored andcommitted
Default caffeine cache size (apache#5738)
* add default caffeine cache size based on runtime Xmx or max 1GB * update docs for caffeine cache * fix formatting * test caffeine size should never be less than 0 * set caffeine max default size to 1G not 1M * fix caffeine cache tests
1 parent 13c616b commit c2b5e5e

File tree

3 files changed

+15
-13
lines changed

3 files changed

+15
-13
lines changed

docs/content/configuration/caching.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -49,7 +49,7 @@ Below are the configuration options known to this module:
4949
|`runtime.properties`|Description|Default|
5050
|--------------------|-----------|-------|
5151
|`druid.cache.type`| Set this to `caffeine` or leave out parameter|`caffeine`|
52-
|`druid.cache.sizeInBytes`|The maximum size of the cache in bytes on heap.|None (unlimited)|
52+
|`druid.cache.sizeInBytes`|The maximum size of the cache in bytes on heap.|min(1GB, Runtime.maxMemory / 10)|
5353
|`druid.cache.expireAfter`|The time (in ms) after an access for which a cache entry may be expired|None (no time limit)|
5454
|`druid.cache.cacheExecutorFactory`|The executor factory to use for Caffeine maintenance. One of `COMMON_FJP`, `SINGLE_THREAD`, or `SAME_THREAD`|ForkJoinPool common pool (`COMMON_FJP`)|
5555
|`druid.cache.evictOnClose`|If a close of a namespace (ex: removing a segment from a node) should cause an eager eviction of associated cache values|`false`|

server/src/main/java/io/druid/client/cache/CaffeineCache.java

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ public class CaffeineCache implements io.druid.client.cache.Cache
4343
{
4444
private static final Logger log = new Logger(CaffeineCache.class);
4545
private static final int FIXED_COST = 8; // Minimum cost in "weight" per entry;
46+
private static final int MAX_DEFAULT_BYTES = 1024 * 1024 * 1024;
4647
private static final LZ4Factory LZ4_FACTORY = LZ4Factory.fastestInstance();
4748
private static final LZ4FastDecompressor LZ4_DECOMPRESSOR = LZ4_FACTORY.fastDecompressor();
4849
private static final LZ4Compressor LZ4_COMPRESSOR = LZ4_FACTORY.fastCompressor();
@@ -51,7 +52,6 @@ public class CaffeineCache implements io.druid.client.cache.Cache
5152
private final AtomicReference<CacheStats> priorStats = new AtomicReference<>(CacheStats.empty());
5253
private final CaffeineCacheConfig config;
5354

54-
5555
public static CaffeineCache create(final CaffeineCacheConfig config)
5656
{
5757
return create(config, config.createExecutor());
@@ -66,14 +66,16 @@ public static CaffeineCache create(final CaffeineCacheConfig config, final Execu
6666
.expireAfterAccess(config.getExpireAfter(), TimeUnit.MILLISECONDS);
6767
}
6868
if (config.getSizeInBytes() >= 0) {
69-
builder
70-
.maximumWeight(config.getSizeInBytes())
71-
.weigher((NamedKey key, byte[] value) -> value.length
72-
+ key.key.length
73-
+ key.namespace.length() * Character.BYTES
74-
+ FIXED_COST);
69+
builder.maximumWeight(config.getSizeInBytes());
70+
} else {
71+
builder.maximumWeight(Math.min(MAX_DEFAULT_BYTES, Runtime.getRuntime().maxMemory() / 10));
7572
}
76-
builder.executor(executor);
73+
builder
74+
.weigher((NamedKey key, byte[] value) -> value.length
75+
+ key.key.length
76+
+ key.namespace.length() * Character.BYTES
77+
+ FIXED_COST)
78+
.executor(executor);
7779
return new CaffeineCache(builder.build(), config);
7880
}
7981

server/src/test/java/io/druid/client/cache/CaffeineCacheTest.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -360,19 +360,19 @@ public long getSizeInBytes()
360360

361361
CacheStats stats = cache.getStats();
362362
Assert.assertEquals(0L, stats.getNumEntries());
363-
Assert.assertEquals(-1L, stats.getSizeInBytes());
363+
Assert.assertEquals(0L, stats.getSizeInBytes());
364364

365365
cache.put(key1, val1);
366366

367367
stats = cache.getStats();
368368
Assert.assertEquals(1L, stats.getNumEntries());
369-
Assert.assertEquals(-1L, stats.getSizeInBytes());
369+
Assert.assertEquals(34L, stats.getSizeInBytes());
370370

371371
cache.put(key2, val2);
372372

373373
stats = cache.getStats();
374374
Assert.assertEquals(2L, stats.getNumEntries());
375-
Assert.assertEquals(-1L, stats.getSizeInBytes());
375+
Assert.assertEquals(68L, stats.getSizeInBytes());
376376
}
377377

378378
@Test
@@ -461,7 +461,7 @@ public void testDefaultFromProperties()
461461
caffeineCacheConfigJsonConfigProvider.inject(properties, configurator);
462462
final CaffeineCacheConfig config = caffeineCacheConfigJsonConfigProvider.get().get();
463463
Assert.assertEquals(-1, config.getExpireAfter());
464-
Assert.assertEquals(-1, config.getSizeInBytes());
464+
Assert.assertEquals(-1L, config.getSizeInBytes());
465465
Assert.assertEquals(ForkJoinPool.commonPool(), config.createExecutor());
466466
}
467467

0 commit comments

Comments
 (0)