From ba78816e73e0645ddd537b52a1a77e25d3f5855a Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 31 Oct 2017 15:28:38 -0500 Subject: [PATCH 01/25] Add result level caching to Brokers --- docs/content/configuration/broker.md | 2 + docs/content/querying/caching.md | 8 +- docs/content/querying/query-context.md | 2 + .../java/io/druid/query/QueryContexts.java | 22 +++ .../main/java/io/druid/client/CacheUtil.java | 56 +++++++- .../druid/client/CachingClusteredClient.java | 130 +++++++++++++++++- .../io/druid/client/CachingQueryRunner.java | 2 +- .../io/druid/client/cache/CacheConfig.java | 24 ++++ .../druid/client/CachingQueryRunnerTest.java | 3 +- 9 files changed, 235 insertions(+), 14 deletions(-) diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index af97d2dffa50..9bcc45895446 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -104,6 +104,8 @@ You can optionally only configure caching to be enabled on the broker by setting |--------|---------------|-----------|-------| |`druid.broker.cache.useCache`|true, false|Enable the cache on the broker.|false| |`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false| +|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| +|`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false| |`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`| diff --git a/docs/content/querying/caching.md b/docs/content/querying/caching.md index 1d6f60af441e..2dd2c88d6b62 100644 --- a/docs/content/querying/caching.md +++ b/docs/content/querying/caching.md @@ -3,9 +3,10 @@ layout: doc_page --- # Query Caching -Druid supports query result caching through an LRU cache. Results are stored on a per segment basis, along with the -parameters of a given query. This allows Druid to return final results based partially on segment results in the cache and partially -on segment results from scanning historical/real-time segments. +Druid supports query result caching through an LRU cache. Results are stored as a whole or either on a per segment basis along with the +parameters of a given query. Segment level caching allows Druid to return final results based partially on segment results in the cache +and partially on segment results from scanning historical/real-time segments. Result level caching enables Druid to cache the entire +result set, so that query results can be completely retrieved from the cache for identical queries. Segment results can be stored in a local heap cache or in an external distributed key/value store. Segment query caches can be enabled at either the Historical and Broker level (it is not recommended to enable caching on both). @@ -15,6 +16,7 @@ can be enabled at either the Historical and Broker level (it is not recommended Enabling caching on the broker can yield faster results than if query caches were enabled on Historicals for small clusters. This is the recommended setup for smaller production clusters (< 20 servers). Take note that when caching is enabled on the Broker, results from Historicals are returned on a per segment basis, and Historicals will not be able to do any local result merging. +Result level caching is enabled only on the Broker side. ## Query caching on Historicals diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index 70d90e6f23c0..e69cb456c547 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -15,6 +15,8 @@ The query context is used for various query configuration parameters. The follow |queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | |useCache | `true` | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useCache or druid.historical.cache.useCache to determine whether or not to read from the query cache | |populateCache | `true` | Flag indicating whether to save the results of the query to the query cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache or druid.historical.cache.populateCache to determine whether or not to save the results of this query to the query cache | +|useResultLevelCache | `true` | Flag indicating whether to leverage the result level cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useResultLevelCache to determine whether or not to read from the query cache | +|populateResultLevelCache | `true` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache | |bySegment | `false` | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from | |finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` | |chunkPeriod | `P0D` (off) | At the broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure "druid.processing.numThreads" is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. | diff --git a/processing/src/main/java/io/druid/query/QueryContexts.java b/processing/src/main/java/io/druid/query/QueryContexts.java index b56812d8b0e5..67b8526287a6 100644 --- a/processing/src/main/java/io/druid/query/QueryContexts.java +++ b/processing/src/main/java/io/druid/query/QueryContexts.java @@ -37,6 +37,8 @@ public class QueryContexts public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; public static final boolean DEFAULT_USE_CACHE = true; + public static final boolean DEFAULT_POPULATE_RESULTLEVEL_CACHE = true; + public static final boolean DEFAULT_USE_RESULTLEVEL_CACHE = true; public static final int DEFAULT_PRIORITY = 0; public static final int DEFAULT_UNCOVERED_INTERVALS_LIMIT = 0; public static final long DEFAULT_TIMEOUT_MILLIS = 300_000; // 5 minutes @@ -72,6 +74,26 @@ public static boolean isUseCache(Query query, boolean defaultValue) return parseBoolean(query, "useCache", defaultValue); } + public static boolean isPopulateResultLevelCache(Query query) + { + return isPopulateResultLevelCache(query, DEFAULT_POPULATE_RESULTLEVEL_CACHE); + } + + public static boolean isPopulateResultLevelCache(Query query, boolean defaultValue) + { + return parseBoolean(query, "populateResultLevelCache", defaultValue); + } + + public static boolean isUseResultLevelCache(Query query) + { + return isUseResultLevelCache(query, DEFAULT_USE_RESULTLEVEL_CACHE); + } + + public static boolean isUseResultLevelCache(Query query, boolean defaultValue) + { + return parseBoolean(query, "useResultLevelCache", defaultValue); + } + public static boolean isFinalize(Query query, boolean defaultValue) { return parseBoolean(query, "finalize", defaultValue); diff --git a/server/src/main/java/io/druid/client/CacheUtil.java b/server/src/main/java/io/druid/client/CacheUtil.java index af59bee8bcbb..86111bc134e7 100644 --- a/server/src/main/java/io/druid/client/CacheUtil.java +++ b/server/src/main/java/io/druid/client/CacheUtil.java @@ -57,7 +57,16 @@ public static Cache.NamedKey computeSegmentCacheKey( ); } - public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key, Iterable results) + public static Cache.NamedKey computeResultLevelCacheKey( + String resultLevelCacheIdentifier + ) + { + return new Cache.NamedKey( + resultLevelCacheIdentifier, resultLevelCacheIdentifier.getBytes() + ); + } + + public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key, Iterable results, int cacheLimit) { try { ByteArrayOutputStream bytes = new ByteArrayOutputStream(); @@ -66,7 +75,9 @@ public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key gen.writeObject(result); } } - + if (cacheLimit != 0 && bytes.size() > cacheLimit) { + return; + } cache.put(key, bytes.toByteArray()); } catch (IOException e) { @@ -92,6 +103,24 @@ public static boolean populateCacheOnBrokers( return populateCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); } + public static boolean useResultLevelCacheOnBrokers( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return useResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); + } + + public static boolean populateResultLevelCacheOnBrokers( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return populateResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); + } + public static boolean useCacheOnDataNodes( Query query, CacheStrategy> strategy, @@ -134,4 +163,27 @@ private static boolean populateCache( && cacheConfig.isQueryCacheable(query); } + private static boolean useResultLevelCache( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return QueryContexts.isUseResultLevelCache(query) + && strategy != null + && cacheConfig.isUseResultLevelCache() + && cacheConfig.isQueryCacheable(query); + } + + private static boolean populateResultLevelCache( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return QueryContexts.isPopulateResultLevelCache(query) + && strategy != null + && cacheConfig.isPopulateResultLevelCache() + && cacheConfig.isQueryCacheable(query); + } } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index ab7cae42dd02..f891f368378f 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -55,6 +55,7 @@ import io.druid.java.util.common.guava.LazySequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; import io.druid.query.Query; @@ -216,6 +217,8 @@ private class SpecificQueryRunnable private final CacheStrategy> strategy; private final boolean useCache; private final boolean populateCache; + private final boolean useResultCache; + private final boolean populateResultCache; private final boolean isBySegment; private final int uncoveredIntervalsLimit; private final Query downstreamQuery; @@ -231,6 +234,8 @@ private class SpecificQueryRunnable this.useCache = CacheUtil.useCacheOnBrokers(query, strategy, cacheConfig); this.populateCache = CacheUtil.populateCacheOnBrokers(query, strategy, cacheConfig); + this.useResultCache = CacheUtil.useResultLevelCacheOnBrokers(query, strategy, cacheConfig); + this.populateResultCache = CacheUtil.populateResultLevelCacheOnBrokers(query, strategy, cacheConfig); this.isBySegment = QueryContexts.isBySegment(query); // Note that enabling this leads to putting uncovered intervals information in the response headers // and might blow up in some cases https://github.com/druid-io/druid/issues/2108 @@ -255,7 +260,8 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + @Nullable + TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } @@ -265,24 +271,54 @@ Sequence run(final UnaryOperator> time } final Set segments = computeSegmentsToQuery(timeline); - @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); + @Nullable + final byte[] queryCacheKey = computeQueryCacheKey(); + @Nullable + final String queryResultKey = computeCurrentEtag(segments, queryCacheKey); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); - @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + @Nullable + final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + @Nullable + final String currentEtag = queryResultKey; if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } } + @Nullable + final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); + if (cachedResultSet != null) { + log.info("Fetching entire result set from cache"); + return fetchSequenceFromResultLevelCache(cachedResultSet); + } final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - return new LazySequence<>(() -> { + return Sequences.wrap(new LazySequence<>(() -> { List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); return Sequences .simple(sequencesByInterval) .flatMerge(seq -> seq, query.getResultOrdering()); + }).map(r -> { + final Function cacheFn = strategy.prepareForCache(); + final CachePopulator resultCachePopulator = + getResultCachePopulator(queryResultKey); + if (resultCachePopulator != null) { + resultCachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cacheFn.apply(r))); + } + return r; + }), new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) throws Exception + { + final CachePopulator cachePopulator = + getResultCachePopulator(queryResultKey); + if (cachePopulator != null) { + cachePopulator.populate(); + } + } }); } @@ -360,7 +396,7 @@ private void computeUncoveredIntervals(TimelineLookup ti @Nullable private byte[] computeQueryCacheKey() { - if ((populateCache || useCache) // implies strategy != null + if ((populateCache || useCache || populateResultCache) // implies strategy != null && !isBySegment) { // explicit bySegment queries are never cached assert strategy != null; return strategy.computeCacheKey(query); @@ -423,6 +459,21 @@ private List> pruneSegmentsWithCachedResults( return alreadyCachedResults; } + private byte[] fetchFromResultLevelCache( + final String queryCacheKey + ) + { + if (queryCacheKey == null) { + return null; + } + Cache.NamedKey queryKey = CacheUtil.computeResultLevelCacheKey(queryCacheKey); + final byte[] cachedValue = computeResultLevelCacheValue(queryKey); + if (cachedValue == null && populateResultCache) { + addResultCachePopulator(queryKey, queryCacheKey); + } + return cachedValue; + } + private Map computePerSegmentCacheKeys( Set segments, byte[] queryCacheKey @@ -450,6 +501,15 @@ private Map computeCachedValues(Map> groupSegmentsByServer(Set segments) { final SortedMap> serverSegments = Maps.newTreeMap(); @@ -559,6 +636,45 @@ private void addSequencesFromServer( }); } + private Sequence fetchSequenceFromResultLevelCache( + final byte[] cachedResult + ) + { + if (strategy == null) { + return null; + } + final Function pullFromCacheFunction = strategy.pullFromCache(); + final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); + Sequence cachedSequence = new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + try { + if (cachedResult.length == 0) { + return Iterators.emptyIterator(); + } + + return objectMapper.readValues( + objectMapper.getFactory().createParser(cachedResult), + cacheObjectClazz + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void cleanup(Iterator iterFromMake) + { + } + } + ); + return Sequences.map(cachedSequence, pullFromCacheFunction); + } + @SuppressWarnings("unchecked") private Sequence getBySegmentServerResults( final QueryRunner serverRunner, @@ -665,7 +781,7 @@ public void populate() @Override public void onSuccess(List cacheData) { - CacheUtil.populate(cache, mapper, key, cacheData); + CacheUtil.populate(cache, mapper, key, cacheData, cacheConfig.getResultLevelCacheLimit()); // Help out GC by making sure all references are gone cacheFutures.clear(); } diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 17df72a59967..376135d20f2d 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -181,7 +181,7 @@ public void run() public void run() { try { - CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get()); + CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get(), 0); } catch (Exception e) { log.error(e, "Error while getting future for cache task"); diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index acfe6a6153df..17829556d9f1 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -37,6 +37,12 @@ public class CacheConfig @JsonProperty private boolean populateCache = false; + @JsonProperty + private boolean useResultLevelCache = false; + + @JsonProperty + private boolean populateResultLevelCache = false; + @JsonProperty @Min(0) private int numBackgroundThreads = 0; @@ -45,6 +51,9 @@ public class CacheConfig @Min(0) private int cacheBulkMergeLimit = Integer.MAX_VALUE; + @JsonProperty + private int resultLevelCacheLimit = 10485760; + @JsonProperty private List unCacheable = Arrays.asList(Query.GROUP_BY, Query.SELECT); @@ -58,6 +67,16 @@ public boolean isUseCache() return useCache; } + public boolean isPopulateResultLevelCache() + { + return populateResultLevelCache; + } + + public boolean isUseResultLevelCache() + { + return useResultLevelCache; + } + public int getNumBackgroundThreads() { return numBackgroundThreads; @@ -68,6 +87,11 @@ public int getCacheBulkMergeLimit() return cacheBulkMergeLimit; } + public int getResultLevelCacheLimit() + { + return resultLevelCacheLimit; + } + public boolean isQueryCacheable(Query query) { return isQueryCacheable(query.getType()); diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 0ba202eb0091..837ef3564fbb 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -351,7 +351,8 @@ private void testUseCache( cache, objectMapper, cacheKey, - Iterables.transform(expectedResults, cacheStrategy.prepareForCache()) + Iterables.transform(expectedResults, cacheStrategy.prepareForCache()), + 0 ); CachingQueryRunner runner = new CachingQueryRunner( From 7cb33cdf3802f9e7220be766c29fa405759c8aa8 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 31 Oct 2017 15:43:03 -0500 Subject: [PATCH 02/25] Minor doc changes --- docs/content/configuration/broker.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index 9bcc45895446..b130faa5e197 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -104,8 +104,9 @@ You can optionally only configure caching to be enabled on the broker by setting |--------|---------------|-----------|-------| |`druid.broker.cache.useCache`|true, false|Enable the cache on the broker.|false| |`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false| -|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| +|`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| |`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false| +|`druid.broker.cache.resultLevelCacheLimit`|positive integer or 0|Maximum size of query response that can be cached.|10485760| |`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`| From 83ee76e34d5449920d70ef80a2f213c340d14002 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 31 Oct 2017 20:41:06 -0500 Subject: [PATCH 03/25] Simplify sequences --- .../druid/client/CachingClusteredClient.java | 60 ++++++++----------- 1 file changed, 26 insertions(+), 34 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index f891f368378f..010a595f5752 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -300,15 +300,7 @@ Sequence run(final UnaryOperator> time return Sequences .simple(sequencesByInterval) .flatMerge(seq -> seq, query.getResultOrdering()); - }).map(r -> { - final Function cacheFn = strategy.prepareForCache(); - final CachePopulator resultCachePopulator = - getResultCachePopulator(queryResultKey); - if (resultCachePopulator != null) { - resultCachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cacheFn.apply(r))); - } - return r; - }), new SequenceWrapper() + }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() { @Override public void after(boolean isDone, Throwable thrown) throws Exception @@ -322,6 +314,17 @@ public void after(boolean isDone, Throwable thrown) throws Exception }); } + private T cacheResultEntry(T result, String queryResultKey) + { + final Function cacheFn = strategy.prepareForCache(); + final CachePopulator resultCachePopulator = + getResultCachePopulator(queryResultKey); + if (resultCachePopulator != null) { + resultCachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cacheFn.apply(result))); + } + return result; + } + private Set computeSegmentsToQuery(TimelineLookup timeline) { final List> serversLookup = toolChest.filterSegments( @@ -645,32 +648,21 @@ private Sequence fetchSequenceFromResultLevelCache( } final Function pullFromCacheFunction = strategy.pullFromCache(); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); - Sequence cachedSequence = new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - try { - if (cachedResult.length == 0) { - return Iterators.emptyIterator(); - } - - return objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public void cleanup(Iterator iterFromMake) - { - } + Sequence cachedSequence = Sequences.simple(() -> { + try { + if (cachedResult.length == 0) { + return Iterators.emptyIterator(); } + + return objectMapper.readValues( + objectMapper.getFactory().createParser(cachedResult), + cacheObjectClazz + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } ); return Sequences.map(cachedSequence, pullFromCacheFunction); } From 24a7595499b3446f14a1f8b05da556f358a48cee Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 10 Nov 2017 10:25:47 -0600 Subject: [PATCH 04/25] Move etag execution --- .../druid/client/CachingClusteredClient.java | 65 ++++++++++++------- 1 file changed, 42 insertions(+), 23 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 010a595f5752..9381a129edb0 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -284,34 +284,49 @@ Sequence run(final UnaryOperator> time return Sequences.empty(); } } + if(false) { + @Nullable + final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); + if (cachedResultSet != null) { + log.info("Fetching entire result set from cache"); + return fetchSequenceFromResultLevelCache(cachedResultSet); + } + } - @Nullable - final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); - if (cachedResultSet != null) { - log.info("Fetching entire result set from cache"); - return fetchSequenceFromResultLevelCache(cachedResultSet); - } final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - return Sequences.wrap(new LazySequence<>(() -> { - List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); - addSequencesFromCache(sequencesByInterval, alreadyCachedResults); - addSequencesFromServer(sequencesByInterval, segmentsByServer); - return Sequences - .simple(sequencesByInterval) - .flatMerge(seq -> seq, query.getResultOrdering()); - }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() - { - @Override - public void after(boolean isDone, Throwable thrown) throws Exception + if (true) { + return new LazySequence<>(() -> { + List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + + segmentsByServer.size()); + addSequencesFromCache(sequencesByInterval, alreadyCachedResults); + addSequencesFromServer(sequencesByInterval, segmentsByServer); + return Sequences + .simple(sequencesByInterval) + .flatMerge(seq -> seq, query.getResultOrdering()); + }); + } else { + return Sequences.wrap(new LazySequence<>(() -> { + List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + + segmentsByServer.size()); + addSequencesFromCache(sequencesByInterval, alreadyCachedResults); + addSequencesFromServer(sequencesByInterval, segmentsByServer); + return Sequences + .simple(sequencesByInterval) + .flatMerge(seq -> seq, query.getResultOrdering()); + }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() { - final CachePopulator cachePopulator = - getResultCachePopulator(queryResultKey); - if (cachePopulator != null) { - cachePopulator.populate(); + @Override + public void after(boolean isDone, Throwable thrown) throws Exception + { + final CachePopulator cachePopulator = + getResultCachePopulator(queryResultKey); + if (cachePopulator != null) { + cachePopulator.populate(); + } } - } - }); + }); + } } private T cacheResultEntry(T result, String queryResultKey) @@ -414,6 +429,7 @@ private String computeCurrentEtag(final Set segments, @Nullable Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; for (ServerToSegment p : segments) { + log.info(p.getServer().pick().getServer().getType().toString()); if (!p.getServer().pick().getServer().segmentReplicatable()) { hasOnlyHistoricalSegments = false; break; @@ -628,6 +644,7 @@ private void addSequencesFromServer( final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); Sequence serverResults; + log.info("[A2L] Sending request to historical "); if (isBySegment) { serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); } else if (!server.segmentReplicatable() || !populateCache) { @@ -635,6 +652,8 @@ private void addSequencesFromServer( } else { serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); } + log.info("[A2L] Got query results "); + listOfSequences.add(serverResults); }); } From 130ee631c8f7047bc973c030241680a61c0ae320 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 10 Nov 2017 13:10:26 -0600 Subject: [PATCH 05/25] Modify cacheLimit criteria --- .../main/java/io/druid/client/CacheUtil.java | 4 +- .../druid/client/CachingClusteredClient.java | 80 +++++++++---------- .../io/druid/client/CachingQueryRunner.java | 2 +- .../io/druid/client/cache/CacheConfig.java | 4 +- .../druid/client/CachingQueryRunnerTest.java | 2 +- 5 files changed, 43 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/io/druid/client/CacheUtil.java b/server/src/main/java/io/druid/client/CacheUtil.java index 86111bc134e7..bfc359a542e2 100644 --- a/server/src/main/java/io/druid/client/CacheUtil.java +++ b/server/src/main/java/io/druid/client/CacheUtil.java @@ -62,7 +62,7 @@ public static Cache.NamedKey computeResultLevelCacheKey( ) { return new Cache.NamedKey( - resultLevelCacheIdentifier, resultLevelCacheIdentifier.getBytes() + resultLevelCacheIdentifier, StringUtils.toUtf8(resultLevelCacheIdentifier) ); } @@ -75,7 +75,7 @@ public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key gen.writeObject(result); } } - if (cacheLimit != 0 && bytes.size() > cacheLimit) { + if (cacheLimit > 0 && bytes.size() > cacheLimit) { return; } cache.put(key, bytes.toByteArray()); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 9381a129edb0..e316e92a1131 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -273,8 +273,15 @@ Sequence run(final UnaryOperator> time final Set segments = computeSegmentsToQuery(timeline); @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); - @Nullable - final String queryResultKey = computeCurrentEtag(segments, queryCacheKey); + String queryKeyFromEtag; + if (useResultCache) { + log.debug("Result level caching has been enabled."); + queryKeyFromEtag = computeCurrentEtag(segments, queryCacheKey); + } else { + queryKeyFromEtag = null; + } + + final String queryResultKey = queryKeyFromEtag; if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); @@ -284,49 +291,36 @@ Sequence run(final UnaryOperator> time return Sequences.empty(); } } - if(false) { - @Nullable - final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); - if (cachedResultSet != null) { - log.info("Fetching entire result set from cache"); - return fetchSequenceFromResultLevelCache(cachedResultSet); - } - } + @Nullable + final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); + if (cachedResultSet != null) { + log.debug("Fetching entire result set from cache"); + return fetchSequenceFromResultLevelCache(cachedResultSet); + } final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - if (true) { - return new LazySequence<>(() -> { - List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() - + segmentsByServer.size()); - addSequencesFromCache(sequencesByInterval, alreadyCachedResults); - addSequencesFromServer(sequencesByInterval, segmentsByServer); - return Sequences - .simple(sequencesByInterval) - .flatMerge(seq -> seq, query.getResultOrdering()); - }); - } else { - return Sequences.wrap(new LazySequence<>(() -> { - List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() - + segmentsByServer.size()); - addSequencesFromCache(sequencesByInterval, alreadyCachedResults); - addSequencesFromServer(sequencesByInterval, segmentsByServer); - return Sequences - .simple(sequencesByInterval) - .flatMerge(seq -> seq, query.getResultOrdering()); - }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() + + return Sequences.wrap(new LazySequence<>(() -> { + List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + + segmentsByServer.size()); + addSequencesFromCache(sequencesByInterval, alreadyCachedResults); + addSequencesFromServer(sequencesByInterval, segmentsByServer); + return Sequences + .simple(sequencesByInterval) + .flatMerge(seq -> seq, query.getResultOrdering()); + }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) throws Exception { - @Override - public void after(boolean isDone, Throwable thrown) throws Exception - { - final CachePopulator cachePopulator = - getResultCachePopulator(queryResultKey); - if (cachePopulator != null) { - cachePopulator.populate(); - } + final CachePopulator cachePopulator = + getResultCachePopulator(queryResultKey); + if (cachePopulator != null) { + cachePopulator.populate(); } - }); - } + } + }); } private T cacheResultEntry(T result, String queryResultKey) @@ -644,7 +638,6 @@ private void addSequencesFromServer( final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); Sequence serverResults; - log.info("[A2L] Sending request to historical "); if (isBySegment) { serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); } else if (!server.segmentReplicatable() || !populateCache) { @@ -652,8 +645,6 @@ private void addSequencesFromServer( } else { serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); } - log.info("[A2L] Got query results "); - listOfSequences.add(serverResults); }); } @@ -679,7 +670,8 @@ private Sequence fetchSequenceFromResultLevelCache( ); } catch (IOException e) { - throw new RuntimeException(e); + log.error("Exception while parsing cached results"); + return null; } } ); diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 376135d20f2d..d8dd261e709b 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -181,7 +181,7 @@ public void run() public void run() { try { - CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get(), 0); + CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get(), -1); } catch (Exception e) { log.error(e, "Error while getting future for cache task"); diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index 17829556d9f1..d2d8baf15d2f 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -31,6 +31,8 @@ public class CacheConfig public static final String USE_CACHE = "useCache"; public static final String POPULATE_CACHE = "populateCache"; + // The defaults defined here for cache related parameters are different from the QueryContext defaults due to legacy reasons. + // They should be made the same at some point in the future. @JsonProperty private boolean useCache = false; @@ -52,7 +54,7 @@ public class CacheConfig private int cacheBulkMergeLimit = Integer.MAX_VALUE; @JsonProperty - private int resultLevelCacheLimit = 10485760; + private int resultLevelCacheLimit = Integer.MAX_VALUE; @JsonProperty private List unCacheable = Arrays.asList(Query.GROUP_BY, Query.SELECT); diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 837ef3564fbb..a338c6122d26 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -352,7 +352,7 @@ private void testUseCache( objectMapper, cacheKey, Iterables.transform(expectedResults, cacheStrategy.prepareForCache()), - 0 + -1 ); CachingQueryRunner runner = new CachingQueryRunner( From efeb2b2bdfe97732e65df546c4d09629ced59dbd Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 10 Nov 2017 13:47:42 -0600 Subject: [PATCH 06/25] Fix incorrect etag computation --- .../src/main/java/io/druid/client/CachingClusteredClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index e316e92a1131..4cf087b8ea11 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -286,7 +286,7 @@ Sequence run(final UnaryOperator> time @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); @Nullable - final String currentEtag = queryResultKey; + final String currentEtag = computeCurrentEtag(segments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } From 08b4feb5d85fb326ba6f9f239061cc16f4b05f87 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 10 Nov 2017 14:43:21 -0600 Subject: [PATCH 07/25] Fix docs --- docs/content/configuration/broker.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index 55bd132dc83a..2847a6b9b994 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -106,7 +106,7 @@ You can optionally only configure caching to be enabled on the broker by setting |`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false| |`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| |`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false| -|`druid.broker.cache.resultLevelCacheLimit`|positive integer or 0|Maximum size of query response that can be cached.|10485760| +|`druid.broker.cache.resultLevelCacheLimit`|positive integer or 0|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`| From c805b923862dbf71e259aeb2d6c9dbaced1469cc Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Wed, 22 Nov 2017 15:20:11 -0600 Subject: [PATCH 08/25] Add separate query runner for result level caching --- .../main/java/io/druid/client/CacheUtil.java | 56 +--- .../druid/client/CachingClusteredClient.java | 142 ++------- .../io/druid/client/CachingQueryRunner.java | 2 +- .../io/druid/client/ResultLevelCacheUtil.java | 111 +++++++ .../query/ResultLevelCachingQueryRunner.java | 284 ++++++++++++++++++ .../server/ClientQuerySegmentWalker.java | 54 +++- .../druid/client/CachingQueryRunnerTest.java | 3 +- 7 files changed, 455 insertions(+), 197 deletions(-) create mode 100644 server/src/main/java/io/druid/client/ResultLevelCacheUtil.java create mode 100644 server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java diff --git a/server/src/main/java/io/druid/client/CacheUtil.java b/server/src/main/java/io/druid/client/CacheUtil.java index bfc359a542e2..af59bee8bcbb 100644 --- a/server/src/main/java/io/druid/client/CacheUtil.java +++ b/server/src/main/java/io/druid/client/CacheUtil.java @@ -57,16 +57,7 @@ public static Cache.NamedKey computeSegmentCacheKey( ); } - public static Cache.NamedKey computeResultLevelCacheKey( - String resultLevelCacheIdentifier - ) - { - return new Cache.NamedKey( - resultLevelCacheIdentifier, StringUtils.toUtf8(resultLevelCacheIdentifier) - ); - } - - public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key, Iterable results, int cacheLimit) + public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key, Iterable results) { try { ByteArrayOutputStream bytes = new ByteArrayOutputStream(); @@ -75,9 +66,7 @@ public static void populate(Cache cache, ObjectMapper mapper, Cache.NamedKey key gen.writeObject(result); } } - if (cacheLimit > 0 && bytes.size() > cacheLimit) { - return; - } + cache.put(key, bytes.toByteArray()); } catch (IOException e) { @@ -103,24 +92,6 @@ public static boolean populateCacheOnBrokers( return populateCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); } - public static boolean useResultLevelCacheOnBrokers( - Query query, - CacheStrategy> strategy, - CacheConfig cacheConfig - ) - { - return useResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); - } - - public static boolean populateResultLevelCacheOnBrokers( - Query query, - CacheStrategy> strategy, - CacheConfig cacheConfig - ) - { - return populateResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); - } - public static boolean useCacheOnDataNodes( Query query, CacheStrategy> strategy, @@ -163,27 +134,4 @@ private static boolean populateCache( && cacheConfig.isQueryCacheable(query); } - private static boolean useResultLevelCache( - Query query, - CacheStrategy> strategy, - CacheConfig cacheConfig - ) - { - return QueryContexts.isUseResultLevelCache(query) - && strategy != null - && cacheConfig.isUseResultLevelCache() - && cacheConfig.isQueryCacheable(query); - } - - private static boolean populateResultLevelCache( - Query query, - CacheStrategy> strategy, - CacheConfig cacheConfig - ) - { - return QueryContexts.isPopulateResultLevelCache(query) - && strategy != null - && cacheConfig.isPopulateResultLevelCache() - && cacheConfig.isQueryCacheable(query); - } } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 4cf087b8ea11..87a64fb3ac8c 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -55,7 +55,6 @@ import io.druid.java.util.common.guava.LazySequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; -import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; import io.druid.query.Query; @@ -217,8 +216,6 @@ private class SpecificQueryRunnable private final CacheStrategy> strategy; private final boolean useCache; private final boolean populateCache; - private final boolean useResultCache; - private final boolean populateResultCache; private final boolean isBySegment; private final int uncoveredIntervalsLimit; private final Query downstreamQuery; @@ -234,8 +231,6 @@ private class SpecificQueryRunnable this.useCache = CacheUtil.useCacheOnBrokers(query, strategy, cacheConfig); this.populateCache = CacheUtil.populateCacheOnBrokers(query, strategy, cacheConfig); - this.useResultCache = CacheUtil.useResultLevelCacheOnBrokers(query, strategy, cacheConfig); - this.populateResultCache = CacheUtil.populateResultLevelCacheOnBrokers(query, strategy, cacheConfig); this.isBySegment = QueryContexts.isBySegment(query); // Note that enabling this leads to putting uncovered intervals information in the response headers // and might blow up in some cases https://github.com/druid-io/druid/issues/2108 @@ -260,8 +255,7 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable - TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } @@ -271,69 +265,38 @@ Sequence run(final UnaryOperator> time } final Set segments = computeSegmentsToQuery(timeline); - @Nullable - final byte[] queryCacheKey = computeQueryCacheKey(); - String queryKeyFromEtag; - if (useResultCache) { - log.debug("Result level caching has been enabled."); - queryKeyFromEtag = computeCurrentEtag(segments, queryCacheKey); - } else { - queryKeyFromEtag = null; + @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); + + if (responseContext.containsKey("resultLevelCacheEnabled")) { + @Nullable final String prevResultCacheKeyFromEtag = (String) responseContext.get("prevResultSetIdentifier"); + @Nullable final String newResultCacheKeyFromEtag = computeCurrentEtag(segments, queryCacheKey); + responseContext.remove("resultLevelCacheEnabled"); + responseContext.put("currentResultSetIdentifier", newResultCacheKeyFromEtag); + if (newResultCacheKeyFromEtag != null && newResultCacheKeyFromEtag.equals(prevResultCacheKeyFromEtag)) { + return Sequences.empty(); + } } - final String queryResultKey = queryKeyFromEtag; if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable - final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); - @Nullable - final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } } - @Nullable - final byte[] cachedResultSet = fetchFromResultLevelCache(queryResultKey); - if (cachedResultSet != null) { - log.debug("Fetching entire result set from cache"); - return fetchSequenceFromResultLevelCache(cachedResultSet); - } final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - - return Sequences.wrap(new LazySequence<>(() -> { - List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() - + segmentsByServer.size()); + return new LazySequence<>(() -> { + List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); return Sequences .simple(sequencesByInterval) .flatMerge(seq -> seq, query.getResultOrdering()); - }).map(r -> cacheResultEntry(r, queryResultKey)), new SequenceWrapper() - { - @Override - public void after(boolean isDone, Throwable thrown) throws Exception - { - final CachePopulator cachePopulator = - getResultCachePopulator(queryResultKey); - if (cachePopulator != null) { - cachePopulator.populate(); - } - } }); } - private T cacheResultEntry(T result, String queryResultKey) - { - final Function cacheFn = strategy.prepareForCache(); - final CachePopulator resultCachePopulator = - getResultCachePopulator(queryResultKey); - if (resultCachePopulator != null) { - resultCachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cacheFn.apply(result))); - } - return result; - } - private Set computeSegmentsToQuery(TimelineLookup timeline) { final List> serversLookup = toolChest.filterSegments( @@ -408,7 +371,7 @@ private void computeUncoveredIntervals(TimelineLookup ti @Nullable private byte[] computeQueryCacheKey() { - if ((populateCache || useCache || populateResultCache) // implies strategy != null + if ((populateCache || useCache) // implies strategy != null && !isBySegment) { // explicit bySegment queries are never cached assert strategy != null; return strategy.computeCacheKey(query); @@ -423,7 +386,6 @@ private String computeCurrentEtag(final Set segments, @Nullable Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; for (ServerToSegment p : segments) { - log.info(p.getServer().pick().getServer().getType().toString()); if (!p.getServer().pick().getServer().segmentReplicatable()) { hasOnlyHistoricalSegments = false; break; @@ -472,21 +434,6 @@ private List> pruneSegmentsWithCachedResults( return alreadyCachedResults; } - private byte[] fetchFromResultLevelCache( - final String queryCacheKey - ) - { - if (queryCacheKey == null) { - return null; - } - Cache.NamedKey queryKey = CacheUtil.computeResultLevelCacheKey(queryCacheKey); - final byte[] cachedValue = computeResultLevelCacheValue(queryKey); - if (cachedValue == null && populateResultCache) { - addResultCachePopulator(queryKey, queryCacheKey); - } - return cachedValue; - } - private Map computePerSegmentCacheKeys( Set segments, byte[] queryCacheKey @@ -514,15 +461,6 @@ private Map computeCachedValues(Map> groupSegmentsByServer(Set segments) { final SortedMap> serverSegments = Maps.newTreeMap(); @@ -649,35 +570,6 @@ private void addSequencesFromServer( }); } - private Sequence fetchSequenceFromResultLevelCache( - final byte[] cachedResult - ) - { - if (strategy == null) { - return null; - } - final Function pullFromCacheFunction = strategy.pullFromCache(); - final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); - Sequence cachedSequence = Sequences.simple(() -> { - try { - if (cachedResult.length == 0) { - return Iterators.emptyIterator(); - } - - return objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - ); - } - catch (IOException e) { - log.error("Exception while parsing cached results"); - return null; - } - } - ); - return Sequences.map(cachedSequence, pullFromCacheFunction); - } - @SuppressWarnings("unchecked") private Sequence getBySegmentServerResults( final QueryRunner serverRunner, @@ -784,7 +676,7 @@ public void populate() @Override public void onSuccess(List cacheData) { - CacheUtil.populate(cache, mapper, key, cacheData, cacheConfig.getResultLevelCacheLimit()); + CacheUtil.populate(cache, mapper, key, cacheData); // Help out GC by making sure all references are gone cacheFutures.clear(); } diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index d8dd261e709b..17df72a59967 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -181,7 +181,7 @@ public void run() public void run() { try { - CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get(), -1); + CacheUtil.populate(cache, mapper, key, Futures.allAsList(cacheFutures).get()); } catch (Exception e) { log.error(e, "Error while getting future for cache task"); diff --git a/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java new file mode 100644 index 000000000000..1b0a1dd92f49 --- /dev/null +++ b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java @@ -0,0 +1,111 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.client; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.java.util.common.StringUtils; +import io.druid.query.CacheStrategy; +import io.druid.query.Query; +import io.druid.query.QueryContexts; + + +import java.io.IOException; +import java.util.Collections; +import java.util.Map; + + +public class ResultLevelCacheUtil +{ + public static Cache.NamedKey computeResultLevelCacheKey( + String resultLevelCacheIdentifier + ) + { + return new Cache.NamedKey( + resultLevelCacheIdentifier, StringUtils.toUtf8(resultLevelCacheIdentifier) + ); + } + + public static void populate( + Cache cache, + ObjectMapper mapper, + Cache.NamedKey key, + Iterable results, + int cacheLimit, + String etag + ) + { + try { + Map> vals = Collections.singletonMap(etag, results); + byte[] bytes = mapper.writeValueAsBytes(vals); + if (cacheLimit > 0 && bytes.length > cacheLimit) { + return; + } + cache.put(key, bytes); + } + catch (IOException e) { + throw Throwables.propagate(e); + } + } + + public static boolean useResultLevelCacheOnBrokers( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return useResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); + } + + public static boolean populateResultLevelCacheOnBrokers( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return populateResultLevelCache(query, strategy, cacheConfig) && strategy.isCacheable(query, false); + } + + private static boolean useResultLevelCache( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return QueryContexts.isUseResultLevelCache(query) + && strategy != null + && cacheConfig.isUseResultLevelCache() + && cacheConfig.isQueryCacheable(query); + } + + private static boolean populateResultLevelCache( + Query query, + CacheStrategy> strategy, + CacheConfig cacheConfig + ) + { + return QueryContexts.isPopulateResultLevelCache(query) + && strategy != null + && cacheConfig.isPopulateResultLevelCache() + && cacheConfig.isQueryCacheable(query); + } +} diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java new file mode 100644 index 000000000000..b60a4a763256 --- /dev/null +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -0,0 +1,284 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package io.druid.query; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; +import io.druid.client.ResultLevelCacheUtil; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.SequenceWrapper; +import io.druid.java.util.common.guava.Sequences; +import io.druid.java.util.common.logger.Logger; + + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; + +public class ResultLevelCachingQueryRunner implements QueryRunner +{ + private static final Logger log = new Logger(ResultLevelCachingQueryRunner.class); + private final QueryRunner baseRunner; + private final ListeningExecutorService backgroundExecutorService; + private QueryToolChest queryToolChest; + private ObjectMapper objectMapper; + private final Cache cache; + private final CacheConfig cacheConfig; + private final boolean useResultCache; + private final boolean populateResultCache; + private final Query query; + private final CacheStrategy> strategy; + + + public ResultLevelCachingQueryRunner( + QueryRunner baseRunner, + QueryToolChest queryToolChest, + Query query, + ObjectMapper objectMapper, + ExecutorService cachingExec, + Cache cache, + CacheConfig cacheConfig + ) + { + this.baseRunner = baseRunner; + this.queryToolChest = queryToolChest; + this.backgroundExecutorService = MoreExecutors.listeningDecorator(cachingExec); + this.objectMapper = objectMapper; + this.cache = cache; + this.cacheConfig = cacheConfig; + this.query = query; + this.strategy = queryToolChest.getCacheStrategy(query); + this.populateResultCache = ResultLevelCacheUtil.populateResultLevelCacheOnBrokers(query, strategy, cacheConfig); + this.useResultCache = ResultLevelCacheUtil.useResultLevelCacheOnBrokers(query, strategy, cacheConfig); + } + + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + if (useResultCache) { + + final String cacheKeyStr = StringUtils.fromUtf8(strategy.computeCacheKey(query)); + + Map> resultMap; + @Nullable + final byte[] cachedResultSet = fetchResultsFromResultLevelCache(cacheKeyStr); + + resultMap = extractMapFromResults(cachedResultSet); + responseContext.put("resultLevelCacheEnabled", true); + + if (!resultMap.isEmpty()) { + String resultSetIdentifier = resultMap.entrySet().iterator().next().getKey(); + responseContext.put("prevResultSetIdentifier", resultSetIdentifier); + } + + @Nullable + ResultLevelCachePopulator resultLevelCachePopulator = createResultLevelCachePopulator( + cachedResultSet, + cacheKeyStr + ); + Sequence resultFromClient = baseRunner.run( + queryPlus, + responseContext + ); + + String currentResultSetIdentifier = (String) responseContext.get("currentResultSetIdentifier"); + String prevResultSetIdentifier = (String) responseContext.get("prevResultSetIdentifier"); + responseContext.remove("currentResultSetIdentifier"); + responseContext.remove("prevResultSetIdentifier"); + if (currentResultSetIdentifier != null && currentResultSetIdentifier.equals(prevResultSetIdentifier)) { + log.info("Return cached result set as there is no change in identifiers for query %s ", query.getId()); + Iterable cachedResults = resultMap.entrySet().iterator().next().getValue(); + return fetchSequenceFromResults(cachedResults, strategy); + } else { + return Sequences.wrap(Sequences.map( + resultFromClient, + new Function() + { + @Override + public T apply(T input) + { + cacheResultEntry(resultLevelCachePopulator, input); + return input; + } + } + ), new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) throws Exception + { + if (resultLevelCachePopulator != null) { + // The resultset identifier is cached along with the resultset + resultLevelCachePopulator.populateResults(currentResultSetIdentifier); + } + } + }); + } + } else { + return baseRunner.run( + queryPlus, + responseContext + ); + } + } + + private T cacheResultEntry( + ResultLevelCachePopulator resultLevelCachePopulator, + T resultEntry + ) + { + final Function cacheFn = strategy.prepareForCache(); + if (resultLevelCachePopulator != null) { + resultLevelCachePopulator.cacheFutures + .add(backgroundExecutorService.submit(() -> cacheFn.apply(resultEntry))); + } + return resultEntry; + } + + private byte[] fetchResultsFromResultLevelCache( + final String queryCacheKey + ) + { + if (useResultCache && queryCacheKey != null) { + log.info("Fetching cached result for query: %s", query.getId()); + return cache.get(ResultLevelCacheUtil.computeResultLevelCacheKey(queryCacheKey)); + } + return null; + } + + private Map> extractMapFromResults( + final byte[] cachedResult + ) + { + if (cachedResult == null) { + return Collections.emptyMap(); + } + final TypeReference>> cacheObjectClazz = new TypeReference>>() + { + }; + try { + Map> res = objectMapper.readValue( + cachedResult, cacheObjectClazz + ); + return res; + } + catch (IOException ioe) { + log.error("Error parsing cached result set."); + return Collections.emptyMap(); + } + } + + private Sequence fetchSequenceFromResults( + Iterable cachedResult, CacheStrategy strategy + ) + { + if (strategy == null) { + return null; + } + final Function pullFromCacheFunction = strategy.pullFromCache(); + Sequence cachedSequence = Sequences.simple(() -> cachedResult.iterator()); + return Sequences.map(cachedSequence, pullFromCacheFunction); + } + + private ResultLevelCachePopulator createResultLevelCachePopulator(byte[] cachedResultSet, String cacheKeyStr) + { + if (cachedResultSet == null && populateResultCache) { + return new ResultLevelCachePopulator( + cache, + objectMapper, + ResultLevelCacheUtil.computeResultLevelCacheKey(cacheKeyStr), + cacheConfig, + backgroundExecutorService + ); + } else { + return null; + } + } + + public class ResultLevelCachePopulator + { + private final Cache cache; + private final ObjectMapper mapper; + private final Cache.NamedKey key; + private final ConcurrentLinkedQueue> cacheFutures = new ConcurrentLinkedQueue<>(); + private final CacheConfig cacheConfig; + private final ListeningExecutorService backgroundExecutorService; + + private ResultLevelCachePopulator( + Cache cache, + ObjectMapper mapper, + Cache.NamedKey key, + CacheConfig cacheConfig, + ListeningExecutorService backgroundExecutorService + ) + { + this.cache = cache; + this.mapper = mapper; + this.key = key; + this.cacheConfig = cacheConfig; + this.backgroundExecutorService = backgroundExecutorService; + } + + public void populateResults( + String resultSetIdentifier + ) + { + Futures.addCallback( + Futures.allAsList(cacheFutures), + new FutureCallback>() + { + @Override + public void onSuccess(List cacheData) + { + ResultLevelCacheUtil.populate( + cache, + mapper, + key, + cacheData, + cacheConfig.getResultLevelCacheLimit(), + resultSetIdentifier + ); + // Help out GC by making sure all references are gone + cacheFutures.clear(); + } + + @Override + public void onFailure(Throwable throwable) + { + log.error(throwable, "Result-Level caching failed"); + } + }, + backgroundExecutorService + ); + } + } +} diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 993747b7a3e2..256c3ab9710a 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -24,6 +24,9 @@ import com.google.inject.Inject; import com.metamx.emitter.service.ServiceEmitter; import io.druid.client.CachingClusteredClient; +import io.druid.client.cache.Cache; +import io.druid.client.cache.CacheConfig; +import io.druid.guice.annotations.BackgroundCaching; import io.druid.query.FluentQueryRunnerBuilder; import io.druid.query.PostProcessingOperator; import io.druid.query.Query; @@ -31,12 +34,15 @@ import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; +import io.druid.query.ResultLevelCachingQueryRunner; import io.druid.query.RetryQueryRunner; import io.druid.query.RetryQueryRunnerConfig; import io.druid.query.SegmentDescriptor; import io.druid.server.initialization.ServerConfig; import org.joda.time.Interval; +import java.util.concurrent.ExecutorService; + /** */ public class ClientQuerySegmentWalker implements QuerySegmentWalker @@ -47,6 +53,10 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final RetryQueryRunnerConfig retryConfig; private final ObjectMapper objectMapper; private final ServerConfig serverConfig; + private final ExecutorService cachingExec; + private final Cache cache; + private final CacheConfig cacheConfig; + @Inject public ClientQuerySegmentWalker( @@ -55,7 +65,10 @@ public ClientQuerySegmentWalker( QueryToolChestWarehouse warehouse, RetryQueryRunnerConfig retryConfig, ObjectMapper objectMapper, - ServerConfig serverConfig + ServerConfig serverConfig, + @BackgroundCaching ExecutorService cachingExec, + Cache cache, + CacheConfig cacheConfig ) { this.emitter = emitter; @@ -64,6 +77,9 @@ public ClientQuerySegmentWalker( this.retryConfig = retryConfig; this.objectMapper = objectMapper; this.serverConfig = serverConfig; + this.cachingExec = cachingExec; + this.cache = cache; + this.cacheConfig = cacheConfig; } @Override @@ -88,22 +104,30 @@ private QueryRunner makeRunner(Query query, QueryRunner baseClientR } ); - return new FluentQueryRunnerBuilder<>(toolChest) - .create( - new SetAndVerifyContextQueryRunner( - serverConfig, - new RetryQueryRunner<>( - baseClientRunner, - retryConfig, - objectMapper + return new ResultLevelCachingQueryRunner<>( + new FluentQueryRunnerBuilder<>(toolChest) + .create( + new SetAndVerifyContextQueryRunner( + serverConfig, + new RetryQueryRunner<>( + baseClientRunner, + retryConfig, + objectMapper + ) ) ) - ) - .applyPreMergeDecoration() - .mergeResults() - .applyPostMergeDecoration() - .emitCPUTimeMetric(emitter) - .postProcess(postProcessing); + .applyPreMergeDecoration() + .mergeResults() + .applyPostMergeDecoration() + .emitCPUTimeMetric(emitter) + .postProcess(postProcessing), + toolChest, + query, + objectMapper, + cachingExec, + cache, + cacheConfig + ); } diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index a338c6122d26..0ba202eb0091 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -351,8 +351,7 @@ private void testUseCache( cache, objectMapper, cacheKey, - Iterables.transform(expectedResults, cacheStrategy.prepareForCache()), - -1 + Iterables.transform(expectedResults, cacheStrategy.prepareForCache()) ); CachingQueryRunner runner = new CachingQueryRunner( From d81d81c704a9e6f22362f64b87f341c916b1f944 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Wed, 22 Nov 2017 15:23:55 -0600 Subject: [PATCH 09/25] Update docs --- docs/content/configuration/broker.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/configuration/broker.md b/docs/content/configuration/broker.md index 4a28a6ed225f..8454698da220 100644 --- a/docs/content/configuration/broker.md +++ b/docs/content/configuration/broker.md @@ -107,7 +107,7 @@ You can optionally only configure caching to be enabled on the broker by setting |`druid.broker.cache.populateCache`|true, false|Populate the cache on the broker.|false| |`druid.broker.cache.useResultLevelCache`|true, false|Enable result level caching on the broker.|false| |`druid.broker.cache.populateResultLevelCache`|true, false|Populate the result level cache on the broker.|false| -|`druid.broker.cache.resultLevelCacheLimit`|positive integer or 0|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| +|`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`| From fc69327e8918139ca45ac68eb8ea341d7d097681 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Sun, 3 Dec 2017 19:38:38 -0600 Subject: [PATCH 10/25] Add post aggregated results to result level cache --- .../java/io/druid/query/CacheStrategy.java | 17 +++ .../groupby/GroupByQueryQueryToolChest.java | 82 +++++++++++ .../SegmentMetadataQueryQueryToolChest.java | 17 ++- .../search/SearchQueryQueryToolChest.java | 138 ++++++++++-------- .../select/SelectQueryQueryToolChest.java | 54 +++++-- .../TimeBoundaryQueryQueryToolChest.java | 13 ++ .../TimeseriesQueryQueryToolChest.java | 59 ++++++++ .../query/topn/TopNQueryQueryToolChest.java | 75 +++++++++- .../druid/client/CachingClusteredClient.java | 30 ++-- .../io/druid/client/ResultLevelCacheUtil.java | 23 ++- .../io/druid/client/cache/CacheConfig.java | 2 +- .../query/ResultLevelCachingQueryRunner.java | 108 ++++++++------ .../java/io/druid/server/QueryResource.java | 1 + 13 files changed, 474 insertions(+), 145 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index 95681dd1b067..f666a7d90aa8 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -71,4 +71,21 @@ public interface CacheStrategy> * @return A function that does the inverse of the operation that the function prepareForCache returns */ Function pullFromCache(); + + /** + * Returns a function that converts the query result set including the post-aggregation results + * into something cacheable. + * + * The resulting function must be thread-safe. + * + * @return a thread-safe function that converts the QueryType's result type into something cacheable + */ + Function prepareForResultLevelCache(); + + /** + * A function that does the inverse of the operation that the function prepareForResultLevelCache returns + * + * @return A function that does the inverse of the operation that the function prepareForResultLevelCache returns + */ + Function pullFromResultLevelCache(); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index 28d5acb42de0..f6e05c9c2fa8 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -50,6 +50,7 @@ import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.MetricManipulationFn; import io.druid.query.aggregation.MetricManipulatorFns; +import io.druid.query.aggregation.PostAggregator; import io.druid.query.cache.CacheKeyBuilder; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; @@ -477,6 +478,87 @@ public Row apply(Object input) } }; } + + @Override + public Function prepareForResultLevelCache() + { + return new Function() + { + @Override + public Object apply(Row input) + { + if (input instanceof MapBasedRow) { + final MapBasedRow row = (MapBasedRow) input; + final List retVal = Lists.newArrayListWithCapacity(1 + dims.size() + aggs.size()); + retVal.add(row.getTimestamp().getMillis()); + Map event = row.getEvent(); + for (DimensionSpec dim : dims) { + retVal.add(event.get(dim.getOutputName())); + } + for (AggregatorFactory agg : aggs) { + retVal.add(event.get(agg.getName())); + } + // Add postaggregated data to the result + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + retVal.add(event.get(postAgg.getName())); + } + return retVal; + } + + throw new ISE("Don't know how to cache input rows of type[%s]", input.getClass()); + } + }; + } + + @Override + public Function pullFromResultLevelCache() + { + return new Function() + { + private final Granularity granularity = query.getGranularity(); + + @Override + public Row apply(Object input) + { + Iterator results = ((List) input).iterator(); + + DateTime timestamp = granularity.toDateTime(((Number) results.next()).longValue()); + + Map event = Maps.newLinkedHashMap(); + Iterator dimsIter = dims.iterator(); + while (dimsIter.hasNext() && results.hasNext()) { + final DimensionSpec factory = dimsIter.next(); + event.put(factory.getOutputName(), results.next()); + } + + Iterator aggsIter = aggs.iterator(); + while (aggsIter.hasNext() && results.hasNext()) { + final AggregatorFactory factory = aggsIter.next(); + event.put(factory.getName(), factory.deserialize(results.next())); + } + + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + while (postItr.hasNext() && results.hasNext()) { + event.put(postItr.next().getName(), results.next()); + } + + if (dimsIter.hasNext() || aggsIter.hasNext() || results.hasNext()) { + throw new ISE( + "Found left over objects while reading from cache!! dimsIter[%s] aggsIter[%s] results[%s]", + dimsIter.hasNext(), + aggsIter.hasNext(), + results.hasNext() + ); + } + + return new MapBasedRow( + timestamp, + event + ); + } + }; + } + }; } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 4a921480f30f..bcbc43145c9a 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -70,7 +70,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest TYPE_REFERENCE = new TypeReference() { }; - private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; + private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[] {0x4}; private static final Function MERGE_TRANSFORM_FN = new Function() { @Override @@ -111,7 +111,8 @@ public Sequence doRun( Map context ) { - SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config); + SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes( + config); QueryPlus updatedQueryPlus = queryPlus.withQuery(updatedQuery); return new MappedSequence<>( CombiningSequence.create( @@ -222,6 +223,18 @@ public SegmentAnalysis apply(@Nullable SegmentAnalysis input) } }; } + + @Override + public Function prepareForResultLevelCache() + { + return prepareForCache(); + } + + @Override + public Function pullFromResultLevelCache() + { + return pullFromCache(); + } }; } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 74148d66aea4..f1de39a3a078 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -150,8 +150,10 @@ public CacheStrategy, Object, SearchQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.size() > 0 ? - Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); + private final List dimOutputNames = dimensionSpecs.size() > 0 + ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) + : Collections.emptyList(); @Override public boolean isCacheable(SearchQuery query, boolean willMergeRunners) @@ -163,7 +165,7 @@ public boolean isCacheable(SearchQuery query, boolean willMergeRunners) public byte[] computeCacheKey(SearchQuery query) { final DimFilter dimFilter = query.getDimensionsFilter(); - final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); + final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey(); final byte[] querySpecBytes = query.getQuery().getCacheKey(); final byte[] granularityBytes = query.getGranularity().getCacheKey(); @@ -214,8 +216,8 @@ public Function, Object> prepareForCache() public Object apply(Result input) { return dimensionSpecs.size() > 0 - ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) - : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); + ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) + : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); } }; } @@ -234,8 +236,10 @@ public Result apply(Object input) final Map outputNameMap = Maps.newHashMap(); if (hasOutputName(result)) { List cachedOutputNames = (List) result.get(2); - Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(), - "cache hit, but number of dimensions mismatch"); + Preconditions.checkArgument( + cachedOutputNames.size() == dimOutputNames.size(), + "cache hit, but number of dimensions mismatch" + ); needsRename = false; for (int idx = 0; idx < cachedOutputNames.size(); idx++) { String cachedOutputName = cachedOutputNames.get(idx); @@ -248,67 +252,79 @@ public Result apply(Object input) } return !needsRename - ? new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - if (input instanceof Map) { - return new SearchHit( - (String) ((Map) input).get("dimension"), - (String) ((Map) input).get("value"), - (Integer) ((Map) input).get("count") - ); - } else if (input instanceof SearchHit) { - return (SearchHit) input; - } else { - throw new IAE("Unknown format [%s]", input.getClass()); - } - } + ? new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + if (input instanceof Map) { + return new SearchHit( + (String) ((Map) input).get("dimension"), + (String) ((Map) input).get("value"), + (Integer) ((Map) input).get("count") + ); + } else if (input instanceof SearchHit) { + return (SearchHit) input; + } else { + throw new IAE("Unknown format [%s]", input.getClass()); } - ) + } + } ) ) - : new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - String dim = null; - String val = null; - Integer cnt = null; - if (input instanceof Map) { - dim = outputNameMap.get((String) ((Map) input).get("dimension")); - val = (String) ((Map) input).get("value"); - cnt = (Integer) ((Map) input).get("count"); - } else if (input instanceof SearchHit) { - SearchHit cached = (SearchHit) input; - dim = outputNameMap.get(cached.getDimension()); - val = cached.getValue(); - cnt = cached.getCount(); - } else { - throw new IAE("Unknown format [%s]", input.getClass()); - } - return new SearchHit(dim, val, cnt); - } - } - ) - ) - ); + ) + : new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + String dim = null; + String val = null; + Integer cnt = null; + if (input instanceof Map) { + dim = outputNameMap.get((String) ((Map) input).get("dimension")); + val = (String) ((Map) input).get("value"); + cnt = (Integer) ((Map) input).get("count"); + } else if (input instanceof SearchHit) { + SearchHit cached = (SearchHit) input; + dim = outputNameMap.get(cached.getDimension()); + val = cached.getValue(); + cnt = cached.getCount(); + } else { + throw new IAE("Unknown format [%s]", input.getClass()); + } + return new SearchHit(dim, val, cnt); + } + } + ) + ) + ); } }; } + @Override + public Function, Object> prepareForResultLevelCache() + { + return prepareForCache(); + } + + @Override + public Function> pullFromResultLevelCache() + { + return pullFromCache(); + } + private boolean hasOutputName(List cachedEntry) { /* diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 0b567ca6fc79..80061c804347 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -163,8 +163,10 @@ public CacheStrategy, Object, SelectQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.size() > 0 ? - Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); + private final List dimOutputNames = dimensionSpecs.size() > 0 + ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) + : Collections.emptyList(); @Override public boolean isCacheable(SelectQuery query, boolean willMergeRunners) @@ -176,7 +178,7 @@ public boolean isCacheable(SelectQuery query, boolean willMergeRunners) public byte[] computeCacheKey(SelectQuery query) { final DimFilter dimFilter = query.getDimensionsFilter(); - final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); + final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey(); final byte[] granularityBytes = query.getGranularity().getCacheKey(); final List dimensionSpecs = @@ -287,26 +289,36 @@ public Result apply(Object input) DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); Map pageIdentifier = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() {} - ); + resultIter.next(), new TypeReference>() + { + } + ); Set dimensionSet = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() {} + resultIter.next(), new TypeReference>() + { + } ); Set metricSet = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() {} + resultIter.next(), new TypeReference>() + { + } ); List eventHolders = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() {} - ); + resultIter.next(), new TypeReference>() + { + } + ); // check the condition that outputName of cached result should be updated if (resultIter.hasNext()) { List cachedOutputNames = (List) resultIter.next(); - Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(), - "Cache hit but different number of dimensions??"); + Preconditions.checkArgument( + cachedOutputNames.size() == dimOutputNames.size(), + "Cache hit but different number of dimensions??" + ); for (int idx = 0; idx < dimOutputNames.size(); idx++) { if (!cachedOutputNames.get(idx).equals(dimOutputNames.get(idx))) { // rename outputName in the EventHolder - for (EventHolder eventHolder: eventHolders) { + for (EventHolder eventHolder : eventHolders) { Object obj = eventHolder.getEvent().remove(cachedOutputNames.get(idx)); if (obj != null) { eventHolder.getEvent().put(dimOutputNames.get(idx), obj); @@ -328,6 +340,18 @@ public Result apply(Object input) } }; } + + @Override + public Function, Object> prepareForResultLevelCache() + { + return prepareForCache(); + } + + @Override + public Function> pullFromResultLevelCache() + { + return pullFromCache(); + } }; } @@ -411,7 +435,8 @@ public boolean apply(String input) if (query.isDescending()) { while (it.hasNext()) { Interval interval = it.next().getInterval(); - Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()).getMillis()); + Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()) + .getMillis()); if (ceiling == null || interval.getStartMillis() >= ceiling.getValue()) { it.remove(); } @@ -419,7 +444,8 @@ public boolean apply(String input) } else { while (it.hasNext()) { Interval interval = it.next().getInterval(); - Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()).getMillis()); + Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()) + .getMillis()); if (floor == null || interval.getEndMillis() <= floor.getValue()) { it.remove(); } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a36461d22346..7ccbe9219f59 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -203,6 +203,19 @@ public Result apply(Object input) } }; } + + @Override + public Function, Object> prepareForResultLevelCache() + { + return prepareForCache(); + } + + @Override + public Function> pullFromResultLevelCache() + { + return pullFromCache(); + } + }; } } diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 473775b71bc0..1af4e5f4e582 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -224,6 +224,65 @@ public Result apply(@Nullable Object input) } }; } + + @Override + public Function, Object> prepareForResultLevelCache() + { + return new Function, Object>() + { + @Override + public Object apply(final Result input) + { + TimeseriesResultValue results = input.getValue(); + final List retVal = Lists.newArrayListWithCapacity(1 + aggs.size()); + + retVal.add(input.getTimestamp().getMillis()); + for (AggregatorFactory agg : aggs) { + retVal.add(results.getMetric(agg.getName())); + } + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + retVal.add(results.getMetric(postAgg.getName())); + } + + return retVal; + } + }; + } + + @Override + public Function> pullFromResultLevelCache() + { + return new Function>() + { + private final Granularity granularity = query.getGranularity(); + + @Override + public Result apply(@Nullable Object input) + { + List results = (List) input; + Map retVal = Maps.newLinkedHashMap(); + + Iterator aggsIter = aggs.iterator(); + Iterator resultIter = results.iterator(); + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + + DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); + + while (aggsIter.hasNext() && resultIter.hasNext()) { + final AggregatorFactory factory = aggsIter.next(); + retVal.put(factory.getName(), factory.deserialize(resultIter.next())); + } + + while (postItr.hasNext() && resultIter.hasNext()) { + retVal.put(postItr.next().getName(), resultIter.next()); + } + return new Result( + timestamp, + new TimeseriesResultValue(retVal) + ); + } + }; + } }; } diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index 81761afc6a37..ea21025a2093 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -293,7 +293,6 @@ public TypeReference> getResultTypeReference() } - @Override public CacheStrategy, Object, TopNQuery> getCacheStrategy(final TopNQuery query) { @@ -409,6 +408,80 @@ public Result apply(Object input) } }; } + + @Override + public Function, Object> prepareForResultLevelCache() + { + return new Function, Object>() + { + private final String[] aggFactoryNames = extractFactoryName(query.getAggregatorSpecs()); + + @Override + public Object apply(final Result input) + { + List results = Lists.newArrayList(input.getValue()); + final List retVal = Lists.newArrayListWithCapacity(results.size() + 1); + + // make sure to preserve timezone information when caching results + retVal.add(input.getTimestamp().getMillis()); + for (DimensionAndMetricValueExtractor result : results) { + List vals = Lists.newArrayListWithCapacity(aggFactoryNames.length + 2); + vals.add(result.getDimensionValue(query.getDimensionSpec().getOutputName())); + for (String aggName : aggFactoryNames) { + vals.add(result.getMetric(aggName)); + } + // Add post aggregated results + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + vals.add(result.getMetric(postAgg.getName())); + } + retVal.add(vals); + } + return retVal; + } + }; + } + + @Override + public Function> pullFromResultLevelCache() + { + return new Function>() + { + private final Granularity granularity = query.getGranularity(); + + @Override + public Result apply(Object input) + { + List results = (List) input; + List> retVal = Lists.newArrayListWithCapacity(results.size()); + + Iterator inputIter = results.iterator(); + DateTime timestamp = granularity.toDateTime(((Number) inputIter.next()).longValue()); + + while (inputIter.hasNext()) { + List result = (List) inputIter.next(); + Map vals = Maps.newLinkedHashMap(); + + Iterator aggIter = aggs.iterator(); + Iterator resultIter = result.iterator(); + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + + vals.put(query.getDimensionSpec().getOutputName(), resultIter.next()); + + while (aggIter.hasNext() && resultIter.hasNext()) { + final AggregatorFactory factory = aggIter.next(); + vals.put(factory.getName(), factory.deserialize(resultIter.next())); + } + + while (postItr.hasNext() && resultIter.hasNext()) { + vals.put(postItr.next().getName(), resultIter.next()); + } + retVal.add(vals); + } + + return new Result<>(timestamp, new TopNResultValue(retVal)); + } + }; + } }; } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 87a64fb3ac8c..639a256319ce 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -255,7 +255,8 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + @Nullable + TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } @@ -265,21 +266,28 @@ Sequence run(final UnaryOperator> time } final Set segments = computeSegmentsToQuery(timeline); - @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); - - if (responseContext.containsKey("resultLevelCacheEnabled")) { - @Nullable final String prevResultCacheKeyFromEtag = (String) responseContext.get("prevResultSetIdentifier"); - @Nullable final String newResultCacheKeyFromEtag = computeCurrentEtag(segments, queryCacheKey); - responseContext.remove("resultLevelCacheEnabled"); - responseContext.put("currentResultSetIdentifier", newResultCacheKeyFromEtag); - if (newResultCacheKeyFromEtag != null && newResultCacheKeyFromEtag.equals(prevResultCacheKeyFromEtag)) { + @Nullable + final byte[] queryCacheKey = computeQueryCacheKey(); + + if (responseContext.containsKey(CacheConfig.ENABLE_RESULTLEVEL_CACHE) + || query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { + @Nullable + final String prevEtag = (responseContext.get(QueryResource.EXISTING_RESULT_ID) == null) + ? (String) query.getContext() + .get(QueryResource.HEADER_IF_NONE_MATCH) + : (String) responseContext.get(QueryResource.EXISTING_RESULT_ID); + @Nullable + final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } } if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); - @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + @Nullable + final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + @Nullable + final String currentEtag = computeCurrentEtag(segments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } diff --git a/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java index 1b0a1dd92f49..24725b917c7b 100644 --- a/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java +++ b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java @@ -19,6 +19,7 @@ package io.druid.client; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import io.druid.client.cache.Cache; @@ -29,10 +30,9 @@ import io.druid.query.QueryContexts; +import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.util.Collections; -import java.util.Map; - +import java.nio.ByteBuffer; public class ResultLevelCacheUtil { @@ -51,16 +51,23 @@ public static void populate( Cache.NamedKey key, Iterable results, int cacheLimit, - String etag + String resultSetId ) { try { - Map> vals = Collections.singletonMap(etag, results); - byte[] bytes = mapper.writeValueAsBytes(vals); - if (cacheLimit > 0 && bytes.length > cacheLimit) { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (JsonGenerator gen = mapper.getFactory().createGenerator(bytes)) { + // Save the resultSetId and its length + bytes.write(ByteBuffer.allocate(Integer.BYTES).putInt(resultSetId.length()).array()); + bytes.write(StringUtils.toUtf8(resultSetId)); + for (Object result : results) { + gen.writeObject(result); + } + } + if (cacheLimit > 0 && bytes.size() > cacheLimit) { return; } - cache.put(key, bytes); + cache.put(key, bytes.toByteArray()); } catch (IOException e) { throw Throwables.propagate(e); diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index d2d8baf15d2f..782779b6a3c3 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -30,7 +30,7 @@ public class CacheConfig { public static final String USE_CACHE = "useCache"; public static final String POPULATE_CACHE = "populateCache"; - + public static final String ENABLE_RESULTLEVEL_CACHE = "resultLevelCacheEnabled"; // The defaults defined here for cache related parameters are different from the QueryContext defaults due to legacy reasons. // They should be made the same at some point in the future. @JsonProperty diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index b60a4a763256..8161135e52c6 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.collect.Iterators; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -35,11 +36,13 @@ import io.druid.java.util.common.guava.SequenceWrapper; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.logger.Logger; +import io.druid.server.QueryResource; import javax.annotation.Nullable; import java.io.IOException; -import java.util.Collections; +import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; @@ -88,37 +91,32 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) if (useResultCache) { final String cacheKeyStr = StringUtils.fromUtf8(strategy.computeCacheKey(query)); - - Map> resultMap; @Nullable final byte[] cachedResultSet = fetchResultsFromResultLevelCache(cacheKeyStr); + String existingResultSetId = extractEtagFromResults(cachedResultSet); - resultMap = extractMapFromResults(cachedResultSet); - responseContext.put("resultLevelCacheEnabled", true); + responseContext.put(CacheConfig.ENABLE_RESULTLEVEL_CACHE, true); - if (!resultMap.isEmpty()) { - String resultSetIdentifier = resultMap.entrySet().iterator().next().getKey(); - responseContext.put("prevResultSetIdentifier", resultSetIdentifier); + if (existingResultSetId != null) { + responseContext.put(QueryResource.EXISTING_RESULT_ID, existingResultSetId); } - @Nullable - ResultLevelCachePopulator resultLevelCachePopulator = createResultLevelCachePopulator( - cachedResultSet, - cacheKeyStr - ); Sequence resultFromClient = baseRunner.run( queryPlus, responseContext ); + String newResultSetId = (String) responseContext.get(QueryResource.HEADER_ETAG); + + @Nullable + ResultLevelCachePopulator resultLevelCachePopulator = createResultLevelCachePopulator( + cacheKeyStr, + newResultSetId + ); - String currentResultSetIdentifier = (String) responseContext.get("currentResultSetIdentifier"); - String prevResultSetIdentifier = (String) responseContext.get("prevResultSetIdentifier"); - responseContext.remove("currentResultSetIdentifier"); - responseContext.remove("prevResultSetIdentifier"); - if (currentResultSetIdentifier != null && currentResultSetIdentifier.equals(prevResultSetIdentifier)) { + responseContext.remove(QueryResource.EXISTING_RESULT_ID); + if (newResultSetId != null && newResultSetId.equals(existingResultSetId)) { log.info("Return cached result set as there is no change in identifiers for query %s ", query.getId()); - Iterable cachedResults = resultMap.entrySet().iterator().next().getValue(); - return fetchSequenceFromResults(cachedResults, strategy); + return deserializeResults(cachedResultSet, strategy, existingResultSetId); } else { return Sequences.wrap(Sequences.map( resultFromClient, @@ -137,8 +135,9 @@ public T apply(T input) public void after(boolean isDone, Throwable thrown) throws Exception { if (resultLevelCachePopulator != null) { - // The resultset identifier is cached along with the resultset - resultLevelCachePopulator.populateResults(currentResultSetIdentifier); + // The resultset identifier and its length is cached along with the resultset + resultLevelCachePopulator.populateResults(newResultSetId); + log.info("Cache population complete for query %s", query.getId()); } } }); @@ -156,7 +155,7 @@ private T cacheResultEntry( T resultEntry ) { - final Function cacheFn = strategy.prepareForCache(); + final Function cacheFn = strategy.prepareForResultLevelCache(); if (resultLevelCachePopulator != null) { resultLevelCachePopulator.cacheFutures .add(backgroundExecutorService.submit(() -> cacheFn.apply(resultEntry))); @@ -169,49 +168,64 @@ private byte[] fetchResultsFromResultLevelCache( ) { if (useResultCache && queryCacheKey != null) { - log.info("Fetching cached result for query: %s", query.getId()); return cache.get(ResultLevelCacheUtil.computeResultLevelCacheKey(queryCacheKey)); } return null; } - private Map> extractMapFromResults( + private String extractEtagFromResults( final byte[] cachedResult ) { if (cachedResult == null) { - return Collections.emptyMap(); - } - final TypeReference>> cacheObjectClazz = new TypeReference>>() - { - }; - try { - Map> res = objectMapper.readValue( - cachedResult, cacheObjectClazz - ); - return res; - } - catch (IOException ioe) { - log.error("Error parsing cached result set."); - return Collections.emptyMap(); + return null; } + log.info("Fetching result level cache identifier for query: %s", query.getId()); + int etagLength = ByteBuffer.wrap(cachedResult, 0, Integer.BYTES).getInt(); + return StringUtils.fromUtf8(Arrays.copyOfRange(cachedResult, Integer.BYTES, etagLength + Integer.BYTES)); } - private Sequence fetchSequenceFromResults( - Iterable cachedResult, CacheStrategy strategy + private Sequence deserializeResults( + final byte[] cachedResult, CacheStrategy strategy, String resultSetId ) { - if (strategy == null) { + if (cachedResult == null) { return null; } - final Function pullFromCacheFunction = strategy.pullFromCache(); - Sequence cachedSequence = Sequences.simple(() -> cachedResult.iterator()); + final Function pullFromCacheFunction = strategy.pullFromResultLevelCache(); + final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); + //Skip the resultsetID and its length bytes + byte[] prunedCacheData = Arrays.copyOfRange( + cachedResult, + Integer.BYTES + resultSetId.length(), + cachedResult.length + ); + Sequence cachedSequence = Sequences.simple(() -> { + try { + if (cachedResult.length == 0) { + return Iterators.emptyIterator(); + } + + return objectMapper.readValues( + objectMapper.getFactory().createParser(prunedCacheData), + cacheObjectClazz + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + return Sequences.map(cachedSequence, pullFromCacheFunction); } - private ResultLevelCachePopulator createResultLevelCachePopulator(byte[] cachedResultSet, String cacheKeyStr) + private ResultLevelCachePopulator createResultLevelCachePopulator( + String cacheKeyStr, + String resultSetId + ) { - if (cachedResultSet == null && populateResultCache) { + // Results need to be cached only if all the segments are historical segments + if (resultSetId != null && populateResultCache) { return new ResultLevelCachePopulator( cache, objectMapper, @@ -274,7 +288,7 @@ public void onSuccess(List cacheData) @Override public void onFailure(Throwable throwable) { - log.error(throwable, "Result-Level caching failed"); + log.error(throwable, "Result-Level caching failed!"); } }, backgroundExecutorService diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 8c1208709116..8d0eea855620 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -85,6 +85,7 @@ public class QueryResource implements QueryCountStatsProvider public static final String HEADER_IF_NONE_MATCH = "If-None-Match"; public static final String HEADER_ETAG = "ETag"; + public static final String EXISTING_RESULT_ID = "existingResultSetIdentifier"; protected final QueryLifecycleFactory queryLifecycleFactory; protected final ObjectMapper jsonMapper; From 0d409ea52e550021228690da21807279917710d3 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Sun, 3 Dec 2017 19:47:44 -0600 Subject: [PATCH 11/25] Fix indents --- .../SegmentMetadataQueryQueryToolChest.java | 5 +- .../search/SearchQueryQueryToolChest.java | 126 +++++++++--------- .../select/SelectQueryQueryToolChest.java | 42 ++---- 3 files changed, 77 insertions(+), 96 deletions(-) diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index bcbc43145c9a..c8e2ec20895f 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -70,7 +70,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest TYPE_REFERENCE = new TypeReference() { }; - private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[] {0x4}; + private static final byte[] SEGMENT_METADATA_CACHE_PREFIX = new byte[]{0x4}; private static final Function MERGE_TRANSFORM_FN = new Function() { @Override @@ -111,8 +111,7 @@ public Sequence doRun( Map context ) { - SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes( - config); + SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config); QueryPlus updatedQueryPlus = queryPlus.withQuery(updatedQuery); return new MappedSequence<>( CombiningSequence.create( diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index f1de39a3a078..fcabfdf9f47a 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -150,10 +150,8 @@ public CacheStrategy, Object, SearchQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.size() > 0 - ? - Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) - : Collections.emptyList(); + private final List dimOutputNames = dimensionSpecs.size() > 0 ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); @Override public boolean isCacheable(SearchQuery query, boolean willMergeRunners) @@ -165,7 +163,7 @@ public boolean isCacheable(SearchQuery query, boolean willMergeRunners) public byte[] computeCacheKey(SearchQuery query) { final DimFilter dimFilter = query.getDimensionsFilter(); - final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey(); + final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); final byte[] querySpecBytes = query.getQuery().getCacheKey(); final byte[] granularityBytes = query.getGranularity().getCacheKey(); @@ -216,8 +214,8 @@ public Function, Object> prepareForCache() public Object apply(Result input) { return dimensionSpecs.size() > 0 - ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) - : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); + ? Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue(), dimOutputNames) + : Lists.newArrayList(input.getTimestamp().getMillis(), input.getValue()); } }; } @@ -236,10 +234,8 @@ public Result apply(Object input) final Map outputNameMap = Maps.newHashMap(); if (hasOutputName(result)) { List cachedOutputNames = (List) result.get(2); - Preconditions.checkArgument( - cachedOutputNames.size() == dimOutputNames.size(), - "cache hit, but number of dimensions mismatch" - ); + Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(), + "cache hit, but number of dimensions mismatch"); needsRename = false; for (int idx = 0; idx < cachedOutputNames.size(); idx++) { String cachedOutputName = cachedOutputNames.get(idx); @@ -252,63 +248,63 @@ public Result apply(Object input) } return !needsRename - ? new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - if (input instanceof Map) { - return new SearchHit( - (String) ((Map) input).get("dimension"), - (String) ((Map) input).get("value"), - (Integer) ((Map) input).get("count") - ); - } else if (input instanceof SearchHit) { - return (SearchHit) input; - } else { - throw new IAE("Unknown format [%s]", input.getClass()); + ? new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + if (input instanceof Map) { + return new SearchHit( + (String) ((Map) input).get("dimension"), + (String) ((Map) input).get("value"), + (Integer) ((Map) input).get("count") + ); + } else if (input instanceof SearchHit) { + return (SearchHit) input; + } else { + throw new IAE("Unknown format [%s]", input.getClass()); + } + } } - } - } + ) ) ) - ) - : new Result<>( - DateTimes.utc(((Number) result.get(0)).longValue()), - new SearchResultValue( - Lists.transform( - (List) result.get(1), - new Function() - { - @Override - public SearchHit apply(@Nullable Object input) - { - String dim = null; - String val = null; - Integer cnt = null; - if (input instanceof Map) { - dim = outputNameMap.get((String) ((Map) input).get("dimension")); - val = (String) ((Map) input).get("value"); - cnt = (Integer) ((Map) input).get("count"); - } else if (input instanceof SearchHit) { - SearchHit cached = (SearchHit) input; - dim = outputNameMap.get(cached.getDimension()); - val = cached.getValue(); - cnt = cached.getCount(); - } else { - throw new IAE("Unknown format [%s]", input.getClass()); - } - return new SearchHit(dim, val, cnt); - } - } - ) - ) - ); + : new Result<>( + DateTimes.utc(((Number) result.get(0)).longValue()), + new SearchResultValue( + Lists.transform( + (List) result.get(1), + new Function() + { + @Override + public SearchHit apply(@Nullable Object input) + { + String dim = null; + String val = null; + Integer cnt = null; + if (input instanceof Map) { + dim = outputNameMap.get((String) ((Map) input).get("dimension")); + val = (String) ((Map) input).get("value"); + cnt = (Integer) ((Map) input).get("count"); + } else if (input instanceof SearchHit) { + SearchHit cached = (SearchHit) input; + dim = outputNameMap.get(cached.getDimension()); + val = cached.getValue(); + cnt = cached.getCount(); + } else { + throw new IAE("Unknown format [%s]", input.getClass()); + } + return new SearchHit(dim, val, cnt); + } + } + ) + ) + ); } }; } diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 80061c804347..8e9e4a3b8d8c 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -163,10 +163,8 @@ public CacheStrategy, Object, SelectQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.size() > 0 - ? - Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) - : Collections.emptyList(); + private final List dimOutputNames = dimensionSpecs.size() > 0 ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); @Override public boolean isCacheable(SelectQuery query, boolean willMergeRunners) @@ -178,7 +176,7 @@ public boolean isCacheable(SelectQuery query, boolean willMergeRunners) public byte[] computeCacheKey(SelectQuery query) { final DimFilter dimFilter = query.getDimensionsFilter(); - final byte[] filterBytes = dimFilter == null ? new byte[] {} : dimFilter.getCacheKey(); + final byte[] filterBytes = dimFilter == null ? new byte[]{} : dimFilter.getCacheKey(); final byte[] granularityBytes = query.getGranularity().getCacheKey(); final List dimensionSpecs = @@ -289,36 +287,26 @@ public Result apply(Object input) DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); Map pageIdentifier = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() - { - } - ); + resultIter.next(), new TypeReference>() {} + ); Set dimensionSet = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() - { - } + resultIter.next(), new TypeReference>() {} ); Set metricSet = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() - { - } + resultIter.next(), new TypeReference>() {} ); List eventHolders = jsonMapper.convertValue( - resultIter.next(), new TypeReference>() - { - } - ); + resultIter.next(), new TypeReference>() {} + ); // check the condition that outputName of cached result should be updated if (resultIter.hasNext()) { List cachedOutputNames = (List) resultIter.next(); - Preconditions.checkArgument( - cachedOutputNames.size() == dimOutputNames.size(), - "Cache hit but different number of dimensions??" - ); + Preconditions.checkArgument(cachedOutputNames.size() == dimOutputNames.size(), + "Cache hit but different number of dimensions??"); for (int idx = 0; idx < dimOutputNames.size(); idx++) { if (!cachedOutputNames.get(idx).equals(dimOutputNames.get(idx))) { // rename outputName in the EventHolder - for (EventHolder eventHolder : eventHolders) { + for (EventHolder eventHolder: eventHolders) { Object obj = eventHolder.getEvent().remove(cachedOutputNames.get(idx)); if (obj != null) { eventHolder.getEvent().put(dimOutputNames.get(idx), obj); @@ -435,8 +423,7 @@ public boolean apply(String input) if (query.isDescending()) { while (it.hasNext()) { Interval interval = it.next().getInterval(); - Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()) - .getMillis()); + Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()).getMillis()); if (ceiling == null || interval.getStartMillis() >= ceiling.getValue()) { it.remove(); } @@ -444,8 +431,7 @@ public boolean apply(String input) } else { while (it.hasNext()) { Interval interval = it.next().getInterval(); - Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()) - .getMillis()); + Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()).getMillis()); if (floor == null || interval.getEndMillis() <= floor.getValue()) { it.remove(); } From ead2dd9f78ce84ae08e6fbfea60738291504d08c Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Wed, 13 Dec 2017 11:13:57 -0600 Subject: [PATCH 12/25] Check byte size for exceeding cache limit --- .../java/io/druid/query/CacheStrategy.java | 10 +- .../SegmentMetadataQueryQueryToolChest.java | 12 -- .../search/SearchQueryQueryToolChest.java | 12 -- .../select/SelectQueryQueryToolChest.java | 12 -- .../TimeBoundaryQueryQueryToolChest.java | 13 -- .../TimeseriesQueryQueryToolChestTest.java | 27 +++- .../topn/TopNQueryQueryToolChestTest.java | 33 ++++- .../druid/client/CachingClusteredClient.java | 14 -- .../io/druid/client/ResultLevelCacheUtil.java | 36 +---- .../io/druid/client/cache/CacheConfig.java | 1 - .../query/ResultLevelCachingQueryRunner.java | 130 +++++++++--------- .../java/io/druid/server/QueryResource.java | 1 - 12 files changed, 133 insertions(+), 168 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index f666a7d90aa8..67a3b986c7a0 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -80,12 +80,18 @@ public interface CacheStrategy> * * @return a thread-safe function that converts the QueryType's result type into something cacheable */ - Function prepareForResultLevelCache(); + default Function prepareForResultLevelCache() + { + return prepareForCache(); + } /** * A function that does the inverse of the operation that the function prepareForResultLevelCache returns * * @return A function that does the inverse of the operation that the function prepareForResultLevelCache returns */ - Function pullFromResultLevelCache(); + default Function pullFromResultLevelCache() + { + return pullFromCache(); + } } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index c8e2ec20895f..4a921480f30f 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -222,18 +222,6 @@ public SegmentAnalysis apply(@Nullable SegmentAnalysis input) } }; } - - @Override - public Function prepareForResultLevelCache() - { - return prepareForCache(); - } - - @Override - public Function pullFromResultLevelCache() - { - return pullFromCache(); - } }; } diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index fcabfdf9f47a..74148d66aea4 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -309,18 +309,6 @@ public SearchHit apply(@Nullable Object input) }; } - @Override - public Function, Object> prepareForResultLevelCache() - { - return prepareForCache(); - } - - @Override - public Function> pullFromResultLevelCache() - { - return pullFromCache(); - } - private boolean hasOutputName(List cachedEntry) { /* diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 8e9e4a3b8d8c..0b567ca6fc79 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -328,18 +328,6 @@ public Result apply(Object input) } }; } - - @Override - public Function, Object> prepareForResultLevelCache() - { - return prepareForCache(); - } - - @Override - public Function> pullFromResultLevelCache() - { - return pullFromCache(); - } }; } diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index 7ccbe9219f59..a36461d22346 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -203,19 +203,6 @@ public Result apply(Object input) } }; } - - @Override - public Function, Object> prepareForResultLevelCache() - { - return prepareForCache(); - } - - @Override - public Function> pullFromResultLevelCache() - { - return pullFromCache(); - } - }; } } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index b23cb0fe28fe..9dc04e53493e 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -32,6 +32,8 @@ import io.druid.query.TableDataSource; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.query.aggregation.LongSumAggregatorFactory; +import io.druid.query.aggregation.PostAggregator; +import io.druid.query.aggregation.post.ConstantPostAggregator; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.segment.TestHelper; import io.druid.segment.VirtualColumns; @@ -77,12 +79,12 @@ public void testCacheStrategy() throws Exception new CountAggregatorFactory("metric1"), new LongSumAggregatorFactory("metric0", "metric0") ), - null, + ImmutableList.of(new ConstantPostAggregator("post", 10)), null ) ); - final Result result = new Result<>( + final Result result1 = new Result<>( // test timestamps that result in integer size millis DateTimes.utc(123L), new TimeseriesResultValue( @@ -90,7 +92,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache().apply(result); + Object preparedValue = strategy.prepareForCache().apply(result1); ObjectMapper objectMapper = TestHelper.getJsonMapper(); Object fromCacheValue = objectMapper.readValue( @@ -100,7 +102,24 @@ public void testCacheStrategy() throws Exception Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); - Assert.assertEquals(result, fromCacheResult); + Assert.assertEquals(result1, fromCacheResult); + + final Result result2 = new Result<>( + // test timestamps that result in integer size millis + DateTimes.utc(123L), + new TimeseriesResultValue( + ImmutableMap.of("metric1", 2, "metric0", 3, "post", 10) + ) + ); + + Object preparedResultLevelCacheValue = strategy.prepareForResultLevelCache().apply(result2); + Object fromResultLevelCacheValue = objectMapper.readValue( + objectMapper.writeValueAsBytes(preparedResultLevelCacheValue), + strategy.getCacheObjectClazz() + ); + + Result fromResultLevelCacheRes = strategy.pullFromResultLevelCache().apply(fromResultLevelCacheValue); + Assert.assertEquals(result2, fromResultLevelCacheRes); } @Test diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index e1d570bf1dd3..939277ce19ba 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -78,7 +78,7 @@ public void testCacheStrategy() throws Exception ) ); - final Result result = new Result<>( + final Result result1 = new Result<>( // test timestamps that result in integer size millis DateTimes.utc(123L), new TopNResultValue( @@ -92,7 +92,7 @@ public void testCacheStrategy() throws Exception ); Object preparedValue = strategy.prepareForCache().apply( - result + result1 ); ObjectMapper objectMapper = TestHelper.getJsonMapper(); @@ -103,7 +103,34 @@ public void testCacheStrategy() throws Exception Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); - Assert.assertEquals(result, fromCacheResult); + Assert.assertEquals(result1, fromCacheResult); + + final Result result2 = new Result<>( + // test timestamps that result in integer size millis + DateTimes.utc(123L), + new TopNResultValue( + Arrays.asList( + ImmutableMap.of( + "test", "val1", + "metric1", 2, + "post", 10 + ) + ) + ) + ); + + Object preparedResultCacheValue = strategy.prepareForResultLevelCache().apply( + result2 + ); + + Object fromResultCacheValue = objectMapper.readValue( + objectMapper.writeValueAsBytes(preparedResultCacheValue), + strategy.getCacheObjectClazz() + ); + + Result fromResultCacheResult = strategy.pullFromResultLevelCache().apply(fromResultCacheValue); + Assert.assertEquals(result2, fromResultCacheResult); + } @Test diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 639a256319ce..b23e3521b7a7 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -269,20 +269,6 @@ Sequence run(final UnaryOperator> time @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); - if (responseContext.containsKey(CacheConfig.ENABLE_RESULTLEVEL_CACHE) - || query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable - final String prevEtag = (responseContext.get(QueryResource.EXISTING_RESULT_ID) == null) - ? (String) query.getContext() - .get(QueryResource.HEADER_IF_NONE_MATCH) - : (String) responseContext.get(QueryResource.EXISTING_RESULT_ID); - @Nullable - final String currentEtag = computeCurrentEtag(segments, queryCacheKey); - if (currentEtag != null && currentEtag.equals(prevEtag)) { - return Sequences.empty(); - } - } - if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); diff --git a/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java index 24725b917c7b..f55d2e6bba68 100644 --- a/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java +++ b/server/src/main/java/io/druid/client/ResultLevelCacheUtil.java @@ -19,23 +19,18 @@ package io.druid.client; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Throwables; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.java.util.common.logger.Logger; import io.druid.java.util.common.StringUtils; import io.druid.query.CacheStrategy; import io.druid.query.Query; import io.druid.query.QueryContexts; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.nio.ByteBuffer; - public class ResultLevelCacheUtil { + private static final Logger log = new Logger(ResultLevelCacheUtil.class); + public static Cache.NamedKey computeResultLevelCacheKey( String resultLevelCacheIdentifier ) @@ -47,31 +42,12 @@ public static Cache.NamedKey computeResultLevelCacheKey( public static void populate( Cache cache, - ObjectMapper mapper, Cache.NamedKey key, - Iterable results, - int cacheLimit, - String resultSetId + byte[] resultBytes ) { - try { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (JsonGenerator gen = mapper.getFactory().createGenerator(bytes)) { - // Save the resultSetId and its length - bytes.write(ByteBuffer.allocate(Integer.BYTES).putInt(resultSetId.length()).array()); - bytes.write(StringUtils.toUtf8(resultSetId)); - for (Object result : results) { - gen.writeObject(result); - } - } - if (cacheLimit > 0 && bytes.size() > cacheLimit) { - return; - } - cache.put(key, bytes.toByteArray()); - } - catch (IOException e) { - throw Throwables.propagate(e); - } + log.debug("Populating results into cache"); + cache.put(key, resultBytes); } public static boolean useResultLevelCacheOnBrokers( diff --git a/server/src/main/java/io/druid/client/cache/CacheConfig.java b/server/src/main/java/io/druid/client/cache/CacheConfig.java index 782779b6a3c3..eaa22c6cde9d 100644 --- a/server/src/main/java/io/druid/client/cache/CacheConfig.java +++ b/server/src/main/java/io/druid/client/cache/CacheConfig.java @@ -30,7 +30,6 @@ public class CacheConfig { public static final String USE_CACHE = "useCache"; public static final String POPULATE_CACHE = "populateCache"; - public static final String ENABLE_RESULTLEVEL_CACHE = "resultLevelCacheEnabled"; // The defaults defined here for cache related parameters are different from the QueryContext defaults due to legacy reasons. // They should be made the same at some point in the future. @JsonProperty diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 8161135e52c6..1c1df66c0e1d 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -19,18 +19,19 @@ package io.druid.query; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.ResultLevelCacheUtil; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.java.util.common.RE; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.SequenceWrapper; @@ -40,26 +41,26 @@ import javax.annotation.Nullable; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; public class ResultLevelCachingQueryRunner implements QueryRunner { private static final Logger log = new Logger(ResultLevelCachingQueryRunner.class); private final QueryRunner baseRunner; - private final ListeningExecutorService backgroundExecutorService; - private QueryToolChest queryToolChest; + private final ListeningExecutorService cachingExec; private ObjectMapper objectMapper; private final Cache cache; private final CacheConfig cacheConfig; private final boolean useResultCache; private final boolean populateResultCache; - private final Query query; + private Query query; private final CacheStrategy> strategy; @@ -74,8 +75,7 @@ public ResultLevelCachingQueryRunner( ) { this.baseRunner = baseRunner; - this.queryToolChest = queryToolChest; - this.backgroundExecutorService = MoreExecutors.listeningDecorator(cachingExec); + this.cachingExec = MoreExecutors.listeningDecorator(cachingExec); this.objectMapper = objectMapper; this.cache = cache; this.cacheConfig = cacheConfig; @@ -91,33 +91,29 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) if (useResultCache) { final String cacheKeyStr = StringUtils.fromUtf8(strategy.computeCacheKey(query)); - @Nullable final byte[] cachedResultSet = fetchResultsFromResultLevelCache(cacheKeyStr); String existingResultSetId = extractEtagFromResults(cachedResultSet); + existingResultSetId = existingResultSetId == null ? "" : existingResultSetId; - responseContext.put(CacheConfig.ENABLE_RESULTLEVEL_CACHE, true); - - if (existingResultSetId != null) { - responseContext.put(QueryResource.EXISTING_RESULT_ID, existingResultSetId); + if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) == null) { + query = query.withOverriddenContext( + ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); } - Sequence resultFromClient = baseRunner.run( - queryPlus, + QueryPlus.wrap(query), responseContext ); String newResultSetId = (String) responseContext.get(QueryResource.HEADER_ETAG); - @Nullable - ResultLevelCachePopulator resultLevelCachePopulator = createResultLevelCachePopulator( - cacheKeyStr, - newResultSetId - ); - - responseContext.remove(QueryResource.EXISTING_RESULT_ID); if (newResultSetId != null && newResultSetId.equals(existingResultSetId)) { - log.info("Return cached result set as there is no change in identifiers for query %s ", query.getId()); + log.debug("Return cached result set as there is no change in identifiers for query %s ", query.getId()); return deserializeResults(cachedResultSet, strategy, existingResultSetId); } else { + @Nullable + ResultLevelCachePopulator resultLevelCachePopulator = createResultLevelCachePopulator( + cacheKeyStr, + newResultSetId + ); return Sequences.wrap(Sequences.map( resultFromClient, new Function() @@ -137,7 +133,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception if (resultLevelCachePopulator != null) { // The resultset identifier and its length is cached along with the resultset resultLevelCachePopulator.populateResults(newResultSetId); - log.info("Cache population complete for query %s", query.getId()); + log.debug("Cache population complete for query %s", query.getId()); } } }); @@ -158,7 +154,7 @@ private T cacheResultEntry( final Function cacheFn = strategy.prepareForResultLevelCache(); if (resultLevelCachePopulator != null) { resultLevelCachePopulator.cacheFutures - .add(backgroundExecutorService.submit(() -> cacheFn.apply(resultEntry))); + .add(cachingExec.submit(() -> cacheFn.apply(resultEntry))); } return resultEntry; } @@ -180,7 +176,7 @@ private String extractEtagFromResults( if (cachedResult == null) { return null; } - log.info("Fetching result level cache identifier for query: %s", query.getId()); + log.debug("Fetching result level cache identifier for query: %s", query.getId()); int etagLength = ByteBuffer.wrap(cachedResult, 0, Integer.BYTES).getInt(); return StringUtils.fromUtf8(Arrays.copyOfRange(cachedResult, Integer.BYTES, etagLength + Integer.BYTES)); } @@ -190,16 +186,11 @@ private Sequence deserializeResults( ) { if (cachedResult == null) { - return null; + log.error("Cached result set is null"); } final Function pullFromCacheFunction = strategy.pullFromResultLevelCache(); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); //Skip the resultsetID and its length bytes - byte[] prunedCacheData = Arrays.copyOfRange( - cachedResult, - Integer.BYTES + resultSetId.length(), - cachedResult.length - ); Sequence cachedSequence = Sequences.simple(() -> { try { if (cachedResult.length == 0) { @@ -207,12 +198,16 @@ private Sequence deserializeResults( } return objectMapper.readValues( - objectMapper.getFactory().createParser(prunedCacheData), + objectMapper.getFactory().createParser(Arrays.copyOfRange( + cachedResult, + Integer.BYTES + resultSetId.length(), + cachedResult.length + )), cacheObjectClazz ); } catch (IOException e) { - throw new RuntimeException(e); + throw new RE(e, "Failed to retrieve results from cache for query ID [%s]", query.getId()); } }); @@ -231,7 +226,7 @@ private ResultLevelCachePopulator createResultLevelCachePopulator( objectMapper, ResultLevelCacheUtil.computeResultLevelCacheKey(cacheKeyStr), cacheConfig, - backgroundExecutorService + cachingExec ); } else { return null; @@ -262,37 +257,44 @@ private ResultLevelCachePopulator( this.backgroundExecutorService = backgroundExecutorService; } - public void populateResults( - String resultSetIdentifier - ) + public void populateResults(String resultSetIdentifier) { - Futures.addCallback( - Futures.allAsList(cacheFutures), - new FutureCallback>() - { - @Override - public void onSuccess(List cacheData) - { - ResultLevelCacheUtil.populate( - cache, - mapper, - key, - cacheData, - cacheConfig.getResultLevelCacheLimit(), - resultSetIdentifier - ); - // Help out GC by making sure all references are gone - cacheFutures.clear(); - } + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try { + // Save the resultSetId and its length + bytes.write(ByteBuffer.allocate(Integer.BYTES).putInt(resultSetIdentifier.length()).array()); + bytes.write(StringUtils.toUtf8(resultSetIdentifier)); + byte[] resultBytes = fetchResultBytes(bytes, cacheConfig.getCacheBulkMergeLimit()); + if (resultBytes != null) { + ResultLevelCacheUtil.populate( + cache, + key, + resultBytes + ); + } + // Help out GC by making sure all references are gone + cacheFutures.clear(); + } + catch (IOException ioe) { + log.error("Failed to write cached values for query %s", query.getId()); + } + } - @Override - public void onFailure(Throwable throwable) - { - log.error(throwable, "Result-Level caching failed!"); - } - }, - backgroundExecutorService - ); + private byte[] fetchResultBytes(ByteArrayOutputStream resultStream, int cacheLimit) + { + for (ListenableFuture lsf : cacheFutures) { + try (JsonGenerator gen = mapper.getFactory().createGenerator(resultStream)) { + gen.writeObject(lsf.get()); + if (cacheLimit > 0 && resultStream.size() > cacheLimit) { + return null; + } + } + catch (ExecutionException | InterruptedException | IOException ex) { + log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); + return null; + } + } + return resultStream.toByteArray(); } } } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 8d0eea855620..8c1208709116 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -85,7 +85,6 @@ public class QueryResource implements QueryCountStatsProvider public static final String HEADER_IF_NONE_MATCH = "If-None-Match"; public static final String HEADER_ETAG = "ETag"; - public static final String EXISTING_RESULT_ID = "existingResultSetIdentifier"; protected final QueryLifecycleFactory queryLifecycleFactory; protected final ObjectMapper jsonMapper; From 8abf5acc22900d2c3c101f3a229bd0e978e3a8b5 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Wed, 13 Dec 2017 11:23:50 -0600 Subject: [PATCH 13/25] Fix indents --- .../java/io/druid/client/CachingClusteredClient.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index b23e3521b7a7..b6474a9d5b47 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -255,8 +255,7 @@ private ImmutableMap makeDownstreamQueryContext() Sequence run(final UnaryOperator> timelineConverter) { - @Nullable - TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); + @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { return Sequences.empty(); } @@ -266,14 +265,11 @@ Sequence run(final UnaryOperator> time } final Set segments = computeSegmentsToQuery(timeline); - @Nullable - final byte[] queryCacheKey = computeQueryCacheKey(); + @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { - @Nullable - final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); - @Nullable - final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } From d00bb28d3b273e52b009345071a7f1b35a0168ae Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Wed, 13 Dec 2017 11:24:44 -0600 Subject: [PATCH 14/25] Fix indents --- server/src/main/java/io/druid/client/CachingClusteredClient.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index b6474a9d5b47..ab7cae42dd02 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -266,7 +266,6 @@ Sequence run(final UnaryOperator> time final Set segments = computeSegmentsToQuery(timeline); @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); - if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); @Nullable final String currentEtag = computeCurrentEtag(segments, queryCacheKey); From 53e80564a5005415a9cec67f5ec3bdee600b086a Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Thu, 14 Dec 2017 16:26:38 -0600 Subject: [PATCH 15/25] Add flag for result caching --- .../java/io/druid/query/CacheStrategy.java | 29 +----- .../groupby/GroupByQueryQueryToolChest.java | 93 +++---------------- .../SegmentMetadataQueryQueryToolChest.java | 4 +- .../search/SearchQueryQueryToolChest.java | 4 +- .../select/SelectQueryQueryToolChest.java | 4 +- .../TimeBoundaryQueryQueryToolChest.java | 4 +- .../TimeseriesQueryQueryToolChest.java | 73 +++------------ .../query/topn/TopNQueryQueryToolChest.java | 88 +++--------------- ...egmentMetadataQueryQueryToolChestTest.java | 4 +- .../search/SearchQueryQueryToolChestTest.java | 4 +- .../TimeBoundaryQueryQueryToolChestTest.java | 4 +- .../TimeseriesQueryQueryToolChestTest.java | 8 +- .../topn/TopNQueryQueryToolChestTest.java | 8 +- .../druid/client/CachingClusteredClient.java | 4 +- .../io/druid/client/CachingQueryRunner.java | 4 +- .../query/ResultLevelCachingQueryRunner.java | 6 +- .../druid/client/CachingQueryRunnerTest.java | 4 +- 17 files changed, 71 insertions(+), 274 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index 67a3b986c7a0..fe3d73357dc8 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -61,37 +61,16 @@ public interface CacheStrategy> * * The resulting function must be thread-safe. * + * @param isResultLevelCache indicates whether the function is invoked for result-level caching or segment-level caching * @return a thread-safe function that converts the QueryType's result type into something cacheable */ - Function prepareForCache(); + Function prepareForCache(boolean isResultLevelCache); /** * A function that does the inverse of the operation that the function prepareForCache returns * + * @param isResultLevelCache indicates whether the function is invoked for result-level caching or segment-level caching * @return A function that does the inverse of the operation that the function prepareForCache returns */ - Function pullFromCache(); - - /** - * Returns a function that converts the query result set including the post-aggregation results - * into something cacheable. - * - * The resulting function must be thread-safe. - * - * @return a thread-safe function that converts the QueryType's result type into something cacheable - */ - default Function prepareForResultLevelCache() - { - return prepareForCache(); - } - - /** - * A function that does the inverse of the operation that the function prepareForResultLevelCache returns - * - * @return A function that does the inverse of the operation that the function prepareForResultLevelCache returns - */ - default Function pullFromResultLevelCache() - { - return pullFromCache(); - } + Function pullFromCache(boolean isResultLevelCache); } diff --git a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java index f6e05c9c2fa8..1252caa40ddd 100644 --- a/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -409,7 +409,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function prepareForCache() + public Function prepareForCache(boolean isResultLevelCache) { return new Function() { @@ -427,80 +427,10 @@ public Object apply(Row input) for (AggregatorFactory agg : aggs) { retVal.add(event.get(agg.getName())); } - return retVal; - } - - throw new ISE("Don't know how to cache input rows of type[%s]", input.getClass()); - } - }; - } - - @Override - public Function pullFromCache() - { - return new Function() - { - private final Granularity granularity = query.getGranularity(); - - @Override - public Row apply(Object input) - { - Iterator results = ((List) input).iterator(); - - DateTime timestamp = granularity.toDateTime(((Number) results.next()).longValue()); - - Map event = Maps.newLinkedHashMap(); - Iterator dimsIter = dims.iterator(); - while (dimsIter.hasNext() && results.hasNext()) { - final DimensionSpec factory = dimsIter.next(); - event.put(factory.getOutputName(), results.next()); - } - - Iterator aggsIter = aggs.iterator(); - while (aggsIter.hasNext() && results.hasNext()) { - final AggregatorFactory factory = aggsIter.next(); - event.put(factory.getName(), factory.deserialize(results.next())); - } - - if (dimsIter.hasNext() || aggsIter.hasNext() || results.hasNext()) { - throw new ISE( - "Found left over objects while reading from cache!! dimsIter[%s] aggsIter[%s] results[%s]", - dimsIter.hasNext(), - aggsIter.hasNext(), - results.hasNext() - ); - } - - return new MapBasedRow( - timestamp, - event - ); - } - }; - } - - @Override - public Function prepareForResultLevelCache() - { - return new Function() - { - @Override - public Object apply(Row input) - { - if (input instanceof MapBasedRow) { - final MapBasedRow row = (MapBasedRow) input; - final List retVal = Lists.newArrayListWithCapacity(1 + dims.size() + aggs.size()); - retVal.add(row.getTimestamp().getMillis()); - Map event = row.getEvent(); - for (DimensionSpec dim : dims) { - retVal.add(event.get(dim.getOutputName())); - } - for (AggregatorFactory agg : aggs) { - retVal.add(event.get(agg.getName())); - } - // Add postaggregated data to the result - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - retVal.add(event.get(postAgg.getName())); + if (isResultLevelCache) { + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + retVal.add(event.get(postAgg.getName())); + } } return retVal; } @@ -511,7 +441,7 @@ public Object apply(Row input) } @Override - public Function pullFromResultLevelCache() + public Function pullFromCache(boolean isResultLevelCache) { return new Function() { @@ -536,12 +466,12 @@ public Row apply(Object input) final AggregatorFactory factory = aggsIter.next(); event.put(factory.getName(), factory.deserialize(results.next())); } - - Iterator postItr = query.getPostAggregatorSpecs().iterator(); - while (postItr.hasNext() && results.hasNext()) { - event.put(postItr.next().getName(), results.next()); + if (isResultLevelCache) { + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + while (postItr.hasNext() && results.hasNext()) { + event.put(postItr.next().getName(), results.next()); + } } - if (dimsIter.hasNext() || aggsIter.hasNext() || results.hasNext()) { throw new ISE( "Found left over objects while reading from cache!! dimsIter[%s] aggsIter[%s] results[%s]", @@ -558,7 +488,6 @@ public Row apply(Object input) } }; } - }; } diff --git a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 4a921480f30f..5025d9dbc975 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -198,7 +198,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function prepareForCache() + public Function prepareForCache(boolean isResultLevelCache) { return new Function() { @@ -211,7 +211,7 @@ public SegmentAnalysis apply(@Nullable SegmentAnalysis input) } @Override - public Function pullFromCache() + public Function pullFromCache(boolean isResultLevelCache) { return new Function() { diff --git a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java index 74148d66aea4..9efad496b351 100644 --- a/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/search/SearchQueryQueryToolChest.java @@ -206,7 +206,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function, Object> prepareForCache() + public Function, Object> prepareForCache(boolean isResultLevelCache) { return new Function, Object>() { @@ -221,7 +221,7 @@ public Object apply(Result input) } @Override - public Function> pullFromCache() + public Function> pullFromCache(boolean isResultLevelCache) { return new Function>() { diff --git a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java index 0b567ca6fc79..023e532940da 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -243,7 +243,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function, Object> prepareForCache() + public Function, Object> prepareForCache(boolean isResultLevelCache) { return new Function, Object>() { @@ -272,7 +272,7 @@ public Object apply(final Result input) } @Override - public Function> pullFromCache() + public Function> pullFromCache(boolean isResultLevelCache) { return new Function>() { diff --git a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java index a36461d22346..ae286458e49b 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -173,7 +173,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function, Object> prepareForCache() + public Function, Object> prepareForCache(boolean isResultLevelCache) { return new Function, Object>() { @@ -186,7 +186,7 @@ public Object apply(Result input) } @Override - public Function> pullFromCache() + public Function> pullFromCache(boolean isResultLevelCache) { return new Function>() { diff --git a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 1af4e5f4e582..6e9f0307f70d 100644 --- a/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -174,7 +174,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function, Object> prepareForCache() + public Function, Object> prepareForCache(boolean isResultLevelCache) { return new Function, Object>() { @@ -188,69 +188,18 @@ public Object apply(final Result input) for (AggregatorFactory agg : aggs) { retVal.add(results.getMetric(agg.getName())); } - - return retVal; - } - }; - } - - @Override - public Function> pullFromCache() - { - return new Function>() - { - private final Granularity granularity = query.getGranularity(); - - @Override - public Result apply(@Nullable Object input) - { - List results = (List) input; - Map retVal = Maps.newLinkedHashMap(); - - Iterator aggsIter = aggs.iterator(); - Iterator resultIter = results.iterator(); - - DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); - - while (aggsIter.hasNext() && resultIter.hasNext()) { - final AggregatorFactory factory = aggsIter.next(); - retVal.put(factory.getName(), factory.deserialize(resultIter.next())); + if (isResultLevelCache) { + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + retVal.add(results.getMetric(postAgg.getName())); + } } - - return new Result( - timestamp, - new TimeseriesResultValue(retVal) - ); - } - }; - } - - @Override - public Function, Object> prepareForResultLevelCache() - { - return new Function, Object>() - { - @Override - public Object apply(final Result input) - { - TimeseriesResultValue results = input.getValue(); - final List retVal = Lists.newArrayListWithCapacity(1 + aggs.size()); - - retVal.add(input.getTimestamp().getMillis()); - for (AggregatorFactory agg : aggs) { - retVal.add(results.getMetric(agg.getName())); - } - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - retVal.add(results.getMetric(postAgg.getName())); - } - return retVal; } }; } @Override - public Function> pullFromResultLevelCache() + public Function> pullFromCache(boolean isResultLevelCache) { return new Function>() { @@ -264,7 +213,6 @@ public Result apply(@Nullable Object input) Iterator aggsIter = aggs.iterator(); Iterator resultIter = results.iterator(); - Iterator postItr = query.getPostAggregatorSpecs().iterator(); DateTime timestamp = granularity.toDateTime(((Number) resultIter.next()).longValue()); @@ -272,10 +220,13 @@ public Result apply(@Nullable Object input) final AggregatorFactory factory = aggsIter.next(); retVal.put(factory.getName(), factory.deserialize(resultIter.next())); } - - while (postItr.hasNext() && resultIter.hasNext()) { - retVal.put(postItr.next().getName(), resultIter.next()); + if (isResultLevelCache) { + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + while (postItr.hasNext() && resultIter.hasNext()) { + retVal.put(postItr.next().getName(), resultIter.next()); + } } + return new Result( timestamp, new TimeseriesResultValue(retVal) diff --git a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java index ea21025a2093..c34e43eb1a46 100644 --- a/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/topn/TopNQueryQueryToolChest.java @@ -340,7 +340,7 @@ public TypeReference getCacheObjectClazz() } @Override - public Function, Object> prepareForCache() + public Function, Object> prepareForCache(boolean isResultLevelCache) { return new Function, Object>() { @@ -360,6 +360,11 @@ public Object apply(final Result input) for (String aggName : aggFactoryNames) { vals.add(result.getMetric(aggName)); } + if (isResultLevelCache) { + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + vals.add(result.getMetric(postAgg.getName())); + } + } retVal.add(vals); } return retVal; @@ -368,7 +373,7 @@ public Object apply(final Result input) } @Override - public Function> pullFromCache() + public Function> pullFromCache(boolean isResultLevelCache) { return new Function>() { @@ -400,80 +405,11 @@ public Result apply(Object input) for (PostAggregator postAgg : postAggs) { vals.put(postAgg.getName(), postAgg.compute(vals)); } - - retVal.add(vals); - } - - return new Result<>(timestamp, new TopNResultValue(retVal)); - } - }; - } - - @Override - public Function, Object> prepareForResultLevelCache() - { - return new Function, Object>() - { - private final String[] aggFactoryNames = extractFactoryName(query.getAggregatorSpecs()); - - @Override - public Object apply(final Result input) - { - List results = Lists.newArrayList(input.getValue()); - final List retVal = Lists.newArrayListWithCapacity(results.size() + 1); - - // make sure to preserve timezone information when caching results - retVal.add(input.getTimestamp().getMillis()); - for (DimensionAndMetricValueExtractor result : results) { - List vals = Lists.newArrayListWithCapacity(aggFactoryNames.length + 2); - vals.add(result.getDimensionValue(query.getDimensionSpec().getOutputName())); - for (String aggName : aggFactoryNames) { - vals.add(result.getMetric(aggName)); - } - // Add post aggregated results - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - vals.add(result.getMetric(postAgg.getName())); - } - retVal.add(vals); - } - return retVal; - } - }; - } - - @Override - public Function> pullFromResultLevelCache() - { - return new Function>() - { - private final Granularity granularity = query.getGranularity(); - - @Override - public Result apply(Object input) - { - List results = (List) input; - List> retVal = Lists.newArrayListWithCapacity(results.size()); - - Iterator inputIter = results.iterator(); - DateTime timestamp = granularity.toDateTime(((Number) inputIter.next()).longValue()); - - while (inputIter.hasNext()) { - List result = (List) inputIter.next(); - Map vals = Maps.newLinkedHashMap(); - - Iterator aggIter = aggs.iterator(); - Iterator resultIter = result.iterator(); - Iterator postItr = query.getPostAggregatorSpecs().iterator(); - - vals.put(query.getDimensionSpec().getOutputName(), resultIter.next()); - - while (aggIter.hasNext() && resultIter.hasNext()) { - final AggregatorFactory factory = aggIter.next(); - vals.put(factory.getName(), factory.deserialize(resultIter.next())); - } - - while (postItr.hasNext() && resultIter.hasNext()) { - vals.put(postItr.next().getName(), resultIter.next()); + if (isResultLevelCache) { + Iterator postItr = query.getPostAggregatorSpecs().iterator(); + while (postItr.hasNext() && resultIter.hasNext()) { + vals.put(postItr.next().getName(), resultIter.next()); + } } retVal.add(vals); } diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index 6a0e3c42a42b..538aaa6581f4 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -89,7 +89,7 @@ public void testCacheStrategy() throws Exception null ); - Object preparedValue = strategy.prepareForCache().apply(result); + Object preparedValue = strategy.prepareForCache(false).apply(result); ObjectMapper objectMapper = new DefaultObjectMapper(); SegmentAnalysis fromCacheValue = objectMapper.readValue( @@ -97,7 +97,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - SegmentAnalysis fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); + SegmentAnalysis fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java index 621a435f5e95..0f6d7ec90896 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java @@ -59,7 +59,7 @@ public void testCacheStrategy() throws Exception new SearchResultValue(ImmutableList.of(new SearchHit("dim1", "a"))) ); - Object preparedValue = strategy.prepareForCache().apply( + Object preparedValue = strategy.prepareForCache(false).apply( result ); @@ -69,7 +69,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java index 3acf7e759ab9..5f2c7c7d0097 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java @@ -215,7 +215,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache().apply( + Object preparedValue = strategy.prepareForCache(false).apply( result ); @@ -225,7 +225,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index fb60e7dc19c6..44e6b7c698b6 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -92,7 +92,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache().apply(result1); + Object preparedValue = strategy.prepareForCache(false).apply(result1); ObjectMapper objectMapper = TestHelper.makeJsonMapper(); Object fromCacheValue = objectMapper.readValue( @@ -100,7 +100,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); Assert.assertEquals(result1, fromCacheResult); @@ -112,13 +112,13 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedResultLevelCacheValue = strategy.prepareForResultLevelCache().apply(result2); + Object preparedResultLevelCacheValue = strategy.prepareForCache(true).apply(result2); Object fromResultLevelCacheValue = objectMapper.readValue( objectMapper.writeValueAsBytes(preparedResultLevelCacheValue), strategy.getCacheObjectClazz() ); - Result fromResultLevelCacheRes = strategy.pullFromResultLevelCache().apply(fromResultLevelCacheValue); + Result fromResultLevelCacheRes = strategy.pullFromCache(true).apply(fromResultLevelCacheValue); Assert.assertEquals(result2, fromResultLevelCacheRes); } diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index faaa0211bdd6..603c21057046 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -91,7 +91,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache().apply( + Object preparedValue = strategy.prepareForCache(false).apply( result1 ); @@ -101,7 +101,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache().apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); Assert.assertEquals(result1, fromCacheResult); @@ -119,7 +119,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedResultCacheValue = strategy.prepareForResultLevelCache().apply( + Object preparedResultCacheValue = strategy.prepareForCache(true).apply( result2 ); @@ -128,7 +128,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromResultCacheResult = strategy.pullFromResultLevelCache().apply(fromResultCacheValue); + Result fromResultCacheResult = strategy.pullFromCache(true).apply(fromResultCacheValue); Assert.assertEquals(result2, fromResultCacheResult); } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index ab7cae42dd02..1a4eb1e3eaf8 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -497,7 +497,7 @@ private void addSequencesFromCache( return; } - final Function pullFromCacheFunction = strategy.pullFromCache(); + final Function pullFromCacheFunction = strategy.pullFromCache(false); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); for (Pair cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; @@ -597,7 +597,7 @@ private Sequence getAndCacheServerResults( .withQuerySegmentSpec(segmentsOfServerSpec), responseContext ); - final Function cacheFn = strategy.prepareForCache(); + final Function cacheFn = strategy.prepareForCache(false); return resultsBySegments .map(result -> { final BySegmentResultValueClass resultsOfSegment = result.getValue(); diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 17df72a59967..62bfa89dbbb9 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -103,7 +103,7 @@ public Sequence run(QueryPlus queryPlus, Map responseConte } if (useCache) { - final Function cacheFn = strategy.pullFromCache(); + final Function cacheFn = strategy.pullFromCache(false); final byte[] cachedResult = cache.get(key); if (cachedResult != null) { final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); @@ -143,7 +143,7 @@ public void cleanup(Iterator iterFromMake) final Collection> cacheFutures = Collections.synchronizedList(Lists.>newLinkedList()); if (populateCache) { - final Function cacheFn = strategy.prepareForCache(); + final Function cacheFn = strategy.prepareForCache(false); return Sequences.withEffect( Sequences.map( diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 1c1df66c0e1d..c2c34563fb95 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -98,6 +98,8 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) == null) { query = query.withOverriddenContext( ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); + query = query.withOverriddenContext(ImmutableMap.of("ENABLE_RESULT_CACHE", true)); + } Sequence resultFromClient = baseRunner.run( QueryPlus.wrap(query), @@ -151,7 +153,7 @@ private T cacheResultEntry( T resultEntry ) { - final Function cacheFn = strategy.prepareForResultLevelCache(); + final Function cacheFn = strategy.prepareForCache(true); if (resultLevelCachePopulator != null) { resultLevelCachePopulator.cacheFutures .add(cachingExec.submit(() -> cacheFn.apply(resultEntry))); @@ -188,7 +190,7 @@ private Sequence deserializeResults( if (cachedResult == null) { log.error("Cached result set is null"); } - final Function pullFromCacheFunction = strategy.pullFromResultLevelCache(); + final Function pullFromCacheFunction = strategy.pullFromCache(true); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); //Skip the resultsetID and its length bytes Sequence cachedSequence = Sequences.simple(() -> { diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index 0ba202eb0091..a828d8c9292d 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -316,7 +316,7 @@ public boolean isUseCache() byte[] cacheValue = cache.get(cacheKey); Assert.assertNotNull(cacheValue); - Function fn = cacheStrategy.pullFromCache(); + Function fn = cacheStrategy.pullFromCache(false); List cacheResults = Lists.newArrayList( Iterators.transform( objectMapper.readValues( @@ -351,7 +351,7 @@ private void testUseCache( cache, objectMapper, cacheKey, - Iterables.transform(expectedResults, cacheStrategy.prepareForCache()) + Iterables.transform(expectedResults, cacheStrategy.prepareForCache(false)) ); CachingQueryRunner runner = new CachingQueryRunner( From d4b823da9a5a13f9a088543cf1305c24d6849926 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Thu, 14 Dec 2017 20:40:41 -0600 Subject: [PATCH 16/25] Remove logs --- .../main/java/io/druid/query/ResultLevelCachingQueryRunner.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index c2c34563fb95..9d1514fc6c5e 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -98,8 +98,6 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) == null) { query = query.withOverriddenContext( ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); - query = query.withOverriddenContext(ImmutableMap.of("ENABLE_RESULT_CACHE", true)); - } Sequence resultFromClient = baseRunner.run( QueryPlus.wrap(query), From cb107f9fe74a999fec41295e62b10cec17068419 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 19 Dec 2017 13:18:26 -0600 Subject: [PATCH 17/25] Make cache object generation synchronous --- .../query/ResultLevelCachingQueryRunner.java | 61 ++++++++----------- .../server/ClientQuerySegmentWalker.java | 7 --- 2 files changed, 26 insertions(+), 42 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 9d1514fc6c5e..417b4e24f8c3 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -23,11 +23,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import io.druid.client.ResultLevelCacheUtil; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; @@ -44,17 +41,15 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; public class ResultLevelCachingQueryRunner implements QueryRunner { private static final Logger log = new Logger(ResultLevelCachingQueryRunner.class); private final QueryRunner baseRunner; - private final ListeningExecutorService cachingExec; private ObjectMapper objectMapper; private final Cache cache; private final CacheConfig cacheConfig; @@ -69,13 +64,11 @@ public ResultLevelCachingQueryRunner( QueryToolChest queryToolChest, Query query, ObjectMapper objectMapper, - ExecutorService cachingExec, Cache cache, CacheConfig cacheConfig ) { this.baseRunner = baseRunner; - this.cachingExec = MoreExecutors.listeningDecorator(cachingExec); this.objectMapper = objectMapper; this.cache = cache; this.cacheConfig = cacheConfig; @@ -93,6 +86,7 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) final String cacheKeyStr = StringUtils.fromUtf8(strategy.computeCacheKey(query)); final byte[] cachedResultSet = fetchResultsFromResultLevelCache(cacheKeyStr); String existingResultSetId = extractEtagFromResults(cachedResultSet); + existingResultSetId = existingResultSetId == null ? "" : existingResultSetId; if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) == null) { @@ -114,6 +108,10 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) cacheKeyStr, newResultSetId ); + + if (resultLevelCachePopulator == null) { + return resultFromClient; + } return Sequences.wrap(Sequences.map( resultFromClient, new Function() @@ -130,7 +128,10 @@ public T apply(T input) @Override public void after(boolean isDone, Throwable thrown) throws Exception { - if (resultLevelCachePopulator != null) { + Preconditions.checkNotNull(resultLevelCachePopulator, "ResultLevelCachePopulator cannot be null during cache population"); + if (thrown != null) { + log.error("Error while preparing for result level caching for query %s ", query.getId()); + } else { // The resultset identifier and its length is cached along with the resultset resultLevelCachePopulator.populateResults(newResultSetId); log.debug("Cache population complete for query %s", query.getId()); @@ -152,10 +153,8 @@ private T cacheResultEntry( ) { final Function cacheFn = strategy.prepareForCache(true); - if (resultLevelCachePopulator != null) { - resultLevelCachePopulator.cacheFutures - .add(cachingExec.submit(() -> cacheFn.apply(resultEntry))); - } + resultLevelCachePopulator.cacheObjects + .add(cacheFn.apply(resultEntry)); return resultEntry; } @@ -193,16 +192,13 @@ private Sequence deserializeResults( //Skip the resultsetID and its length bytes Sequence cachedSequence = Sequences.simple(() -> { try { - if (cachedResult.length == 0) { - return Iterators.emptyIterator(); - } - + int resultOffset = Integer.BYTES + resultSetId.length(); return objectMapper.readValues( - objectMapper.getFactory().createParser(Arrays.copyOfRange( + objectMapper.getFactory().createParser( cachedResult, - Integer.BYTES + resultSetId.length(), - cachedResult.length - )), + resultOffset, + cachedResult.length - resultOffset + ), cacheObjectClazz ); } @@ -219,14 +215,12 @@ private ResultLevelCachePopulator createResultLevelCachePopulator( String resultSetId ) { - // Results need to be cached only if all the segments are historical segments if (resultSetId != null && populateResultCache) { return new ResultLevelCachePopulator( cache, objectMapper, ResultLevelCacheUtil.computeResultLevelCacheKey(cacheKeyStr), - cacheConfig, - cachingExec + cacheConfig ); } else { return null; @@ -238,23 +232,20 @@ public class ResultLevelCachePopulator private final Cache cache; private final ObjectMapper mapper; private final Cache.NamedKey key; - private final ConcurrentLinkedQueue> cacheFutures = new ConcurrentLinkedQueue<>(); + private final List cacheObjects = new ArrayList<>(); private final CacheConfig cacheConfig; - private final ListeningExecutorService backgroundExecutorService; private ResultLevelCachePopulator( Cache cache, ObjectMapper mapper, Cache.NamedKey key, - CacheConfig cacheConfig, - ListeningExecutorService backgroundExecutorService + CacheConfig cacheConfig ) { this.cache = cache; this.mapper = mapper; this.key = key; this.cacheConfig = cacheConfig; - this.backgroundExecutorService = backgroundExecutorService; } public void populateResults(String resultSetIdentifier) @@ -273,7 +264,7 @@ public void populateResults(String resultSetIdentifier) ); } // Help out GC by making sure all references are gone - cacheFutures.clear(); + cacheObjects.clear(); } catch (IOException ioe) { log.error("Failed to write cached values for query %s", query.getId()); @@ -282,14 +273,14 @@ public void populateResults(String resultSetIdentifier) private byte[] fetchResultBytes(ByteArrayOutputStream resultStream, int cacheLimit) { - for (ListenableFuture lsf : cacheFutures) { + for (Object cacheObj : cacheObjects) { try (JsonGenerator gen = mapper.getFactory().createGenerator(resultStream)) { - gen.writeObject(lsf.get()); + gen.writeObject(cacheObj); if (cacheLimit > 0 && resultStream.size() > cacheLimit) { return null; } } - catch (ExecutionException | InterruptedException | IOException ex) { + catch (IOException ex) { log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); return null; } diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 256c3ab9710a..02df194faf34 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -26,7 +26,6 @@ import io.druid.client.CachingClusteredClient; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; -import io.druid.guice.annotations.BackgroundCaching; import io.druid.query.FluentQueryRunnerBuilder; import io.druid.query.PostProcessingOperator; import io.druid.query.Query; @@ -41,8 +40,6 @@ import io.druid.server.initialization.ServerConfig; import org.joda.time.Interval; -import java.util.concurrent.ExecutorService; - /** */ public class ClientQuerySegmentWalker implements QuerySegmentWalker @@ -53,7 +50,6 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final RetryQueryRunnerConfig retryConfig; private final ObjectMapper objectMapper; private final ServerConfig serverConfig; - private final ExecutorService cachingExec; private final Cache cache; private final CacheConfig cacheConfig; @@ -66,7 +62,6 @@ public ClientQuerySegmentWalker( RetryQueryRunnerConfig retryConfig, ObjectMapper objectMapper, ServerConfig serverConfig, - @BackgroundCaching ExecutorService cachingExec, Cache cache, CacheConfig cacheConfig ) @@ -77,7 +72,6 @@ public ClientQuerySegmentWalker( this.retryConfig = retryConfig; this.objectMapper = objectMapper; this.serverConfig = serverConfig; - this.cachingExec = cachingExec; this.cache = cache; this.cacheConfig = cacheConfig; } @@ -124,7 +118,6 @@ private QueryRunner makeRunner(Query query, QueryRunner baseClientR toolChest, query, objectMapper, - cachingExec, cache, cacheConfig ); From 34d01284274881f2441b5845a8b667d260f356fd Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 19 Dec 2017 16:22:35 -0600 Subject: [PATCH 18/25] Avoid saving intermediate cache results to list --- .../query/ResultLevelCachingQueryRunner.java | 120 ++++++++++-------- 1 file changed, 65 insertions(+), 55 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 417b4e24f8c3..0f9d4de6d217 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -41,9 +41,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; -import java.util.List; import java.util.Map; public class ResultLevelCachingQueryRunner implements QueryRunner @@ -108,10 +106,11 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) cacheKeyStr, newResultSetId ); - if (resultLevelCachePopulator == null) { return resultFromClient; } + final Function cacheFn = strategy.prepareForCache(true); + return Sequences.wrap(Sequences.map( resultFromClient, new Function() @@ -119,7 +118,9 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) @Override public T apply(T input) { - cacheResultEntry(resultLevelCachePopulator, input); + if (resultLevelCachePopulator.isShouldPopulate()) { + resultLevelCachePopulator.cacheResultEntry(resultLevelCachePopulator, input, cacheFn); + } return input; } } @@ -128,14 +129,22 @@ public T apply(T input) @Override public void after(boolean isDone, Throwable thrown) throws Exception { - Preconditions.checkNotNull(resultLevelCachePopulator, "ResultLevelCachePopulator cannot be null during cache population"); + Preconditions.checkNotNull( + resultLevelCachePopulator, + "ResultLevelCachePopulator cannot be null during cache population" + ); if (thrown != null) { - log.error("Error while preparing for result level caching for query %s ", query.getId()); - } else { + log.error( + "Error while preparing for result level caching for query %s with error %s ", + query.getId(), + thrown.getMessage() + ); + } else if (resultLevelCachePopulator.isShouldPopulate()) { // The resultset identifier and its length is cached along with the resultset - resultLevelCachePopulator.populateResults(newResultSetId); + resultLevelCachePopulator.populateResults(); log.debug("Cache population complete for query %s", query.getId()); } + resultLevelCachePopulator.cacheObjectStream.close(); } }); } @@ -147,17 +156,6 @@ public void after(boolean isDone, Throwable thrown) throws Exception } } - private T cacheResultEntry( - ResultLevelCachePopulator resultLevelCachePopulator, - T resultEntry - ) - { - final Function cacheFn = strategy.prepareForCache(true); - resultLevelCachePopulator.cacheObjects - .add(cacheFn.apply(resultEntry)); - return resultEntry; - } - private byte[] fetchResultsFromResultLevelCache( final String queryCacheKey ) @@ -216,12 +214,25 @@ private ResultLevelCachePopulator createResultLevelCachePopulator( ) { if (resultSetId != null && populateResultCache) { - return new ResultLevelCachePopulator( + ResultLevelCachePopulator resultLevelCachePopulator = new ResultLevelCachePopulator( cache, objectMapper, ResultLevelCacheUtil.computeResultLevelCacheKey(cacheKeyStr), - cacheConfig + cacheConfig, + true ); + try { + // Save the resultSetId and its length + resultLevelCachePopulator.cacheObjectStream.write(ByteBuffer.allocate(Integer.BYTES) + .putInt(resultSetId.length()) + .array()); + resultLevelCachePopulator.cacheObjectStream.write(StringUtils.toUtf8(resultSetId)); + } + catch (IOException ioe) { + log.error("Failed to write cached values for query %s", query.getId()); + return null; + } + return resultLevelCachePopulator; } else { return null; } @@ -232,60 +243,59 @@ public class ResultLevelCachePopulator private final Cache cache; private final ObjectMapper mapper; private final Cache.NamedKey key; - private final List cacheObjects = new ArrayList<>(); private final CacheConfig cacheConfig; + private final ByteArrayOutputStream cacheObjectStream = new ByteArrayOutputStream(); + + public boolean isShouldPopulate() + { + return shouldPopulate; + } + + private boolean shouldPopulate; private ResultLevelCachePopulator( Cache cache, ObjectMapper mapper, Cache.NamedKey key, - CacheConfig cacheConfig + CacheConfig cacheConfig, + boolean shouldPopulate ) { this.cache = cache; this.mapper = mapper; this.key = key; this.cacheConfig = cacheConfig; + this.shouldPopulate = shouldPopulate; } - public void populateResults(String resultSetIdentifier) + private void cacheResultEntry( + ResultLevelCachePopulator resultLevelCachePopulator, + T resultEntry, + Function cacheFn + ) { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try { - // Save the resultSetId and its length - bytes.write(ByteBuffer.allocate(Integer.BYTES).putInt(resultSetIdentifier.length()).array()); - bytes.write(StringUtils.toUtf8(resultSetIdentifier)); - byte[] resultBytes = fetchResultBytes(bytes, cacheConfig.getCacheBulkMergeLimit()); - if (resultBytes != null) { - ResultLevelCacheUtil.populate( - cache, - key, - resultBytes - ); - } - // Help out GC by making sure all references are gone - cacheObjects.clear(); + + int cacheLimit = cacheConfig.getResultLevelCacheLimit(); + if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { + shouldPopulate = false; + return; } - catch (IOException ioe) { - log.error("Failed to write cached values for query %s", query.getId()); + try (JsonGenerator gen = mapper.getFactory().createGenerator(resultLevelCachePopulator.cacheObjectStream)) { + gen.writeObject(cacheFn.apply(resultEntry)); + } + catch (IOException ex) { + log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); + shouldPopulate = false; } } - private byte[] fetchResultBytes(ByteArrayOutputStream resultStream, int cacheLimit) + public void populateResults() { - for (Object cacheObj : cacheObjects) { - try (JsonGenerator gen = mapper.getFactory().createGenerator(resultStream)) { - gen.writeObject(cacheObj); - if (cacheLimit > 0 && resultStream.size() > cacheLimit) { - return null; - } - } - catch (IOException ex) { - log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); - return null; - } - } - return resultStream.toByteArray(); + ResultLevelCacheUtil.populate( + cache, + key, + cacheObjectStream.toByteArray() + ); } } } From 46da9e500201edf07dce6c7cd4e7ade7d4474849 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 2 Feb 2018 15:35:17 -0600 Subject: [PATCH 19/25] Fix changes that handle etag based response --- .../query/ResultLevelCachingQueryRunner.java | 22 ++++++++++--------- 1 file changed, 12 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 0f9d4de6d217..9860f5400b12 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -79,25 +79,27 @@ public ResultLevelCachingQueryRunner( @Override public Sequence run(QueryPlus queryPlus, Map responseContext) { - if (useResultCache) { + if (useResultCache || populateResultCache) { final String cacheKeyStr = StringUtils.fromUtf8(strategy.computeCacheKey(query)); final byte[] cachedResultSet = fetchResultsFromResultLevelCache(cacheKeyStr); String existingResultSetId = extractEtagFromResults(cachedResultSet); existingResultSetId = existingResultSetId == null ? "" : existingResultSetId; + String userEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); + query = query.withOverriddenContext( + ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); - if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) == null) { - query = query.withOverriddenContext( - ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); - } Sequence resultFromClient = baseRunner.run( QueryPlus.wrap(query), responseContext ); String newResultSetId = (String) responseContext.get(QueryResource.HEADER_ETAG); - if (newResultSetId != null && newResultSetId.equals(existingResultSetId)) { + if (newResultSetId.equals(userEtag)) { + return Sequences.empty(); + } + if (useResultCache && newResultSetId != null && newResultSetId.equals(existingResultSetId)) { log.debug("Return cached result set as there is no change in identifiers for query %s ", query.getId()); return deserializeResults(cachedResultSet, strategy, existingResultSetId); } else { @@ -276,12 +278,12 @@ private void cacheResultEntry( { int cacheLimit = cacheConfig.getResultLevelCacheLimit(); - if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { - shouldPopulate = false; - return; - } try (JsonGenerator gen = mapper.getFactory().createGenerator(resultLevelCachePopulator.cacheObjectStream)) { gen.writeObject(cacheFn.apply(resultEntry)); + if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { + shouldPopulate = false; + return; + } } catch (IOException ex) { log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); From bcddabfb6857e99ff4888e0f7be1d2183c5ed14a Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 2 Feb 2018 15:47:38 -0600 Subject: [PATCH 20/25] Release bytestream after use --- .../main/java/io/druid/query/ResultLevelCachingQueryRunner.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 9860f5400b12..980923a8978d 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -282,6 +282,7 @@ private void cacheResultEntry( gen.writeObject(cacheFn.apply(resultEntry)); if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { shouldPopulate = false; + resultLevelCachePopulator.cacheObjectStream.reset(); return; } } From 6c42e2cb23157abec337dd2057dcfeba18a7d138 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 2 Feb 2018 16:42:55 -0600 Subject: [PATCH 21/25] Address PR comments --- .../java/io/druid/query/ResultLevelCachingQueryRunner.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 980923a8978d..5bc760876465 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -86,7 +86,6 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) String existingResultSetId = extractEtagFromResults(cachedResultSet); existingResultSetId = existingResultSetId == null ? "" : existingResultSetId; - String userEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); query = query.withOverriddenContext( ImmutableMap.of(QueryResource.HEADER_IF_NONE_MATCH, existingResultSetId)); @@ -96,9 +95,6 @@ public Sequence run(QueryPlus queryPlus, Map responseContext) ); String newResultSetId = (String) responseContext.get(QueryResource.HEADER_ETAG); - if (newResultSetId.equals(userEtag)) { - return Sequences.empty(); - } if (useResultCache && newResultSetId != null && newResultSetId.equals(existingResultSetId)) { log.debug("Return cached result set as there is no change in identifiers for query %s ", query.getId()); return deserializeResults(cachedResultSet, strategy, existingResultSetId); @@ -282,7 +278,7 @@ private void cacheResultEntry( gen.writeObject(cacheFn.apply(resultEntry)); if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { shouldPopulate = false; - resultLevelCachePopulator.cacheObjectStream.reset(); + resultLevelCachePopulator.cacheObjectStream.close(); return; } } From d51c21e6060b1d17f8cf9b057c028e80bd4c0f43 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 2 Feb 2018 17:17:57 -0600 Subject: [PATCH 22/25] Discard resultcache stream after use --- .../java/io/druid/query/ResultLevelCachingQueryRunner.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index 5bc760876465..f3b4b796716c 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -142,7 +142,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception resultLevelCachePopulator.populateResults(); log.debug("Cache population complete for query %s", query.getId()); } - resultLevelCachePopulator.cacheObjectStream.close(); + resultLevelCachePopulator.cacheObjectStream = null; } }); } @@ -242,7 +242,7 @@ public class ResultLevelCachePopulator private final ObjectMapper mapper; private final Cache.NamedKey key; private final CacheConfig cacheConfig; - private final ByteArrayOutputStream cacheObjectStream = new ByteArrayOutputStream(); + private ByteArrayOutputStream cacheObjectStream = new ByteArrayOutputStream(); public boolean isShouldPopulate() { @@ -278,13 +278,14 @@ private void cacheResultEntry( gen.writeObject(cacheFn.apply(resultEntry)); if (cacheLimit > 0 && resultLevelCachePopulator.cacheObjectStream.size() > cacheLimit) { shouldPopulate = false; - resultLevelCachePopulator.cacheObjectStream.close(); + resultLevelCachePopulator.cacheObjectStream = null; return; } } catch (IOException ex) { log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); shouldPopulate = false; + resultLevelCachePopulator.cacheObjectStream = null; } } From 96bbf23c67fef1de430c9db7bb141606f83bc636 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Fri, 9 Mar 2018 13:24:34 -0600 Subject: [PATCH 23/25] Fix docs --- docs/content/querying/query-context.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index 8568177f5814..d4e2be28f123 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -15,8 +15,8 @@ The query context is used for various query configuration parameters. The follow |queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | |useCache | `true` | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useCache or druid.historical.cache.useCache to determine whether or not to read from the query cache | |populateCache | `true` | Flag indicating whether to save the results of the query to the query cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache or druid.historical.cache.populateCache to determine whether or not to save the results of this query to the query cache | -|useResultLevelCache | `true` | Flag indicating whether to leverage the result level cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useResultLevelCache to determine whether or not to read from the query cache | -|populateResultLevelCache | `true` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache | +|useResultLevelCache | `false` | Flag indicating whether to leverage the result level cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Druid uses druid.broker.cache.useResultLevelCache to determine whether or not to read from the query cache | +|populateResultLevelCache | `false` | Flag indicating whether to save the results of the query to the result level cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses druid.broker.cache.populateCache to determine whether or not to save the results of this query to the query cache | |bySegment | `false` | Return "by segment" results. Primarily used for debugging, setting it to `true` returns results associated with the data segment they came from | |finalize | `true` | Flag indicating whether to "finalize" aggregation results. Primarily used for debugging. For instance, the `hyperUnique` aggregator will return the full HyperLogLog sketch instead of the estimated cardinality when this flag is set to `false` | |chunkPeriod | `P0D` (off) | At the broker node level, long interval queries (of any type) may be broken into shorter interval queries to parallelize merging more than normal. Broken up queries will use a larger share of cluster resources, but may be able to complete faster as a result. Use ISO 8601 periods. For example, if this property is set to `P1M` (one month), then a query covering a year would be broken into 12 smaller queries. The broker uses its query processing executor service to initiate processing for query chunks, so make sure "druid.processing.numThreads" is configured appropriately on the broker. [groupBy queries](groupbyquery.html) do not support chunkPeriod by default, although they do if using the legacy "v1" engine. | From 9dec1cd679ff7a1288091d8dea492a9b1c5a71e8 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Tue, 20 Mar 2018 13:27:55 -0500 Subject: [PATCH 24/25] Address comments --- .../java/io/druid/query/CacheStrategy.java | 19 ++++++- ...egmentMetadataQueryQueryToolChestTest.java | 4 +- .../search/SearchQueryQueryToolChestTest.java | 4 +- .../TimeBoundaryQueryQueryToolChestTest.java | 4 +- .../TimeseriesQueryQueryToolChestTest.java | 4 +- .../topn/TopNQueryQueryToolChestTest.java | 4 +- .../druid/client/CachingClusteredClient.java | 4 +- .../io/druid/client/CachingQueryRunner.java | 4 +- .../query/ResultLevelCachingQueryRunner.java | 7 +-- .../server/ClientQuerySegmentWalker.java | 51 ++++++++++--------- .../druid/client/CachingQueryRunnerTest.java | 4 +- 11 files changed, 65 insertions(+), 44 deletions(-) diff --git a/processing/src/main/java/io/druid/query/CacheStrategy.java b/processing/src/main/java/io/druid/query/CacheStrategy.java index fe3d73357dc8..2c3ec35ed193 100644 --- a/processing/src/main/java/io/druid/query/CacheStrategy.java +++ b/processing/src/main/java/io/druid/query/CacheStrategy.java @@ -26,7 +26,7 @@ import java.util.concurrent.ExecutorService; /** -*/ + */ @ExtensionPoint public interface CacheStrategy> { @@ -37,6 +37,7 @@ public interface CacheStrategy> * @param query the query to be cached * @param willMergeRunners indicates that {@link QueryRunnerFactory#mergeRunners(ExecutorService, Iterable)} will be * called on the cached by-segment results + * * @return true if the query is cacheable, otherwise false. */ boolean isCacheable(QueryType query, boolean willMergeRunners); @@ -45,6 +46,7 @@ public interface CacheStrategy> * Computes the cache key for the given query * * @param query the query to compute a cache key for + * * @return the cache key */ byte[] computeCacheKey(QueryType query); @@ -58,10 +60,11 @@ public interface CacheStrategy> /** * Returns a function that converts from the QueryType's result type to something cacheable. - * + *

