From 251f047214b002fc52fa7f1a04776999632d105b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 19 Jul 2019 16:42:38 -0700 Subject: [PATCH 1/5] remove unecessary lock in ForegroundCachePopulator leading to a lot of contention --- .../cache/ForegroundCachePopulator.java | 51 +++++++++---------- 1 file changed, 23 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java index 5ace591c099c..977fc7b71bd3 100644 --- a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java @@ -36,7 +36,6 @@ public class ForegroundCachePopulator implements CachePopulator { private static final Logger log = new Logger(ForegroundCachePopulator.class); - private final Object lock = new Object(); private final ObjectMapper objectMapper; private final CachePopulatorStats cachePopulatorStats; private final long maxEntrySize; @@ -76,20 +75,18 @@ public Sequence wrap( sequence, input -> { if (!tooBig.get()) { - synchronized (lock) { - try { - jsonGenerator.writeObject(cacheFn.apply(input)); + try { + jsonGenerator.writeObject(cacheFn.apply(input)); - // Not flushing jsonGenerator before checking this, but should be ok since Jackson buffers are - // typically just a few KB, and we don't want to waste cycles flushing. - if (maxEntrySize > 0 && bytes.size() > maxEntrySize) { - tooBig.set(true); - } - } - catch (IOException e) { - throw new RuntimeException(e); + // Not flushing jsonGenerator before checking this, but should be ok since Jackson buffers are + // typically just a few KB, and we don't want to waste cycles flushing. + if (maxEntrySize > 0 && bytes.size() > maxEntrySize) { + tooBig.set(true); } } + catch (IOException e) { + throw new RuntimeException(e); + } } return input; @@ -100,24 +97,22 @@ public Sequence wrap( @Override public void after(final boolean isDone, final Throwable thrown) throws Exception { - synchronized (lock) { - jsonGenerator.close(); + jsonGenerator.close(); - if (isDone) { - // Check tooBig, then check maxEntrySize one more time, after closing/flushing jsonGenerator. - if (tooBig.get() || (maxEntrySize > 0 && bytes.size() > maxEntrySize)) { - cachePopulatorStats.incrementOversized(); - return; - } + if (isDone) { + // Check tooBig, then check maxEntrySize one more time, after closing/flushing jsonGenerator. + if (tooBig.get() || (maxEntrySize > 0 && bytes.size() > maxEntrySize)) { + cachePopulatorStats.incrementOversized(); + return; + } - try { - cache.put(cacheKey, bytes.toByteArray()); - cachePopulatorStats.incrementOk(); - } - catch (Exception e) { - log.warn(e, "Unable to write to cache"); - cachePopulatorStats.incrementError(); - } + try { + cache.put(cacheKey, bytes.toByteArray()); + cachePopulatorStats.incrementOk(); + } + catch (Exception e) { + log.warn(e, "Unable to write to cache"); + cachePopulatorStats.incrementError(); } } } From 5a9f0e42a2ff947b84c3fb009947d2d791653742 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 21 Jul 2019 01:29:59 -0700 Subject: [PATCH 2/5] mutableboolean, javadocs,document some cache configs that were missing --- docs/content/configuration/index.md | 6 ++++ .../druid/query/QueryRunnerFactory.java | 30 ++++++++++--------- .../cache/BackgroundCachePopulator.java | 7 +++++ .../druid/client/cache/CachePopulator.java | 5 ++++ .../cache/ForegroundCachePopulator.java | 16 ++++++---- 5 files changed, 45 insertions(+), 19 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 28d434614c39..198b598dc04a 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1176,6 +1176,8 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| +|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1319,6 +1321,8 @@ You can optionally only configure caching to be enabled on the Historical by set |`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false| |`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false| |`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| +|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.historical.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1452,6 +1456,8 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| +|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/QueryRunnerFactory.java index fc4e5a69a18f..0832fb13ccae 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/QueryRunnerFactory.java @@ -25,42 +25,44 @@ import java.util.concurrent.ExecutorService; /** - * An interface that defines the nitty gritty implementation detauls of a Query on a Segment + * An interface that defines the nitty gritty implementation details of a Query on a Segment */ @ExtensionPoint public interface QueryRunnerFactory> { /** - * Given a specific segment, this method will create a QueryRunner. + * Given a specific segment, this method will create a {@link QueryRunner}. * - * The QueryRunner, when asked, will generate a Sequence of results based on the given segment. This - * is the meat of the query processing and is where the results are actually generated. Everything else - * is just merging and reduction logic. + * The {@link QueryRunner}, when asked, will generate a {@link org.apache.druid.java.util.common.guava.Sequence} of + * results based on the given segment. This is the meat of the {@link Query} processing and is where the results are + * actually generated. Everything else is just merging and reduction logic. * - * @param segment The segment to process - * @return A QueryRunner that, when asked, will generate a Sequence of results based on the given segment + * @param segment The segment to process + * @return A {@link QueryRunner} that, when asked, will generate a + * {@link org.apache.druid.java.util.common.guava.Sequence} of results based on the given segment */ QueryRunner createRunner(Segment segment); /** * Runners generated with createRunner() and combined into an Iterable in (time,shardId) order are passed - * along to this method with an ExecutorService. The method should then return a QueryRunner that, when - * asked, will use the ExecutorService to run the base QueryRunners in some fashion. + * along to this method with an {@link ExecutorService}. The method should then return a {@link QueryRunner} that, + * when asked, will use the {@link ExecutorService} to run the base QueryRunners in some fashion. * - * The vast majority of the time, this should be implemented with + * The vast majority of the time, this should be implemented with {@link ChainedExecutionQueryRunner}: * * return new ChainedExecutionQueryRunner<>(queryExecutor, toolChest.getOrdering(), queryWatcher, queryRunners); * * Which will allow for parallel execution up to the maximum number of processing threads allowed. * - * @param queryExecutor ExecutorService to be used for parallel processing - * @param queryRunners Individual QueryRunner objects that produce some results - * @return a QueryRunner that, when asked, will use the ExecutorService to run the base QueryRunners + * @param queryExecutor {@link ExecutorService} to be used for parallel processing + * @param queryRunners Individual {@link QueryRunner} objects that produce some results + * @return a {@link QueryRunner} that, when asked, will use the {@link ExecutorService} to run the base + * {@link QueryRunner} collection. */ QueryRunner mergeRunners(ExecutorService queryExecutor, Iterable> queryRunners); /** - * Provides access to the toolchest for this specific query type. + * Provides access to the {@link QueryToolChest} for this specific {@link Query} type. * * @return an instance of the toolchest for this specific query type. */ diff --git a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java index 0e78e994fd04..84570388c7d6 100644 --- a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java @@ -38,6 +38,13 @@ import java.util.concurrent.ExecutorService; import java.util.function.Function; +/** + * {@link CachePopulator} implementation that uses a {@link ExecutorService} thread pool to populate a cache in the + * background. Used if config "druid.*.cache.numBackgroundThreads" is greater than 0. If maximum cache entry size, + * specified by config "druid.*.cache.maxEntrySize", is exceeded, this {@link CachePopulator} implementation will + * be a bit less efficient than {@link ForegroundCachePopulator} which can stop retaining values early, in exchange + * for interacting with the {@link Cache} in a background thread. + */ public class BackgroundCachePopulator implements CachePopulator { private static final Logger log = new Logger(BackgroundCachePopulator.class); diff --git a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java index 0d67cb9a0eca..b83d4e4b2b8e 100644 --- a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java @@ -23,6 +23,11 @@ import java.util.function.Function; +/** + * Abstraction of mechanism for populating a {@link Cache} by wrapping a {@link Sequence} and providing a function to + * extract the values from it. At runtime, the {@link CachePopulator} implementation is used as a singleton and + * injected where needed to share between all cacheables, which requires the {@link Cache} itself to be thread-safe. + */ public interface CachePopulator { Sequence wrap( diff --git a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java index 977fc7b71bd3..9b36bb4ed106 100644 --- a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; +import org.apache.commons.lang.mutable.MutableBoolean; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.SequenceWrapper; import org.apache.druid.java.util.common.guava.Sequences; @@ -29,9 +30,14 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +/** + * {@link CachePopulator} implementation that populates a cache on the same thread that is processing the + * {@link Sequence}. Used if config "druid.*.cache.numBackgroundThreads" is 0 (the default). This {@link CachePopulator} + * should be more efficient than {@link BackgroundCachePopulator} if maximum cache entry size, specified by config + * "druid.*.cache.maxEntrySize", is exceeded. + */ public class ForegroundCachePopulator implements CachePopulator { private static final Logger log = new Logger(ForegroundCachePopulator.class); @@ -60,7 +66,7 @@ public Sequence wrap( ) { final ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - final AtomicBoolean tooBig = new AtomicBoolean(false); + final MutableBoolean tooBig = new MutableBoolean(false); final JsonGenerator jsonGenerator; try { @@ -74,14 +80,14 @@ public Sequence wrap( Sequences.map( sequence, input -> { - if (!tooBig.get()) { + if (!tooBig.isTrue()) { try { jsonGenerator.writeObject(cacheFn.apply(input)); // Not flushing jsonGenerator before checking this, but should be ok since Jackson buffers are // typically just a few KB, and we don't want to waste cycles flushing. if (maxEntrySize > 0 && bytes.size() > maxEntrySize) { - tooBig.set(true); + tooBig.setValue(true); } } catch (IOException e) { @@ -101,7 +107,7 @@ public void after(final boolean isDone, final Throwable thrown) throws Exception if (isDone) { // Check tooBig, then check maxEntrySize one more time, after closing/flushing jsonGenerator. - if (tooBig.get() || (maxEntrySize > 0 && bytes.size() > maxEntrySize)) { + if (tooBig.isTrue() || (maxEntrySize > 0 && bytes.size() > maxEntrySize)) { cachePopulatorStats.incrementOversized(); return; } From eddc981bca3714fcc37f99973d35d7fe5bcab40f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 21 Jul 2019 02:43:14 -0700 Subject: [PATCH 3/5] more doc stuff --- docs/content/configuration/index.md | 6 +++--- .../client/cache/BackgroundCachePopulator.java | 3 ++- .../apache/druid/client/cache/CachePopulator.java | 14 ++++++++++++++ .../druid/client/cache/CachePopulatorStats.java | 5 +++++ 4 files changed, 24 insertions(+), 4 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 198b598dc04a..c9856d9c9d1f 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1176,7 +1176,7 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| -|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1321,7 +1321,7 @@ You can optionally only configure caching to be enabled on the Historical by set |`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false| |`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false| |`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| -|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.historical.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1456,7 +1456,7 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| -|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size|0| +|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java index 84570388c7d6..ab2e34be94b3 100644 --- a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java @@ -43,7 +43,8 @@ * background. Used if config "druid.*.cache.numBackgroundThreads" is greater than 0. If maximum cache entry size, * specified by config "druid.*.cache.maxEntrySize", is exceeded, this {@link CachePopulator} implementation will * be a bit less efficient than {@link ForegroundCachePopulator} which can stop retaining values early, in exchange - * for interacting with the {@link Cache} in a background thread. + * for interacting with the {@link Cache} in a background thread. Note that there is no back-pressure or load shedding, + * so this implementation might open the door for oom scenarios depending on background threadpool utilization. */ public class BackgroundCachePopulator implements CachePopulator { diff --git a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java index b83d4e4b2b8e..dbbade1cf6a0 100644 --- a/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/CachePopulator.java @@ -27,6 +27,20 @@ * Abstraction of mechanism for populating a {@link Cache} by wrapping a {@link Sequence} and providing a function to * extract the values from it. At runtime, the {@link CachePopulator} implementation is used as a singleton and * injected where needed to share between all cacheables, which requires the {@link Cache} itself to be thread-safe. + * + * Consumers of the {@link Sequence} will either be a processing thread (in the case of a historical or task), or + * an http thread in the case of a broker. See: + * + * historicals: {@link org.apache.druid.server.coordination.ServerManager} and + * {@link org.apache.druid.client.CachingQueryRunner} + * + * realtime tasks: {@link org.apache.druid.segment.realtime.appenderator.SinkQuerySegmentWalker} and + * {@link org.apache.druid.client.CachingQueryRunner} + * + * brokers: {@link org.apache.druid.server.ClientQuerySegmentWalker} and + * {@link org.apache.druid.client.CachingClusteredClient} + * + * for additional details */ public interface CachePopulator { diff --git a/server/src/main/java/org/apache/druid/client/cache/CachePopulatorStats.java b/server/src/main/java/org/apache/druid/client/cache/CachePopulatorStats.java index cfe4587fef7d..e3e768b1c42b 100644 --- a/server/src/main/java/org/apache/druid/client/cache/CachePopulatorStats.java +++ b/server/src/main/java/org/apache/druid/client/cache/CachePopulatorStats.java @@ -22,6 +22,11 @@ import java.util.concurrent.atomic.AtomicLong; /** + * Thread safe collector of {@link CachePopulator} statistics, utilized {@link CacheMonitor} to emit cache metrics. + * Like the {@link CachePopulator}, this is used as a singleton. + * + * See {@link org.apache.druid.guice.DruidProcessingModule#getCachePopulator} which supplies either + * {@link ForegroundCachePopulator} or {@link BackgroundCachePopulator}, as configured, for more details. */ public class CachePopulatorStats { From c159d74bc5579401e9eccc40dbfb6294459aa0fd Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Sun, 21 Jul 2019 02:47:28 -0700 Subject: [PATCH 4/5] adjustments --- docs/content/configuration/index.md | 6 +++--- .../apache/druid/client/cache/BackgroundCachePopulator.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index c9856d9c9d1f..19a7d8fd6d2f 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1176,7 +1176,7 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| -|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| +|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1321,7 +1321,7 @@ You can optionally only configure caching to be enabled on the Historical by set |`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false| |`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false| |`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| -|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| +|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.historical.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1456,7 +1456,7 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| -|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no back-pressure or load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| +|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java index ab2e34be94b3..0af5a44fc7e7 100644 --- a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java @@ -43,8 +43,8 @@ * background. Used if config "druid.*.cache.numBackgroundThreads" is greater than 0. If maximum cache entry size, * specified by config "druid.*.cache.maxEntrySize", is exceeded, this {@link CachePopulator} implementation will * be a bit less efficient than {@link ForegroundCachePopulator} which can stop retaining values early, in exchange - * for interacting with the {@link Cache} in a background thread. Note that there is no back-pressure or load shedding, - * so this implementation might open the door for oom scenarios depending on background threadpool utilization. + * for interacting with the {@link Cache} in a background thread. Note that there is no load shedding, so this + * implementation might open the door for oom scenarios depending on background threadpool utilization. */ public class BackgroundCachePopulator implements CachePopulator { From 93fa2d7b1800cbd6666f0d73656c7e437c64749a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 23 Jul 2019 02:24:38 -0700 Subject: [PATCH 5/5] remove background documentation --- docs/content/configuration/index.md | 3 --- .../apache/druid/client/cache/BackgroundCachePopulator.java | 6 +----- .../apache/druid/client/cache/ForegroundCachePopulator.java | 4 +--- 3 files changed, 2 insertions(+), 11 deletions(-) diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 19a7d8fd6d2f..073b512157fd 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -1176,7 +1176,6 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| -|`druid.realtime.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1321,7 +1320,6 @@ You can optionally only configure caching to be enabled on the Historical by set |`druid.historical.cache.useCache`|true, false|Enable the cache on the Historical.|false| |`druid.historical.cache.populateCache`|true, false|Populate the cache on the Historical.|false| |`druid.historical.cache.unCacheable`|All druid query types|All query types to not cache.|["groupBy", "select"]| -|`druid.historical.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.historical.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1456,7 +1454,6 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`["groupBy", "select"]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| -|`druid.broker.cache.numBackgroundThreads`|If greater than 0, cache will be populated in the background thread pool of the configured size. By default cache is populated in the foreground, which can more efficiently handle reaching `maxEntrySize` than when done in the background. Note that there is no load shedding for background cache population, so it can also lead to out of memory scenarios depending on background threadpool utilization.|0| |`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java index 0af5a44fc7e7..3b7f30dac676 100644 --- a/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/BackgroundCachePopulator.java @@ -40,11 +40,7 @@ /** * {@link CachePopulator} implementation that uses a {@link ExecutorService} thread pool to populate a cache in the - * background. Used if config "druid.*.cache.numBackgroundThreads" is greater than 0. If maximum cache entry size, - * specified by config "druid.*.cache.maxEntrySize", is exceeded, this {@link CachePopulator} implementation will - * be a bit less efficient than {@link ForegroundCachePopulator} which can stop retaining values early, in exchange - * for interacting with the {@link Cache} in a background thread. Note that there is no load shedding, so this - * implementation might open the door for oom scenarios depending on background threadpool utilization. + * background. Used if config "druid.*.cache.numBackgroundThreads" is greater than 0. */ public class BackgroundCachePopulator implements CachePopulator { diff --git a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java index 9b36bb4ed106..0bdeb3a49125 100644 --- a/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java +++ b/server/src/main/java/org/apache/druid/client/cache/ForegroundCachePopulator.java @@ -34,9 +34,7 @@ /** * {@link CachePopulator} implementation that populates a cache on the same thread that is processing the - * {@link Sequence}. Used if config "druid.*.cache.numBackgroundThreads" is 0 (the default). This {@link CachePopulator} - * should be more efficient than {@link BackgroundCachePopulator} if maximum cache entry size, specified by config - * "druid.*.cache.maxEntrySize", is exceeded. + * {@link Sequence}. Used if config "druid.*.cache.numBackgroundThreads" is 0 (the default). */ public class ForegroundCachePopulator implements CachePopulator {