drcrallen commented on a change in pull request #5913: Move Caching Cluster
Client to java streams and allow parallel intermediate merges
URL: https://github.com/apache/incubator-druid/pull/5913#discussion_r202748487
##########
File path: server/src/main/java/io/druid/client/CachingClusteredClient.java
##########
@@ -249,74 +296,88 @@ public CachingClusteredClient(
contextBuilder.put(CacheConfig.POPULATE_CACHE, false);
contextBuilder.put("bySegment", true);
}
- return contextBuilder.build();
+ return Collections.unmodifiableMap(contextBuilder);
}
- Sequence<T> run(final UnaryOperator<TimelineLookup<String,
ServerSelector>> timelineConverter)
+ Stream<Sequence<T>> run(final UnaryOperator<TimelineLookup<String,
ServerSelector>> timelineConverter)
{
@Nullable
TimelineLookup<String, ServerSelector> timeline =
serverView.getTimeline(query.getDataSource());
if (timeline == null) {
- return Sequences.empty();
+ return Stream.empty();
}
timeline = timelineConverter.apply(timeline);
if (uncoveredIntervalsLimit > 0) {
computeUncoveredIntervals(timeline);
}
- final Set<ServerToSegment> segments = computeSegmentsToQuery(timeline);
+ Stream<ServerToSegment> segments = computeSegmentsToQuery(timeline);
@Nullable
final byte[] queryCacheKey = computeQueryCacheKey();
if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) {
+ // Materialize then re-stream
+ List<ServerToSegment> materializedSegments =
segments.collect(Collectors.toList());
+ segments = materializedSegments.stream();
+
@Nullable
final String prevEtag = (String)
query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH);
@Nullable
- final String currentEtag = computeCurrentEtag(segments, queryCacheKey);
+ final String currentEtag = computeCurrentEtag(materializedSegments,
queryCacheKey);
if (currentEtag != null && currentEtag.equals(prevEtag)) {
- return Sequences.empty();
+ return Stream.empty();
}
}
- final List<Pair<Interval, byte[]>> alreadyCachedResults =
pruneSegmentsWithCachedResults(queryCacheKey, segments);
- final SortedMap<DruidServer, List<SegmentDescriptor>> segmentsByServer =
groupSegmentsByServer(segments);
- return new LazySequence<>(() -> {
- List<Sequence<T>> sequencesByInterval = new
ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size());
- addSequencesFromCache(sequencesByInterval, alreadyCachedResults);
- addSequencesFromServer(sequencesByInterval, segmentsByServer);
- return Sequences
- .simple(sequencesByInterval)
- .flatMerge(seq -> seq, query.getResultOrdering());
- });
+ // This pipeline follows a few general steps:
+ // 1. Fetch cache results - Unfortunately this is an eager operation so
that the non cached items can
+ // be batched per server. Cached results are assigned to a mock server
ALREADY_CACHED_SERVER
+ // 2. Group the segment information by server
+ // 3. Per server (including the ALREADY_CACHED_SERVER) create the
appropriate Sequence results - cached results
+ // are handled in their own merge
+ final Stream<SerializablePair<ServerToSegment, Optional<T>>>
cacheResolvedResults = deserializeFromCache(
+ maybeFetchCacheResults(
+ queryCacheKey,
+ segments
+ )
+ );
+ return groupCachedResultsByServer(
Review comment:
I can do this in some place. In others, specifically with lambdas this kind
of style becomes much harder to read. So just to keep things in the same format
I used this style in all places
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]