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_r222860123
##########
File path: server/src/main/java/io/druid/client/CachingClusteredClient.java
##########
@@ -242,74 +298,90 @@ 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 for computeCurrentEtag, then re-stream
+ final 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<Pair<ServerToSegment, Optional<T>>> cacheResolvedResults =
deserializeFromCache(
+ maybeFetchCacheResults(queryCacheKey, segments)
+ );
+ return groupCachedResultsByServer(cacheResolvedResults)
+ .map(this::runOnServer)
+ // We do a hard materialization here so that the resulting
spliterators have properties that we want
+ // Otherwise the stream's spliterator is of a hash map entry
spliterator from the group-by-server operation
+ // This also causes eager initialization of the **sequences**, aka
forking off the direct druid client requests
+ // Sequence result accumulation should still be lazy
+ .collect(Collectors.toList())
+ .stream();
}
- private Set<ServerToSegment> computeSegmentsToQuery(TimelineLookup<String,
ServerSelector> timeline)
+ /**
+ * Create a stream of the partition chunks which are useful in this query
+ *
+ * @param holder The holder of the shard to server component of the
timeline
+ *
+ * @return Chunks and the segment descriptors corresponding to the chunk
+ */
+ private Stream<ServerToSegment>
extractServerAndSegment(TimelineObjectHolder<String, ServerSelector> holder)
{
- final List<TimelineObjectHolder<String, ServerSelector>> serversLookup =
toolChest.filterSegments(
- query,
- query.getIntervals().stream().flatMap(i ->
timeline.lookup(i).stream()).collect(Collectors.toList())
- );
+ return DimFilterUtils
+ .filterShards(
+ query.getFilter(),
+ holder.getObject(),
+ partitionChunk ->
partitionChunk.getObject().getSegment().getShardSpec(),
+ Maps.newHashMap()
+ )
+ .stream()
+ .map(chunk -> new ServerToSegment(
+ chunk.getObject(),
+ new SegmentDescriptor(holder.getInterval(), holder.getVersion(),
chunk.getChunkNumber())
+ ));
+ }
- final Set<ServerToSegment> segments = Sets.newLinkedHashSet();
- final Map<String, Optional<RangeSet<String>>> dimensionRangeCache =
Maps.newHashMap();
- // Filter unneeded chunks based on partition dimension
- for (TimelineObjectHolder<String, ServerSelector> holder :
serversLookup) {
- final Set<PartitionChunk<ServerSelector>> filteredChunks =
DimFilterUtils.filterShards(
- query.getFilter(),
- holder.getObject(),
- partitionChunk ->
partitionChunk.getObject().getSegment().getShardSpec(),
- dimensionRangeCache
- );
- for (PartitionChunk<ServerSelector> chunk : filteredChunks) {
- ServerSelector server = chunk.getObject();
- final SegmentDescriptor segment = new SegmentDescriptor(
- holder.getInterval(),
- holder.getVersion(),
- chunk.getChunkNumber()
- );
- segments.add(new ServerToSegment(server, segment));
- }
- }
- return segments;
+ private Stream<ServerToSegment>
computeSegmentsToQuery(TimelineLookup<String, ServerSelector> timeline)
+ {
+ return toolChest
+ .filterSegments(
+ query,
+ query.getIntervals().stream().flatMap(i ->
timeline.lookup(i).stream()).collect(Collectors.toList())
+ )
+ .stream()
+ .flatMap(this::extractServerAndSegment)
+ .distinct();
}
private void computeUncoveredIntervals(TimelineLookup<String,
ServerSelector> timeline)
Review comment:
done
----------------------------------------------------------------
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]