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_r213811376
########## File path: server/src/main/java/io/druid/client/CachingClusteredClient.java ########## @@ -471,94 +552,162 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter return cachePopulatorMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } - private SortedMap<DruidServer, List<SegmentDescriptor>> groupSegmentsByServer(Set<ServerToSegment> segments) + /** + * Check the input stream to see what was cached and what was not. For the ones that were cached, merge the results + * and return the merged sequence. For the ones that were NOT cached, get the server result sequence queued up into + * the stream response + * + * @param segmentOrResult A list that is traversed in order to determine what should be sent back. All segments + * should be on the same server. + * + * @return A sequence of either the merged cached results, or the server results from any particular server + */ + private Sequence<T> runOnServer(List<ServerMaybeSegmentMaybeCache<T>> segmentOrResult) { - final SortedMap<DruidServer, List<SegmentDescriptor>> serverSegments = Maps.newTreeMap(); - for (ServerToSegment serverToSegment : segments) { - final QueryableDruidServer queryableDruidServer = serverToSegment.getServer().pick(); - - if (queryableDruidServer == null) { - log.makeAlert( - "No servers found for SegmentDescriptor[%s] for DataSource[%s]?! How can this be?!", - serverToSegment.getSegmentDescriptor(), - query.getDataSource() - ).emit(); - } else { - final DruidServer server = queryableDruidServer.getServer(); - serverSegments.computeIfAbsent(server, s -> new ArrayList<>()).add(serverToSegment.getSegmentDescriptor()); - } + final List<SegmentDescriptor> segmentsOfServer = segmentOrResult.stream( + ).map( + ServerMaybeSegmentMaybeCache::getSegmentDescriptor + ).filter( + Optional::isPresent + ).map( + Optional::get + ).collect( + Collectors.toList() + ); + + // We should only ever have cache or queries to run, not both. So if we have no segments, try caches + if (segmentsOfServer.isEmpty()) { + // Have a special sequence for the cache results so the merge doesn't go all crazy. + // See io.druid.java.util.common.guava.MergeSequenceTest.testScrewsUpOnOutOfOrder for an example + // With zero results actually being found (no segments no caches) this should essentially return a no-op + // merge sequence + return new MergeSequence<>(query.getResultOrdering(), Sequences.simple( + segmentOrResult.stream( + ).map( + ServerMaybeSegmentMaybeCache::getCachedValue + ).filter( + Optional::isPresent + ).map( + Optional::get + ).map( + Collections::singletonList + ).map( + Sequences::simple + ) + )); } - return serverSegments; - } - private void addSequencesFromCache( - final List<Sequence<T>> listOfSequences, - final List<Pair<Interval, byte[]>> cachedResults - ) - { - if (strategy == null) { - return; + final DruidServer server = segmentOrResult.get(0).getServer(); + final QueryRunner serverRunner = serverView.getQueryRunner(server); + + if (serverRunner == null) { + log.error("Server[%s] doesn't have a query runner", server); + return Sequences.empty(); } - final Function<Object, T> pullFromCacheFunction = strategy.pullFromSegmentLevelCache(); - final TypeReference<Object> cacheObjectClazz = strategy.getCacheObjectClazz(); - for (Pair<Interval, byte[]> cachedResultPair : cachedResults) { - final byte[] cachedResult = cachedResultPair.rhs; - Sequence<Object> cachedSequence = new BaseSequence<>( - new BaseSequence.IteratorMaker<Object, Iterator<Object>>() - { - @Override - public Iterator<Object> make() - { - try { - if (cachedResult.length == 0) { - return Collections.emptyIterator(); - } + final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); - return objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } + final Sequence<T> serverResults; + if (isBySegment) { + serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); + } else if (!server.segmentReplicatable() || !populateCache) { + serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec); + } else { + serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); + } + return serverResults; + } - @Override - public void cleanup(Iterator<Object> iterFromMake) - { - } - } + private ServerMaybeSegmentMaybeCache<T> pickServer(SerializablePair<ServerToSegment, Optional<T>> tuple) + { + final Optional<T> maybeResult = tuple.getRhs(); + if (maybeResult.isPresent()) { + return new ServerMaybeSegmentMaybeCache<T>(ALREADY_CACHED_SERVER, Optional.empty(), maybeResult); + } + final ServerToSegment serverToSegment = tuple.getLhs(); + final QueryableDruidServer queryableDruidServer = serverToSegment.getServer().pick(); + if (queryableDruidServer == null) { + log.makeAlert( + "No servers found for SegmentDescriptor[%s] for DataSource[%s]?! How can this be?!", + serverToSegment.getSegmentDescriptor(), + query.getDataSource() + ).emit(); + return new ServerMaybeSegmentMaybeCache<T>( Review comment: fixed or refactored or something, I don't think this is present now ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@druid.apache.org For additional commands, e-mail: commits-h...@druid.apache.org