* The resulting function must be thread-safe. * * @param isResultLevelCache indicates whether the function is invoked for result-level caching or segment-level caching + * * @return a thread-safe function that converts the QueryType's result type into something cacheable */ Function prepareForCache(boolean isResultLevelCache); @@ -70,7 +73,19 @@ public interface CacheStrategy> * A function that does the inverse of the operation that the function prepareForCache returns * * @param isResultLevelCache indicates whether the function is invoked for result-level caching or segment-level caching + * * @return A function that does the inverse of the operation that the function prepareForCache returns */ Function pullFromCache(boolean isResultLevelCache); + + + default Function prepareForSegmentLevelCache() + { + return prepareForCache(false); + } + + default Function pullFromSegmentLevelCache() + { + return pullFromCache(false); + } } diff --git a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java index 940fde5130be..a497a6313801 100644 --- a/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChestTest.java @@ -94,7 +94,7 @@ public void testCacheStrategy() throws Exception null ); - Object preparedValue = strategy.prepareForCache(false).apply(result); + Object preparedValue = strategy.prepareForSegmentLevelCache().apply(result); ObjectMapper objectMapper = new DefaultObjectMapper(); SegmentAnalysis fromCacheValue = objectMapper.readValue( @@ -102,7 +102,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - SegmentAnalysis fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); + SegmentAnalysis fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java index 0f6d7ec90896..dd73518ae5fe 100644 --- a/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/search/SearchQueryQueryToolChestTest.java @@ -59,7 +59,7 @@ public void testCacheStrategy() throws Exception new SearchResultValue(ImmutableList.of(new SearchHit("dim1", "a"))) ); - Object preparedValue = strategy.prepareForCache(false).apply( + Object preparedValue = strategy.prepareForSegmentLevelCache().apply( result ); @@ -69,7 +69,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java index fd516bd1af9f..d947613dbf62 100644 --- a/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChestTest.java @@ -215,7 +215,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache(false).apply( + Object preparedValue = strategy.prepareForSegmentLevelCache().apply( result ); @@ -225,7 +225,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result, fromCacheResult); } diff --git a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index fbb2f3c3419e..0c4c0e7b6e86 100644 --- a/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -91,7 +91,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache(false).apply(result1); + Object preparedValue = strategy.prepareForSegmentLevelCache().apply(result1); ObjectMapper objectMapper = TestHelper.makeJsonMapper(); Object fromCacheValue = objectMapper.readValue( @@ -99,7 +99,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result1, fromCacheResult); diff --git a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java index 95381b7805b6..1a4e78351f7e 100644 --- a/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java +++ b/processing/src/test/java/io/druid/query/topn/TopNQueryQueryToolChestTest.java @@ -91,7 +91,7 @@ public void testCacheStrategy() throws Exception ) ); - Object preparedValue = strategy.prepareForCache(false).apply( + Object preparedValue = strategy.prepareForSegmentLevelCache().apply( result1 ); @@ -101,7 +101,7 @@ public void testCacheStrategy() throws Exception strategy.getCacheObjectClazz() ); - Result fromCacheResult = strategy.pullFromCache(false).apply(fromCacheValue); + Result fromCacheResult = strategy.pullFromSegmentLevelCache().apply(fromCacheValue); Assert.assertEquals(result1, fromCacheResult); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 6e342a9e41c2..cf5517b17a74 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -497,7 +497,7 @@ private void addSequencesFromCache( return; } - final Function pullFromCacheFunction = strategy.pullFromCache(false); + final Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache(); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); for (Pair cachedResultPair : cachedResults) { final byte[] cachedResult = cachedResultPair.rhs; @@ -597,7 +597,7 @@ private Sequence getAndCacheServerResults( .withQuerySegmentSpec(segmentsOfServerSpec), responseContext ); - final Function cacheFn = strategy.prepareForCache(false); + final Function cacheFn = strategy.prepareForSegmentLevelCache(); return resultsBySegments .map(result -> { final BySegmentResultValueClass resultsOfSegment = result.getValue(); diff --git a/server/src/main/java/io/druid/client/CachingQueryRunner.java b/server/src/main/java/io/druid/client/CachingQueryRunner.java index 62bfa89dbbb9..ad7fe4cd801d 100644 --- a/server/src/main/java/io/druid/client/CachingQueryRunner.java +++ b/server/src/main/java/io/druid/client/CachingQueryRunner.java @@ -103,7 +103,7 @@ public Sequence run(QueryPlus queryPlus, Map responseConte } if (useCache) { - final Function cacheFn = strategy.pullFromCache(false); + final Function cacheFn = strategy.pullFromSegmentLevelCache(); final byte[] cachedResult = cache.get(key); if (cachedResult != null) { final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); @@ -143,7 +143,7 @@ public void cleanup(Iterator iterFromMake) final Collection> cacheFutures = Collections.synchronizedList(Lists.>newLinkedList()); if (populateCache) { - final Function cacheFn = strategy.prepareForCache(false); + final Function cacheFn = strategy.prepareForSegmentLevelCache(); return Sequences.withEffect( Sequences.map( diff --git a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java index f3b4b796716c..39a5a6de781d 100644 --- a/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java +++ b/server/src/main/java/io/druid/query/ResultLevelCachingQueryRunner.java @@ -125,7 +125,7 @@ public T apply(T input) ), new SequenceWrapper() { @Override - public void after(boolean isDone, Throwable thrown) throws Exception + public void after(boolean isDone, Throwable thrown) { Preconditions.checkNotNull( resultLevelCachePopulator, @@ -133,6 +133,7 @@ public void after(boolean isDone, Throwable thrown) throws Exception ); if (thrown != null) { log.error( + thrown, "Error while preparing for result level caching for query %s with error %s ", query.getId(), thrown.getMessage() @@ -227,7 +228,7 @@ private ResultLevelCachePopulator createResultLevelCachePopulator( resultLevelCachePopulator.cacheObjectStream.write(StringUtils.toUtf8(resultSetId)); } catch (IOException ioe) { - log.error("Failed to write cached values for query %s", query.getId()); + log.error(ioe, "Failed to write cached values for query %s", query.getId()); return null; } return resultLevelCachePopulator; @@ -283,7 +284,7 @@ private void cacheResultEntry( } } catch (IOException ex) { - log.error("Failed to retrieve entry to be cached. Result Level caching will not be performed!"); + log.error(ex, "Failed to retrieve entry to be cached. Result Level caching will not be performed!"); shouldPopulate = false; resultLevelCachePopulator.cacheObjectStream = null; } diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 7850e5cd8294..18518acaf908 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -91,6 +91,20 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner makeRunner(Query query, QueryRunner baseClientRunner) { QueryToolChest> toolChest = warehouse.getToolChest(query); + return new ResultLevelCachingQueryRunner<>(makeRunner(query, baseClientRunner, toolChest), + toolChest, + query, + objectMapper, + cache, + cacheConfig); + } + + private QueryRunner makeRunner( + Query query, + QueryRunner baseClientRunner, + QueryToolChest> toolChest + ) + { PostProcessingOperator postProcessing = objectMapper.convertValue( query.getContextValue("postProcessing"), new TypeReference>() @@ -98,30 +112,21 @@ private QueryRunner makeRunner(Query query, QueryRunner baseClientR } ); - return new ResultLevelCachingQueryRunner<>( - new FluentQueryRunnerBuilder<>(toolChest) - .create( - new SetAndVerifyContextQueryRunner( - serverConfig, - new RetryQueryRunner<>( - baseClientRunner, - retryConfig, - objectMapper - ) + return new FluentQueryRunnerBuilder<>(toolChest) + .create( + new SetAndVerifyContextQueryRunner( + serverConfig, + new RetryQueryRunner<>( + baseClientRunner, + retryConfig, + objectMapper ) ) - .applyPreMergeDecoration() - .mergeResults() - .applyPostMergeDecoration() - .emitCPUTimeMetric(emitter) - .postProcess(postProcessing), - toolChest, - query, - objectMapper, - cache, - cacheConfig - ); + ) + .applyPreMergeDecoration() + .mergeResults() + .applyPostMergeDecoration() + .emitCPUTimeMetric(emitter) + .postProcess(postProcessing); } - - } diff --git a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java index dbdc6fb291b1..65ee95acdfae 100644 --- a/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java +++ b/server/src/test/java/io/druid/client/CachingQueryRunnerTest.java @@ -314,7 +314,7 @@ public boolean isUseCache() byte[] cacheValue = cache.get(cacheKey); Assert.assertNotNull(cacheValue); - Function fn = cacheStrategy.pullFromCache(false); + Function fn = cacheStrategy.pullFromSegmentLevelCache(); List cacheResults = Lists.newArrayList( Iterators.transform( objectMapper.readValues( @@ -349,7 +349,7 @@ private void testUseCache( cache, objectMapper, cacheKey, - Iterables.transform(expectedResults, cacheStrategy.prepareForCache(false)) + Iterables.transform(expectedResults, cacheStrategy.prepareForSegmentLevelCache()) ); CachingQueryRunner runner = new CachingQueryRunner( From 7f2d48d90e83b3ebe1e2a8b779fbfcad3ac827f6 Mon Sep 17 00:00:00 2001 From: Atul Mohan Date: Thu, 22 Mar 2018 10:03:03 -0500 Subject: [PATCH 25/25] Add comment about fluent workflow issue --- .../src/main/java/io/druid/server/ClientQuerySegmentWalker.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java index 18518acaf908..e9ee6959ea32 100644 --- a/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/io/druid/server/ClientQuerySegmentWalker.java @@ -91,6 +91,8 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner makeRunner(Query query, QueryRunner baseClientRunner) { QueryToolChest> toolChest = warehouse.getToolChest(query); + + // This does not adhere to the fluent workflow. See https://github.com/druid-io/druid/issues/5517 return new ResultLevelCachingQueryRunner<>(makeRunner(query, baseClientRunner, toolChest), toolChest, query,