From 700d3a1b10bd867b5e6b018b645d153928022d76 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 20 Jun 2018 15:54:21 -0700 Subject: [PATCH 01/58] Move some sequence implementations to Stream in Caching Clustered Client --- .../java/util/common/guava/MergeSequence.java | 11 +- .../java/util/common/guava/Sequences.java | 21 + .../druid/client/CachingClusteredClient.java | 394 ++++++++++++------ .../java/io/druid/client/cache/Cache.java | 15 + .../io/druid/client/cache/CaffeineCache.java | 18 + .../io/druid/client/cache/HybridCache.java | 42 +- .../java/io/druid/client/cache/MapCache.java | 18 + .../io/druid/client/cache/MemcachedCache.java | 99 +++-- 8 files changed, 442 insertions(+), 176 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index 27277820c599..b22d8a9307c4 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -48,16 +48,7 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat { PriorityQueue> pQueue = new PriorityQueue<>( 32, - ordering.onResultOf( - new Function, T>() - { - @Override - public T apply(Yielder input) - { - return input.get(); - } - } - ) + ordering.onResultOf(Yielder::get) ); pQueue = baseSequences.accumulate( diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index f166c8ce4842..4a0c20181248 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -30,6 +30,7 @@ import java.util.Iterator; import java.util.List; import java.util.concurrent.Executor; +import java.util.stream.Stream; /** */ @@ -58,6 +59,26 @@ public void cleanup(Iterator iterFromMake) ); } + public static Sequence simple(final Stream stream) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return stream.iterator(); + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // stream.close() maybe?? + } + } + ); + } + @SuppressWarnings("unchecked") public static Sequence empty() { diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 2be0a819a351..5e998024236b 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -21,14 +21,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.Optional; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; @@ -42,14 +36,13 @@ import io.druid.client.cache.CacheConfig; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; +import io.druid.collections.SerializablePair; import io.druid.guice.annotations.BackgroundCaching; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; -import io.druid.java.util.common.guava.BaseSequence; -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.emitter.EmittingLogger; @@ -67,8 +60,10 @@ import io.druid.query.aggregation.MetricManipulatorFns; import io.druid.query.filter.DimFilterUtils; import io.druid.query.spec.MultipleSpecificSegmentSpec; +import io.druid.server.DruidNode; import io.druid.server.QueryResource; import io.druid.server.coordination.DruidServerMetadata; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import io.druid.timeline.TimelineLookup; import io.druid.timeline.TimelineObjectHolder; @@ -83,21 +78,36 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; -import java.util.SortedMap; +import java.util.TreeMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; +import java.util.function.Function; import java.util.function.UnaryOperator; import java.util.stream.Collectors; +import java.util.stream.Stream; /** */ public class CachingClusteredClient implements QuerySegmentWalker { private static final EmittingLogger log = new EmittingLogger(CachingClusteredClient.class); + private static final DruidServer ALREADY_CACHED_SERVER = new DruidServer( + new DruidNode( + "__internal-client-cache", + "localhost", + -1, + -1, + true, + false + ), + new DruidServerConfig(), + ServerType.HISTORICAL + ); private final QueryToolChestWarehouse warehouse; private final TimelineServerView serverView; private final Cache cache; @@ -237,9 +247,9 @@ private class SpecificQueryRunnable this.downstreamQuery = query.withOverriddenContext(makeDownstreamQueryContext()); } - private ImmutableMap makeDownstreamQueryContext() + private Map makeDownstreamQueryContext() { - final ImmutableMap.Builder contextBuilder = new ImmutableMap.Builder<>(); + final Map contextBuilder = new LinkedHashMap<>(); final int priority = QueryContexts.getPriority(query); contextBuilder.put(QueryContexts.PRIORITY_KEY, priority); @@ -249,7 +259,7 @@ private ImmutableMap makeDownstreamQueryContext() contextBuilder.put(CacheConfig.POPULATE_CACHE, false); contextBuilder.put("bySegment", true); } - return contextBuilder.build(); + return Collections.unmodifiableMap(contextBuilder); } Sequence run(final UnaryOperator> timelineConverter) @@ -277,16 +287,101 @@ Sequence run(final UnaryOperator> time } } - final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); - final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - 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()); - }); + final Stream> resultStream = deserializeFromCache( + maybeFetchCacheResults( + queryCacheKey, + segments.stream() + ) + ).map( + tuple -> { + final ServerToSegment serverToSegment = tuple.getLhs(); + final Optional maybeResult = tuple.getRhs(); + if (maybeResult.isPresent()) { + return new ServerMaybeSegmentMaybeCache<>(ALREADY_CACHED_SERVER, Optional.empty(), maybeResult); + } + 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( + ALREADY_CACHED_SERVER, + Optional.empty(), + Optional.empty() + ); + } else { + final DruidServer server = queryableDruidServer.getServer(); + return new ServerMaybeSegmentMaybeCache<>( + server, + Optional.ofNullable(serverToSegment.getSegmentDescriptor()), + maybeResult + ); + } + } + ).collect( + Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) + ).entrySet( + ).stream( + ).map( + Map.Entry::getValue + ).filter( + l -> !l.isEmpty() + ).filter( + l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() + ).map( + l -> { + final Stream cachedResults = l.stream( + ).map( + ServerMaybeSegmentMaybeCache::getCachedValue + ).filter( + Optional::isPresent + ).map( + Optional::get + ); + + final List segmentsOfServer = l.stream( + ).map( + ServerMaybeSegmentMaybeCache::getSegmentDescriptor + ).filter( + Optional::isPresent + ).map( + Optional::get + ).collect( + Collectors.toList() + ); + + if (segmentsOfServer.isEmpty()) { + return Sequences.simple(cachedResults); + } + + final DruidServer server = l.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 MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); + + final Sequence serverResults; + if (isBySegment) { + serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); + } else if (!server.segmentReplicatable() || !populateCache) { + serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec); + } else { + serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); + } + return serverResults; + } + ); + final Sequence> resultSeq = Sequences.simple(resultStream.parallel()); + return resultSeq.flatMerge( + Function.identity(), + query.getResultOrdering() + ); } private Set computeSegmentsToQuery(TimelineLookup timeline) @@ -297,14 +392,13 @@ private Set computeSegmentsToQuery(TimelineLookup segments = Sets.newLinkedHashSet(); - final Map>> dimensionRangeCache = Maps.newHashMap(); // Filter unneeded chunks based on partition dimension for (TimelineObjectHolder holder : serversLookup) { final Set> filteredChunks = DimFilterUtils.filterShards( query.getFilter(), holder.getObject(), partitionChunk -> partitionChunk.getObject().getSegment().getShardSpec(), - dimensionRangeCache + Maps.newHashMap() ); for (PartitionChunk chunk : filteredChunks) { ServerSelector server = chunk.getObject(); @@ -396,60 +490,70 @@ private String computeCurrentEtag(final Set segments, @Nullable } } - private List> pruneSegmentsWithCachedResults( + private Stream>> maybeFetchCacheResults( final byte[] queryCacheKey, - final Set segments + final Stream segments ) { if (queryCacheKey == null) { - return Collections.emptyList(); + return segments.map(s -> new SerializablePair<>(s, Optional.empty())); } - final List> alreadyCachedResults = Lists.newArrayList(); - Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); - // Pull cached segments from cache and remove from set of segments to query - final Map cachedValues = computeCachedValues(perSegmentCacheKeys); - - perSegmentCacheKeys.forEach((segment, segmentCacheKey) -> { - final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); - - final byte[] cachedValue = cachedValues.get(segmentCacheKey); - if (cachedValue != null) { - // remove cached segment from set of segments to query - segments.remove(segment); - alreadyCachedResults.add(Pair.of(segmentQueryInterval, cachedValue)); - } else if (populateCache) { - // otherwise, if populating cache, add segment to list of segments to cache - final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); - addCachePopulator(segmentCacheKey, segmentIdentifier, segmentQueryInterval); - } - }); - return alreadyCachedResults; + // We materialize the stream here in order to have the bulk cache fetching work as expected + final List> materializedKeyList = computePerSegmentCacheKeys( + segments, + queryCacheKey + ).collect(Collectors.toList()); + + // Do bulk fetch + final Map> cachedValues = computeCachedValues( + materializedKeyList.stream() + ).collect( + Collectors.toMap( + SerializablePair::getLhs, + SerializablePair::getRhs + ) + ); + // Hash join to return the stream + return materializedKeyList.stream().map( + psck -> { + final ServerToSegment segment = psck.getLhs(); + final Cache.NamedKey segmentCacheKey = psck.getRhs(); + final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); + final Optional cachedValue = cachedValues.get(segmentCacheKey); + if (!cachedValue.isPresent()) { + // if populating cache, add segment to list of segments to cache if it is not cached + final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); + addCachePopulator(segmentCacheKey, segmentIdentifier, segmentQueryInterval); + } + return new SerializablePair<>(segment, cachedValue); + } + ); } - private Map computePerSegmentCacheKeys( - Set segments, + private Stream> computePerSegmentCacheKeys( + Stream segments, byte[] queryCacheKey ) { - // cacheKeys map must preserve segment ordering, in order for shards to always be combined in the same order - Map cacheKeys = Maps.newLinkedHashMap(); - for (ServerToSegment serverToSegment : segments) { - final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( - serverToSegment.getServer().getSegment().getIdentifier(), - serverToSegment.getSegmentDescriptor(), - queryCacheKey - ); - cacheKeys.put(serverToSegment, segmentCacheKey); - } - return cacheKeys; + return segments.map( + serverToSegment -> { + // cacheKeys map must preserve segment ordering, in order for shards to always be combined in the same order + final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( + serverToSegment.getServer().getSegment().getIdentifier(), + serverToSegment.getSegmentDescriptor(), + queryCacheKey + ); + return new SerializablePair<>(serverToSegment, segmentCacheKey); + } + ); } - private Map computeCachedValues(Map cacheKeys) + private Stream>> computeCachedValues(Stream> cacheKeys) { if (useCache) { - return cache.getBulk(Iterables.limit(cacheKeys.values(), cacheConfig.getCacheBulkMergeLimit())); + return cache.getBulk(cacheKeys.limit(cacheConfig.getCacheBulkMergeLimit()).map(SerializablePair::getRhs)); } else { - return ImmutableMap.of(); + return Stream.empty(); } } @@ -471,86 +575,91 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter return cachePopulatorMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } - private SortedMap> groupSegmentsByServer(Set segments) + private Stream>> groupSegmentsByServer(Stream segments) { - final SortedMap> 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()); - } - } - return serverSegments; + return segments.map( + serverToSegment -> { + 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 Optional.>empty(); + } else { + final DruidServer server = queryableDruidServer.getServer(); + return Optional.of(new SerializablePair<>( + server, + serverToSegment.getSegmentDescriptor() + )); + } + } + ).filter( + Optional::isPresent + ).map( + Optional::get + ).collect( + Collectors.groupingBy( + SerializablePair::getLhs, + TreeMap::new, + Collectors.mapping(SerializablePair::getRhs, Collectors.toList()) + ) + ).entrySet( + ).stream( + ).map( + e -> new SerializablePair<>(e.getKey(), e.getValue()) + ); } - private void addSequencesFromCache( - final List> listOfSequences, - final List> cachedResults + private Stream>> deserializeFromCache( + final Stream>> cachedResults ) { if (strategy == null) { - return; + return cachedResults.map(s -> new SerializablePair<>(s.getLhs(), Optional.empty())); } - - final Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache(); + final com.google.common.base.Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache(); final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); - for (Pair cachedResultPair : cachedResults) { - final byte[] cachedResult = cachedResultPair.rhs; - Sequence cachedSequence = new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - try { - if (cachedResult.length == 0) { - return Collections.emptyIterator(); - } - - return objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } + return cachedResults.map(cachedResultPair -> { + if (!cachedResultPair.getRhs().isPresent()) { + return new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty()); + } + final byte[] cachedResult = cachedResultPair.getRhs().get(); + try { + if (cachedResult.length == 0) { + return new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty()); + } - @Override - public void cleanup(Iterator iterFromMake) - { - } - } - ); - listOfSequences.add(Sequences.map(cachedSequence, pullFromCacheFunction)); - } + final T obj = pullFromCacheFunction.apply(objectMapper.readValues( + objectMapper.getFactory().createParser(cachedResult), + cacheObjectClazz + )); + return new SerializablePair<>(cachedResultPair.getLhs(), Optional.ofNullable(obj)); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); } - private void addSequencesFromServer( - final List> listOfSequences, - final SortedMap> segmentsByServer + private Stream> addSequencesFromServer( + final Stream>> segmentsByServer ) { - segmentsByServer.forEach((server, segmentsOfServer) -> { + return segmentsByServer.flatMap(entry -> { + final DruidServer server = entry.getLhs(); + final List segmentsOfServer = entry.getRhs(); final QueryRunner serverRunner = serverView.getQueryRunner(server); if (serverRunner == null) { log.error("Server[%s] doesn't have a query runner", server); - return; + return Stream.empty(); } final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); - Sequence serverResults; + final Sequence serverResults; if (isBySegment) { serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); } else if (!server.segmentReplicatable() || !populateCache) { @@ -558,7 +667,7 @@ private void addSequencesFromServer( } else { serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); } - listOfSequences.add(serverResults); + return Stream.of(serverResults); }); } @@ -600,7 +709,7 @@ private Sequence getAndCacheServerResults( .withQuerySegmentSpec(segmentsOfServerSpec), responseContext ); - final Function cacheFn = strategy.prepareForSegmentLevelCache(); + final com.google.common.base.Function cacheFn = strategy.prepareForSegmentLevelCache(); return resultsBySegments .map(result -> { final BySegmentResultValueClass resultsOfSegment = result.getValue(); @@ -623,7 +732,40 @@ private Sequence getAndCacheServerResults( } return res; }) - .flatMerge(seq -> seq, query.getResultOrdering()); + .flatMerge(Function.identity(), query.getResultOrdering()); + } + } + + private static class ServerMaybeSegmentMaybeCache + { + private final DruidServer server; + private final Optional segmentDescriptor; + private final Optional cachedValue; + + public DruidServer getServer() + { + return server; + } + + public Optional getSegmentDescriptor() + { + return segmentDescriptor; + } + + public Optional getCachedValue() + { + return cachedValue; + } + + private ServerMaybeSegmentMaybeCache( + DruidServer server, + Optional segmentDescriptor, + Optional cachedValue + ) + { + this.server = server; + this.segmentDescriptor = segmentDescriptor; + this.cachedValue = cachedValue; } } diff --git a/server/src/main/java/io/druid/client/cache/Cache.java b/server/src/main/java/io/druid/client/cache/Cache.java index afd1a696b796..abbd2d304f79 100644 --- a/server/src/main/java/io/druid/client/cache/Cache.java +++ b/server/src/main/java/io/druid/client/cache/Cache.java @@ -20,13 +20,17 @@ package io.druid.client.cache; import com.google.common.base.Preconditions; +import io.druid.collections.SerializablePair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.emitter.service.ServiceEmitter; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Iterator; import java.util.Map; +import java.util.Optional; +import java.util.stream.Stream; /** */ @@ -44,6 +48,17 @@ public interface Cache */ Map getBulk(Iterable keys); + /** + * Returns a stream of the input keys with an optional byte array if the key was found in the cache + * + * @param keys + * @return + */ + default Stream>> getBulk(Stream keys) + { + return keys.map(key -> new SerializablePair<>(key, Optional.ofNullable(get(key)))); + } + void close(String namespace); CacheStats getStats(); diff --git a/server/src/main/java/io/druid/client/cache/CaffeineCache.java b/server/src/main/java/io/druid/client/cache/CaffeineCache.java index 6bea41318ed6..1439ba377e0e 100644 --- a/server/src/main/java/io/druid/client/cache/CaffeineCache.java +++ b/server/src/main/java/io/druid/client/cache/CaffeineCache.java @@ -25,6 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import io.druid.collections.SerializablePair; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.java.util.emitter.service.ServiceMetricEvent; @@ -34,10 +35,12 @@ import java.nio.ByteBuffer; import java.util.Map; +import java.util.Optional; import java.util.OptionalLong; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Stream; public class CaffeineCache implements io.druid.client.cache.Cache { @@ -105,6 +108,21 @@ public Map getBulk(Iterable keys) return ImmutableMap.copyOf(Maps.transformValues(cache.getAllPresent(keys), this::deserialize)); } + @Override + public Stream>> getBulk(Stream keys) + { + return keys.map( + k -> new SerializablePair<>( + k, + Optional.ofNullable( + cache.getIfPresent(k) + ).map( + this::deserialize + ) + ) + ); + } + // This is completely racy with put. Any values missed should be evicted later anyways. So no worries. @Override public void close(String namespace) diff --git a/server/src/main/java/io/druid/client/cache/HybridCache.java b/server/src/main/java/io/druid/client/cache/HybridCache.java index 8787a985a395..0a42625a4621 100644 --- a/server/src/main/java/io/druid/client/cache/HybridCache.java +++ b/server/src/main/java/io/druid/client/cache/HybridCache.java @@ -21,14 +21,19 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.druid.java.util.emitter.service.ServiceEmitter; +import io.druid.collections.SerializablePair; import io.druid.java.util.common.logger.Logger; +import io.druid.java.util.emitter.service.ServiceEmitter; import javax.annotation.Nullable; import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import java.util.stream.Stream; public class HybridCache implements Cache { @@ -124,6 +129,41 @@ private Map getBulkL2(Iterable keys) } } + @Override + public Stream>> getBulk(Stream keys) + { + if (!config.getUseL2()) { + return level1.getBulk(keys); + } + final List>> materializedL1Results = level1 + .getBulk(keys) + .collect(Collectors.toList()); + final List>> materializedL2Results = level2 + .getBulk( + materializedL1Results.stream( + ).filter( + s -> !s.getRhs().isPresent() + ).map( + SerializablePair::getLhs + ) + ).collect(Collectors.toList()); + // The l2 list should only have "missing" ones from l1. So we loop through and look for the missing L1 results + // and replace with whatever l2 found + int l2Pos = 0; + for (int i = 0; i < materializedL1Results.size(); i++) { + final SerializablePair> me = materializedL1Results.get(i); + if (!me.getRhs().isPresent()) { + final SerializablePair> other = materializedL2Results.get(l2Pos++); + if (!me.getLhs().equals(other.getLhs())) { + // sanity check for something very broken + break; + } + materializedL1Results.set(i, other); + } + } + return materializedL1Results.stream(); + } + @Override public void close(String namespace) { diff --git a/server/src/main/java/io/druid/client/cache/MapCache.java b/server/src/main/java/io/druid/client/cache/MapCache.java index f9257fc52352..66ad7f4a72fa 100644 --- a/server/src/main/java/io/druid/client/cache/MapCache.java +++ b/server/src/main/java/io/druid/client/cache/MapCache.java @@ -22,6 +22,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.primitives.Ints; +import io.druid.collections.SerializablePair; import io.druid.java.util.emitter.service.ServiceEmitter; import java.nio.ByteBuffer; @@ -29,8 +30,10 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; /** */ @@ -113,6 +116,21 @@ public Map getBulk(Iterable keys) return retVal; } + + @Override + public Stream> getBulk(Stream keys) + { + return keys.map(k -> { + final byte[] v = get(k); + if (v == null) { + return null; + } + return new SerializablePair<>(k, v); + }).filter( + Objects::nonNull + ); + } + @Override public void close(String namespace) { diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCache.java b/server/src/main/java/io/druid/client/cache/MemcachedCache.java index 7010134b2da5..67a00b602b98 100644 --- a/server/src/main/java/io/druid/client/cache/MemcachedCache.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCache.java @@ -19,7 +19,6 @@ package io.druid.client.cache; -import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Supplier; @@ -27,10 +26,10 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Maps; import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.collections.ResourceHolder; +import io.druid.collections.SerializablePair; import io.druid.collections.StupidResourceHolder; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; @@ -57,9 +56,12 @@ import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -69,6 +71,9 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; public class MemcachedCache implements Cache { @@ -524,59 +529,35 @@ private static byte[] deserializeValue(NamedKey key, byte[] bytes) return value; } - @Override - public Map getBulk(Iterable keys) + + Map getCacheMap(Collection keys) { + if (keys.isEmpty()) { + return Collections.emptyMap(); + } + // Hold onto the client until the future is fetched try (ResourceHolder clientHolder = client.get()) { - Map keyLookup = Maps.uniqueIndex( - keys, - new Function() - { - @Override - public String apply( - @Nullable NamedKey input - ) - { - return computeKeyHash(memcachedPrefix, input); - } - } - ); - - Map results = Maps.newHashMap(); - - BulkFuture> future; + final BulkFuture> future; try { - future = clientHolder.get().asyncGetBulk(keyLookup.keySet()); + future = clientHolder.get().asyncGetBulk(keys); } catch (IllegalStateException e) { // operation did not get queued in time (queue is full) errorCount.incrementAndGet(); log.warn(e, "Unable to queue cache operation"); - return results; + return Collections.emptyMap(); } try { - Map some = future.getSome(timeout, TimeUnit.MILLISECONDS); + final Map some = future.getSome(timeout, TimeUnit.MILLISECONDS); if (future.isTimeout()) { future.cancel(false); timeoutCount.incrementAndGet(); } - missCount.addAndGet(keyLookup.size() - some.size()); + missCount.addAndGet(keys.size() - some.size()); hitCount.addAndGet(some.size()); - - for (Map.Entry entry : some.entrySet()) { - final NamedKey key = keyLookup.get(entry.getKey()); - final byte[] value = (byte[]) entry.getValue(); - if (value != null) { - results.put( - key, - deserializeValue(key, value) - ); - } - } - - return results; + return some; } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -585,11 +566,51 @@ public String apply( catch (ExecutionException e) { errorCount.incrementAndGet(); log.warn(e, "Exception pulling item from cache"); - return results; + return Collections.emptyMap(); } } } + @Override + public Stream>> getBulk(Stream keys) + { + final List> materializedKeys = keys.map( + k -> new SerializablePair<>(k, computeKeyHash(memcachedPrefix, k)) + ).collect( + Collectors.toList() + ); + final Map some = getCacheMap( + materializedKeys.stream( + ).map( + SerializablePair::getRhs + ).collect( + Collectors.toList() + ) + ); + return materializedKeys.stream().map(k -> { + final NamedKey key = k.getLhs(); + final String cacheKey = k.getRhs(); + return new SerializablePair<>( + key, + Optional.ofNullable( + some.get(cacheKey) + ).map( + val -> deserializeValue(key, (byte[]) val) + ) + ); + }); + } + + @Override + public Map getBulk(Iterable keys) + { + return getBulk( + StreamSupport.stream(keys.spliterator(), false) + ).filter(s -> s.getRhs().isPresent()).collect( + Collectors.toMap(SerializablePair::getLhs, s -> s.getRhs().get()) + ); + } + @Override public void close(String namespace) { From fe40bca46f6183613dabc68f7a20275d165e7e78 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 20 Jun 2018 16:51:59 -0700 Subject: [PATCH 02/58] Fix up build --- .../java/util/common/guava/MergeSequence.java | 1 - .../DataSourceQueryQueryToolChest.java | 21 +-- .../SegmentMetadataQueryQueryToolChest.java | 25 ++-- .../select/SelectQueryQueryToolChest.java | 139 ++++++++++-------- .../TimeBoundaryQueryQueryToolChest.java | 22 +-- .../druid/client/CachingClusteredClient.java | 125 ++++------------ .../java/io/druid/client/cache/Cache.java | 11 +- .../java/io/druid/client/cache/MapCache.java | 18 --- 8 files changed, 133 insertions(+), 229 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index b22d8a9307c4..e9f0628fccd3 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -19,7 +19,6 @@ package io.druid.java.util.common.guava; -import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import io.druid.java.util.common.io.Closer; diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index 9a9c5c16659a..ce37d4e97d23 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -22,9 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.inject.Inject; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -42,6 +39,7 @@ import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -69,18 +67,11 @@ public List filterSegments(DataSourceMetadataQuery final T max = segments.get(segments.size() - 1); - return Lists.newArrayList( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(T input) - { - return max != null && input.getInterval().overlaps(max.getInterval()); - } - } - ) + return segments.stream( + ).filter( + segment -> max != null && segment.getInterval().overlaps(max.getInterval()) + ).collect( + Collectors.toList() ); } 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 5025d9dbc975..875535c426dd 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -23,17 +23,15 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; import io.druid.common.guava.CombiningSequence; -import io.druid.java.util.common.JodaUtils; import io.druid.data.input.impl.TimestampSpec; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.MappedSequence; @@ -64,6 +62,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { @@ -111,7 +110,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( @@ -240,18 +240,11 @@ public List filterSegments(SegmentMetadataQuery qu DateTime targetEnd = max.getInterval().getEnd(); final Interval targetInterval = new Interval(config.getDefaultHistory(), targetEnd); - return Lists.newArrayList( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(T input) - { - return (input.getInterval().overlaps(targetInterval)); - } - } - ) + return segments.stream( + ).filter( + segment -> segment.getInterval().overlaps(targetInterval) + ).collect( + Collectors.toList() ); } 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 c0c3d8278502..bf09b5ea9a80 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -24,11 +24,9 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; @@ -62,6 +60,9 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.function.Predicate; +import java.util.stream.Collector; +import java.util.stream.Collectors; /** */ @@ -163,8 +164,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) @@ -287,22 +290,32 @@ 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 @@ -369,62 +382,58 @@ public List filterSegments(SelectQuery query, List // A paged select query using a UnionDataSource will return pagingIdentifiers from segments in more than one // dataSource which confuses subsequent queries and causes a failure. To avoid this, filter only the paging keys // that are applicable to this dataSource so that each dataSource in a union query gets the appropriate keys. - final Iterable filteredPagingKeys = Iterables.filter( - paging.keySet(), new Predicate() - { - @Override - public boolean apply(String input) - { - return DataSegmentUtils.valueOf(dataSource, input) != null; - } - } - ); - List intervals = Lists.newArrayList( - Iterables.transform(filteredPagingKeys, DataSegmentUtils.INTERVAL_EXTRACTOR(dataSource)) - ); - Collections.sort( - intervals, query.isDescending() ? Comparators.intervalsByEndThenStart() - : Comparators.intervalsByStartThenEnd() - ); + // The tree map needs to either be based off of max end or min start, depending on if we are ASC or DESC + final Collector> mapCollector = + query.isDescending() + ? Collectors.toMap( + interval -> granularity.bucketStart(interval.getEnd()).getMillis(), + Interval::getEndMillis, + Math::max, + TreeMap::new + ) + : Collectors.toMap( + interval -> granularity.bucketStart(interval.getStart()).getMillis(), + Interval::getStartMillis, + Math::min, + TreeMap::new + ); - TreeMap granularThresholds = Maps.newTreeMap(); - for (Interval interval : intervals) { - if (query.isDescending()) { - long granularEnd = granularity.bucketStart(interval.getEnd()).getMillis(); - Long currentEnd = granularThresholds.get(granularEnd); - if (currentEnd == null || interval.getEndMillis() > currentEnd) { - granularThresholds.put(granularEnd, interval.getEndMillis()); - } - } else { - long granularStart = granularity.bucketStart(interval.getStart()).getMillis(); - Long currentStart = granularThresholds.get(granularStart); - if (currentStart == null || interval.getStartMillis() < currentStart) { - granularThresholds.put(granularStart, interval.getStartMillis()); - } - } - } + final TreeMap granularThresholds = paging.keySet( + ).stream( + ).filter( + key -> DataSegmentUtils.valueOf(dataSource, key) != null + ).map( + DataSegmentUtils.INTERVAL_EXTRACTOR(dataSource)::apply + ).sorted( + query.isDescending() ? Comparators.intervalsByEndThenStart() + : Comparators.intervalsByStartThenEnd() + ).collect( + mapCollector + ); - List queryIntervals = Lists.newArrayList(segments); + // Out of bounds for time if descending and too low, or ascending and too high + final Predicate outOfTimeBounds = query.isDescending( + ) ? segment -> { + final Interval interval = segment.getInterval(); + final Map.Entry ceiling = granularThresholds.ceilingEntry( + granularity.bucketStart(interval.getEnd()) + .getMillis()); + return ceiling != null + && interval.getStartMillis() < ceiling.getValue(); + } : segment -> { + final Interval interval = segment.getInterval(); + final Map.Entry floor = granularThresholds.floorEntry( + granularity.bucketStart(interval.getStart()) + .getMillis()); + return floor != null && interval.getEndMillis() > floor.getValue(); + }; - Iterator it = queryIntervals.iterator(); - if (query.isDescending()) { - while (it.hasNext()) { - Interval interval = it.next().getInterval(); - Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()).getMillis()); - if (ceiling == null || interval.getStartMillis() >= ceiling.getValue()) { - it.remove(); - } - } - } else { - while (it.hasNext()) { - Interval interval = it.next().getInterval(); - Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()).getMillis()); - if (floor == null || interval.getEndMillis() <= floor.getValue()) { - it.remove(); - } - } - } - return queryIntervals; + return segments.stream( + ).filter( + outOfTimeBounds + ).collect( + Collectors.toList() + ); } } 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 a1046a00c05b..f8a0b95b360c 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -23,8 +23,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.inject.Inject; import io.druid.java.util.common.DateTimes; @@ -46,6 +44,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; /** */ @@ -85,19 +84,12 @@ public List filterSegments(TimeBoundaryQuery query final T min = query.isMaxTime() ? null : segments.get(0); final T max = query.isMinTime() ? null : segments.get(segments.size() - 1); - return Lists.newArrayList( - Iterables.filter( - segments, - new Predicate() - { - @Override - public boolean apply(T input) - { - return (min != null && input.getInterval().overlaps(min.getInterval())) || - (max != null && input.getInterval().overlaps(max.getInterval())); - } - } - ) + return segments.stream( + ).filter( + segment -> (min != null && segment.getInterval().overlaps(min.getInterval())) || + (max != null && segment.getInterval().overlaps(max.getInterval())) + ).collect( + Collectors.toList() ); } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 5e998024236b..515e90b29559 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -23,7 +23,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import com.google.common.collect.Sets; import com.google.common.hash.Hasher; import com.google.common.hash.Hashing; import com.google.common.util.concurrent.FutureCallback; @@ -82,7 +81,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.TreeMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; @@ -274,14 +272,18 @@ Sequence run(final UnaryOperator> time computeUncoveredIntervals(timeline); } - final Set segments = computeSegmentsToQuery(timeline); + Stream segments = computeSegmentsToQuery(timeline); @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { + // Materialize then re-stream + List 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(); } @@ -290,7 +292,7 @@ Sequence run(final UnaryOperator> time final Stream> resultStream = deserializeFromCache( maybeFetchCacheResults( queryCacheKey, - segments.stream() + segments ) ).map( tuple -> { @@ -384,33 +386,31 @@ Sequence run(final UnaryOperator> time ); } - private Set computeSegmentsToQuery(TimelineLookup timeline) + private Stream computeSegmentsToQuery(TimelineLookup timeline) { - final List> serversLookup = toolChest.filterSegments( + return toolChest.filterSegments( query, query.getIntervals().stream().flatMap(i -> timeline.lookup(i).stream()).collect(Collectors.toList()) - ); - - final Set segments = Sets.newLinkedHashSet(); - // Filter unneeded chunks based on partition dimension - for (TimelineObjectHolder holder : serversLookup) { - final Set> filteredChunks = DimFilterUtils.filterShards( - query.getFilter(), - holder.getObject(), - partitionChunk -> partitionChunk.getObject().getSegment().getShardSpec(), - Maps.newHashMap() - ); - for (PartitionChunk 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; + ).stream( + ).flatMap( + holder -> DimFilterUtils.filterShards( + query.getFilter(), + holder.getObject(), + partitionChunk -> partitionChunk.getObject().getSegment().getShardSpec(), + Maps.newHashMap() + ).stream( + ).map( + chunk -> { + ServerSelector server = chunk.getObject(); + final SegmentDescriptor segment = new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ); + return new ServerToSegment(server, segment); + } + ) + ).distinct(); } private void computeUncoveredIntervals(TimelineLookup timeline) @@ -467,7 +467,7 @@ private byte[] computeQueryCacheKey() } @Nullable - private String computeCurrentEtag(final Set segments, @Nullable byte[] queryCacheKey) + private String computeCurrentEtag(final Iterable segments, @Nullable byte[] queryCacheKey) { Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; @@ -575,43 +575,6 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter return cachePopulatorMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } - private Stream>> groupSegmentsByServer(Stream segments) - { - return segments.map( - serverToSegment -> { - 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 Optional.>empty(); - } else { - final DruidServer server = queryableDruidServer.getServer(); - return Optional.of(new SerializablePair<>( - server, - serverToSegment.getSegmentDescriptor() - )); - } - } - ).filter( - Optional::isPresent - ).map( - Optional::get - ).collect( - Collectors.groupingBy( - SerializablePair::getLhs, - TreeMap::new, - Collectors.mapping(SerializablePair::getRhs, Collectors.toList()) - ) - ).entrySet( - ).stream( - ).map( - e -> new SerializablePair<>(e.getKey(), e.getValue()) - ); - } - private Stream>> deserializeFromCache( final Stream>> cachedResults ) @@ -643,34 +606,6 @@ private Stream>> deserializeFromCa }); } - private Stream> addSequencesFromServer( - final Stream>> segmentsByServer - ) - { - return segmentsByServer.flatMap(entry -> { - final DruidServer server = entry.getLhs(); - final List segmentsOfServer = entry.getRhs(); - final QueryRunner serverRunner = serverView.getQueryRunner(server); - - if (serverRunner == null) { - log.error("Server[%s] doesn't have a query runner", server); - return Stream.empty(); - } - - final MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); - - final Sequence serverResults; - if (isBySegment) { - serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); - } else if (!server.segmentReplicatable() || !populateCache) { - serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec); - } else { - serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); - } - return Stream.of(serverResults); - }); - } - @SuppressWarnings("unchecked") private Sequence getBySegmentServerResults( final QueryRunner serverRunner, diff --git a/server/src/main/java/io/druid/client/cache/Cache.java b/server/src/main/java/io/druid/client/cache/Cache.java index abbd2d304f79..a5f9e4dd79bb 100644 --- a/server/src/main/java/io/druid/client/cache/Cache.java +++ b/server/src/main/java/io/druid/client/cache/Cache.java @@ -27,7 +27,6 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Iterator; import java.util.Map; import java.util.Optional; import java.util.stream.Stream; @@ -38,12 +37,14 @@ public interface Cache { @Nullable byte[] get(NamedKey key); + void put(NamedKey key, byte[] value); /** * Resulting map should not contain any null values (i.e. cache misses should not be included) * * @param keys + * * @return */ Map getBulk(Iterable keys); @@ -52,6 +53,7 @@ public interface Cache * Returns a stream of the input keys with an optional byte array if the key was found in the cache * * @param keys + * * @return */ default Stream>> getBulk(Stream keys) @@ -67,6 +69,7 @@ default Stream>> getBulk(Stream getBulk(Iterable keys) return retVal; } - - @Override - public Stream> getBulk(Stream keys) - { - return keys.map(k -> { - final byte[] v = get(k); - if (v == null) { - return null; - } - return new SerializablePair<>(k, v); - }).filter( - Objects::nonNull - ); - } - @Override public void close(String namespace) { From 20659d206527f5cd5fc41b1fbd304d1da2b4f3b7 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 20 Jun 2018 18:07:41 -0700 Subject: [PATCH 03/58] Work on getting tests to pass --- .../java/util/common/guava/MergeIterator.java | 9 +- .../druid/client/CachingClusteredClient.java | 37 ++- .../client/CachingClusteredClientTest.java | 297 +++++++++--------- 3 files changed, 180 insertions(+), 163 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java index ada5577b6840..f946951a8133 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java @@ -40,14 +40,7 @@ public MergeIterator( { pQueue = new PriorityQueue<>( 16, - new Comparator>() - { - @Override - public int compare(PeekingIterator lhs, PeekingIterator rhs) - { - return comparator.compare(lhs.peek(), rhs.peek()); - } - } + (lhs, rhs) -> comparator.compare(lhs.peek(), rhs.peek()) ); for (Iterator iterator : iterators) { diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 515e90b29559..2f200bf448bc 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -81,13 +81,14 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.TreeMap; +import java.util.Spliterators; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.function.Function; import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.Stream; +import java.util.stream.StreamSupport; /** */ @@ -379,7 +380,7 @@ Sequence run(final UnaryOperator> time return serverResults; } ); - final Sequence> resultSeq = Sequences.simple(resultStream.parallel()); + final Sequence> resultSeq = Sequences.simple(resultStream); return resultSeq.flatMerge( Function.identity(), query.getResultOrdering() @@ -519,7 +520,8 @@ private Stream>> maybeFetchCa final ServerToSegment segment = psck.getLhs(); final Cache.NamedKey segmentCacheKey = psck.getRhs(); final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); - final Optional cachedValue = cachedValues.get(segmentCacheKey); + final Optional cachedValue = Optional.ofNullable(cachedValues.get(segmentCacheKey)) + .orElse(Optional.empty()); // Shouldn't happen in practice, but can screw up unit tests if (!cachedValue.isPresent()) { // if populating cache, add segment to list of segments to cache if it is not cached final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); @@ -582,23 +584,32 @@ private Stream>> deserializeFromCa if (strategy == null) { return cachedResults.map(s -> new SerializablePair<>(s.getLhs(), Optional.empty())); } - final com.google.common.base.Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache(); + final Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache()::apply; final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); - return cachedResults.map(cachedResultPair -> { + return cachedResults.flatMap(cachedResultPair -> { if (!cachedResultPair.getRhs().isPresent()) { - return new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty()); + return Stream.of(new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty())); } final byte[] cachedResult = cachedResultPair.getRhs().get(); try { if (cachedResult.length == 0) { - return new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty()); + return Stream.of(new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty())); } - - final T obj = pullFromCacheFunction.apply(objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - )); - return new SerializablePair<>(cachedResultPair.getLhs(), Optional.ofNullable(obj)); + // Query granularity in a segment may be higher fidelity than the segment as a file, so this might have multiple results + return StreamSupport.stream( + Spliterators.spliteratorUnknownSize( + objectMapper.readValues( + objectMapper.getFactory().createParser(cachedResult), + cacheObjectClazz + ), + 0 + ), + false + ).map( + pullFromCacheFunction + ).map( + obj -> new SerializablePair<>(cachedResultPair.getLhs(), Optional.ofNullable(obj)) + ); } catch (IOException e) { throw new RuntimeException(e); diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 39ec5f1e5e12..a9a2fa138b3b 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -163,6 +163,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; +import java.util.stream.Stream; /** */ @@ -258,42 +259,43 @@ public class CachingClusteredClientTest private static final Granularity PT1H_TZ_GRANULARITY = new PeriodGranularity(new Period("PT1H"), null, TIMEZONE); private static final String TOP_DIM = "a_dim"; - private static final Supplier selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig(true)); + private static final Supplier selectConfigSupplier = Suppliers.ofInstance(new SelectQueryConfig( + true)); static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse( ImmutableMap., QueryToolChest>builder() - .put( - TimeseriesQuery.class, - new TimeseriesQueryQueryToolChest( - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - TopNQuery.class, new TopNQueryQueryToolChest( - new TopNQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - SearchQuery.class, new SearchQueryQueryToolChest( - new SearchQueryConfig(), - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() - ) - ) - .put( - SelectQuery.class, - new SelectQueryQueryToolChest( - jsonMapper, - QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), - selectConfigSupplier - ) - ) - .put( - GroupByQuery.class, - GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() - ) - .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) - .build() + .put( + TimeseriesQuery.class, + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + TopNQuery.class, new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + SearchQuery.class, new SearchQueryQueryToolChest( + new SearchQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ) + .put( + SelectQuery.class, + new SelectQueryQueryToolChest( + jsonMapper, + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + selectConfigSupplier + ) + ) + .put( + GroupByQuery.class, + GroupByQueryRunnerTest.makeQueryRunnerFactory(new GroupByQueryConfig()).getToolchest() + ) + .put(TimeBoundaryQuery.class, new TimeBoundaryQueryQueryToolChest()) + .build() ); private final Random random; public CachingClusteredClient client; @@ -575,9 +577,9 @@ public void testCachingOverBulkLimitEnforcesLimit() final Map context = new HashMap<>(); final Cache cache = EasyMock.createStrictMock(Cache.class); - final Capture> cacheKeyCapture = EasyMock.newCapture(); + final Capture> cacheKeyCapture = EasyMock.newCapture(); EasyMock.expect(cache.getBulk(EasyMock.capture(cacheKeyCapture))) - .andReturn(ImmutableMap.of()) + .andReturn(Stream.empty()) .once(); EasyMock.replay(cache); client = makeClient(MoreExecutors.sameThreadExecutor(), cache, limit); @@ -595,14 +597,14 @@ public void testCachingOverBulkLimitEnforcesLimit() getDefaultQueryRunner().run(QueryPlus.wrap(query), context); Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); - Assert.assertTrue("Cache key below limit", ImmutableList.copyOf(cacheKeyCapture.getValue()).size() <= limit); + Assert.assertTrue("Cache key below limit", cacheKeyCapture.getValue().count() <= limit); EasyMock.verify(cache); EasyMock.reset(cache); cacheKeyCapture.reset(); EasyMock.expect(cache.getBulk(EasyMock.capture(cacheKeyCapture))) - .andReturn(ImmutableMap.of()) + .andReturn(Stream.empty()) .once(); EasyMock.replay(cache); client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0); @@ -610,7 +612,7 @@ public void testCachingOverBulkLimitEnforcesLimit() EasyMock.verify(cache); EasyMock.verify(dataSegment); Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); - Assert.assertTrue("Cache Keys empty", ImmutableList.copyOf(cacheKeyCapture.getValue()).isEmpty()); + Assert.assertTrue("Cache Keys empty", cacheKeyCapture.getValue().count() == 0); } @Test @@ -1194,14 +1196,14 @@ public void testSearchCaching() public void testSearchCachingRenamedOutput() { final Druids.SearchQueryBuilder builder = Druids.newSearchQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .limit(1000) - .intervals(SEG_SPEC) - .dimensions(Arrays.asList(TOP_DIM)) - .query("how") - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .limit(1000) + .intervals(SEG_SPEC) + .dimensions(Arrays.asList(TOP_DIM)) + .query("how") + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1309,18 +1311,18 @@ public void testSelectCaching() Intervals.of("2011-01-05/2011-01-10"), makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05"), - DateTimes.of("2011-01-06"), - DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), - DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-06"), + DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), + DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9) ), Intervals.of("2011-01-05/2011-01-10"), makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ) ); @@ -1335,17 +1337,17 @@ public void testSelectCaching() HashMap context = new HashMap(); TestHelper.assertExpectedResults( makeSelectResults(dimensions, metrics, DateTimes.of("2011-01-01"), ImmutableMap.of("a", "b", "rows", 1), - DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5), - DateTimes.of("2011-01-05"), - DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), - DateTimes.of("2011-01-06"), - DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), - DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), - DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), - DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), - DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), - DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) + DateTimes.of("2011-01-02"), ImmutableMap.of("a", "c", "rows", 5), + DateTimes.of("2011-01-05"), + DateTimes.of("2011-01-05T01"), ImmutableMap.of("a", "d", "rows", 5), + DateTimes.of("2011-01-06"), + DateTimes.of("2011-01-06T01"), ImmutableMap.of("a", "e", "rows", 6), + DateTimes.of("2011-01-07"), ImmutableMap.of("a", "f", "rows", 7), ImmutableMap.of("a", "ff"), + DateTimes.of("2011-01-07T01"), ImmutableMap.of("a", "f", "rows", 7), + DateTimes.of("2011-01-08"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-08T01"), ImmutableMap.of("a", "g", "rows", 8), + DateTimes.of("2011-01-09"), ImmutableMap.of("a", "h", "rows", 9), + DateTimes.of("2011-01-09T01"), ImmutableMap.of("a", "h", "rows", 9) ), runner.run(QueryPlus.wrap(builder.intervals("2011-01-01/2011-01-10").build()), context) ); @@ -1358,14 +1360,14 @@ public void testSelectCachingRenamedOutputName() final Set metrics = Sets.newHashSet("rows"); Druids.SelectQueryBuilder builder = Druids.newSelectQueryBuilder() - .dataSource(DATA_SOURCE) - .intervals(SEG_SPEC) - .filters(DIM_FILTER) - .granularity(GRANULARITY) - .dimensions(Arrays.asList("a")) - .metrics(Arrays.asList("rows")) - .pagingSpec(new PagingSpec(null, 3)) - .context(CONTEXT); + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .dimensions(Arrays.asList("a")) + .metrics(Arrays.asList("rows")) + .pagingSpec(new PagingSpec(null, 3)) + .context(CONTEXT); testQueryCaching( getDefaultQueryRunner(), @@ -1453,9 +1455,9 @@ public void testSelectCachingRenamedOutputName() public void testGroupByCaching() { List aggsWithUniques = ImmutableList.builder() - .addAll(AGGS) - .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) - .build(); + .addAll(AGGS) + .add(new HyperUniquesAggregatorFactory("uniques", "uniques")) + .build(); final HashFunction hashFn = Hashing.murmur3_128(); @@ -1656,9 +1658,11 @@ For dim1 (2011-01-06/2011-01-10), the combined range for the bound filters is {( makeTimeResults(DateTimes.of("2011-01-01"), 50, 5000, DateTimes.of("2011-01-02"), 10, 1252, DateTimes.of("2011-01-03"), 20, 6213, - DateTimes.of("2011-01-04"), 30, 743), + DateTimes.of("2011-01-04"), 30, 743 + ), makeTimeResults(DateTimes.of("2011-01-07"), 60, 6020, - DateTimes.of("2011-01-08"), 70, 250) + DateTimes.of("2011-01-08"), 70, 250 + ) ); testQueryCachingWithFilter( @@ -1696,14 +1700,14 @@ public void testSingleDimensionPruning() ); final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SOURCE) - .filters(filter) - .granularity(GRANULARITY) - .intervals(SEG_SPEC) - .context(CONTEXT) - .intervals("2011-01-05/2011-01-10") - .aggregators(RENAMED_AGGS) - .postAggregators(RENAMED_POST_AGGS); + .dataSource(DATA_SOURCE) + .filters(filter) + .granularity(GRANULARITY) + .intervals(SEG_SPEC) + .context(CONTEXT) + .intervals("2011-01-05/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS); TimeseriesQuery query = builder.build(); Map context = new HashMap<>(); @@ -1759,7 +1763,8 @@ public void testSingleDimensionPruning() } private ServerSelector makeMockSingleDimensionSelector( - DruidServer server, String dimension, String start, String end, int partitionNum) + DruidServer server, String dimension, String start, String end, int partitionNum + ) { DataSegment segment = EasyMock.createNiceMock(DataSegment.class); EasyMock.expect(segment.getIdentifier()).andReturn(DATA_SOURCE).anyTimes(); @@ -1889,7 +1894,12 @@ public void testQueryCachingWithFilter( @Override public Sequence answer() { - return toFilteredQueryableTimeseriesResults((TimeseriesQuery) capture.getValue().getQuery(), segmentIds, queryIntervals, results); + return toFilteredQueryableTimeseriesResults( + (TimeseriesQuery) capture.getValue().getQuery(), + segmentIds, + queryIntervals, + results + ); } }) .times(0, 1); @@ -1947,7 +1957,11 @@ private Sequence> toFilteredQueryableTimeseriesRes MultipleSpecificSegmentSpec spec = (MultipleSpecificSegmentSpec) query.getQuerySegmentSpec(); List> ret = Lists.newArrayList(); for (SegmentDescriptor descriptor : spec.getDescriptors()) { - String id = StringUtils.format("%s_%s", queryIntervals.indexOf(descriptor.getInterval()), descriptor.getPartitionNumber()); + String id = StringUtils.format( + "%s_%s", + queryIntervals.indexOf(descriptor.getInterval()), + descriptor.getPartitionNumber() + ); int index = segmentIds.indexOf(id); if (index != -1) { ret.add(new Result( @@ -2107,50 +2121,40 @@ public void testQueryCaching( } runWithMocks( - new Runnable() - { - @Override - public void run() - { - HashMap context = new HashMap(); - for (int i = 0; i < numTimesToQuery; ++i) { - TestHelper.assertExpectedResults( - new MergeIterable<>( - Comparators.naturalNullsFirst(), - FunctionalIterable - .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) - .transformCat( - new Function>>>() - { - @Override - public Iterable>> apply(@Nullable Integer input) - { - List>> retVal = Lists.newArrayList(); - - final Map exps = serverExpectationList.get(input); - for (ServerExpectations expectations : exps.values()) { - for (ServerExpectation expectation : expectations) { - retVal.add(expectation.getResults()); - } - } - - return retVal; + () -> { + HashMap context = new HashMap<>(); + for (int i1 = 0; i1 < numTimesToQuery; ++i1) { + TestHelper.assertExpectedResults( + new MergeIterable<>( + Comparators.naturalNullsFirst(), + FunctionalIterable + .create(new RangeIterable(expectedResultsRangeStart, expectedResultsRangeEnd)) + .transformCat( + (Function>>>) input -> { + List>> retVal = Lists.newArrayList(); + + final Map exps = serverExpectationList.get(input); + for (ServerExpectations expectations : exps.values()) { + for (ServerExpectation expectation : expectations) { + retVal.add(expectation.getResults()); } } - ) - ), - runner.run( - QueryPlus.wrap( - query.withQuerySegmentSpec( - new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval)) - ) - ), - context - ) - ); - if (queryCompletedCallback != null) { - queryCompletedCallback.run(); - } + + return retVal; + } + ) + ), + runner.run( + QueryPlus.wrap( + query.withQuerySegmentSpec( + new MultipleIntervalSegmentSpec(ImmutableList.of(actualQueryInterval)) + ) + ), + context + ) + ); + if (queryCompletedCallback != null) { + queryCompletedCallback.run(); } } }, @@ -2459,13 +2463,13 @@ private Iterable> makeTimeResults(Object... object (DateTime) objects[i], new TimeseriesResultValue( ImmutableMap.builder() - .put("rows", objects[i + 1]) - .put("imps", objects[i + 2]) - .put("impers", objects[i + 2]) - .put("avg_imps_per_row", avg_impr) - .put("avg_imps_per_row_half", avg_impr / 2) - .put("avg_imps_per_row_double", avg_impr * 2) - .build() + .put("rows", objects[i + 1]) + .put("imps", objects[i + 2]) + .put("impers", objects[i + 2]) + .put("avg_imps_per_row", avg_impr) + .put("avg_imps_per_row_half", avg_impr / 2) + .put("avg_imps_per_row_double", avg_impr * 2) + .build() ) ) ); @@ -2582,7 +2586,11 @@ private Iterable> makeSearchResults(String dim, Object return retVal; } - private Iterable> makeSelectResults(Set dimensions, Set metrics, Object... objects) + private Iterable> makeSelectResults( + Set dimensions, + Set metrics, + Object... objects + ) { List> retVal = Lists.newArrayList(); int index = 0; @@ -2598,7 +2606,8 @@ private Iterable> makeSelectResults(Set dimens retVal.add(new Result<>( timestamp, new SelectResultValue(ImmutableMap.of(timestamp.toString(), 0), - dimensions, metrics, values) + dimensions, metrics, values + ) )); } return retVal; @@ -3087,7 +3096,10 @@ public void testIfNoneMatch() TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() .dataSource(DATA_SOURCE) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval))) - .context(ImmutableMap.of("If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=")) + .context(ImmutableMap.of( + "If-None-Match", + "aVJV29CJY93rszVW/QBy0arWZo0=" + )) .build(); @@ -3100,7 +3112,8 @@ public void testIfNoneMatch() @SuppressWarnings("unchecked") private QueryRunner getDefaultQueryRunner() { - return new QueryRunner() { + return new QueryRunner() + { @Override public Sequence run(final QueryPlus queryPlus, final Map responseContext) { From 8f00fda3514b6c1b678b308a93c2ed6fdd88d869 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 20 Jun 2018 18:11:23 -0700 Subject: [PATCH 04/58] Don't use guava `Function --- .../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 2f200bf448bc..808722f6e0a6 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -655,7 +655,7 @@ private Sequence getAndCacheServerResults( .withQuerySegmentSpec(segmentsOfServerSpec), responseContext ); - final com.google.common.base.Function cacheFn = strategy.prepareForSegmentLevelCache(); + final Function cacheFn = strategy.prepareForSegmentLevelCache()::apply; return resultsBySegments .map(result -> { final BySegmentResultValueClass resultsOfSegment = result.getValue(); From c673f512ad58e9a80f007f890b845d9d9956a29b Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 20 Jun 2018 20:48:27 -0700 Subject: [PATCH 05/58] Tests pass! --- .../util/common/guava/MergeSequenceTest.java | 111 ++++++++++++++++-- .../util/common/guava/SequenceTestHelper.java | 41 ++++++- .../druid/client/CachingClusteredClient.java | 43 ++++--- 3 files changed, 163 insertions(+), 32 deletions(-) diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index db299eb0f34d..1121a03dda04 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.List; /** */ @@ -39,13 +40,28 @@ public void testSanity() throws Exception TestSequence.create(2, 8), TestSequence.create(4, 6, 8) ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); + } @Test @@ -57,12 +73,24 @@ public void testWorksWhenBeginningOutOfOrder() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + MergeSequence seq = new MergeSequence<>(Ordering.natural(), Sequences.simple(testSeqs)); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); } @Test @@ -70,17 +98,32 @@ public void testMergeEmpties() throws Exception { final ArrayList> testSeqs = Lists.newArrayList( TestSequence.create(1, 3, 5, 7, 9), - TestSequence.create(), + TestSequence.create(), TestSequence.create(2, 8), TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + Sequences.simple(testSeqs) + ); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); } @Test @@ -93,12 +136,27 @@ public void testMergeEmpties1() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + Sequences.simple(testSeqs) + ); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); } @Test @@ -112,12 +170,27 @@ public void testMergeEmpties2() throws Exception TestSequence.create() ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + Sequences.simple(testSeqs) + ); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); } @Test @@ -129,12 +202,24 @@ public void testScrewsUpOnOutOfOrder() throws Exception TestSequence.create(4, 6) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9)); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9); + + MergeSequence seq = new MergeSequence<>(Ordering.natural(), Sequences.simple(testSeqs)); + SequenceTestHelper.testAll(seq, expected); for (TestSequence sequence : testSeqs) { Assert.assertTrue(sequence.isClosed()); } + + SequenceTestHelper.testAll( + () -> new MergeSequence<>( + Ordering.natural(), + Sequences.simple( + testSeqs.stream() + ) + ), + expected + ); } @Test diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java b/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java index 2e24d2ed69ac..6d83f16c0fe7 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/SequenceTestHelper.java @@ -19,29 +19,47 @@ package io.druid.java.util.common.guava; -import junit.framework.Assert; +import org.junit.Assert; import java.io.IOException; import java.util.Iterator; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; /** */ public class SequenceTestHelper { + public static void testAll(Sequence seq, List nums) throws IOException + { + testAll(() -> seq, nums); + } + + public static void testAll(Supplier> seq, List nums) throws IOException { testAll("", seq, nums); } public static void testAll(String prefix, Sequence seq, List nums) throws IOException + { + testAll(prefix, () -> seq, nums); + } + + public static void testAll(String prefix, Supplier> seq, List nums) throws IOException { testAccumulation(prefix, seq, nums); testYield(prefix, seq, nums); } public static void testYield(final String prefix, Sequence seq, final List nums) throws IOException + { + testYield(prefix, () -> seq, nums); + } + + public static void testYield(final String prefix, Supplier> seq, final List nums) + throws IOException { testYield(prefix, 3, seq, nums); testYield(prefix, 1, seq, nums); @@ -53,9 +71,19 @@ public static void testYield( Sequence seq, final List nums ) throws IOException + { + testYield(prefix, numToTake, () -> seq, nums); + } + + public static void testYield( + final String prefix, + final int numToTake, + Supplier> seq, + final List nums + ) throws IOException { Iterator numsIter = nums.iterator(); - Yielder yielder = seq.toYielder( + Yielder yielder = seq.get().toYielder( 0, new YieldingAccumulator() { @@ -97,15 +125,20 @@ public Integer accumulate(Integer accumulated, Integer in) yielder.close(); } - public static void testAccumulation(final String prefix, Sequence seq, final List nums) + { + testAccumulation(prefix, () -> seq, nums); + } + + public static void testAccumulation(final String prefix, Supplier> seq, final List nums) { int expectedSum = 0; for (Integer num : nums) { expectedSum += num; } - int sum = seq.accumulate( + + int sum = seq.get().accumulate( 0, new Accumulator() { diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 808722f6e0a6..6b90bb4cc01c 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -42,6 +42,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.emitter.EmittingLogger; @@ -290,6 +291,12 @@ Sequence run(final UnaryOperator> time } } + // 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 + // 4. Wrap the whole thing up in a merge final Stream> resultStream = deserializeFromCache( maybeFetchCacheResults( queryCacheKey, @@ -327,6 +334,7 @@ Sequence run(final UnaryOperator> time Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) ).entrySet( ).stream( + // At this point we have the segments per server, and a special entry for the pre-cached stuff ).map( Map.Entry::getValue ).filter( @@ -335,15 +343,6 @@ Sequence run(final UnaryOperator> time l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() ).map( l -> { - final Stream cachedResults = l.stream( - ).map( - ServerMaybeSegmentMaybeCache::getCachedValue - ).filter( - Optional::isPresent - ).map( - Optional::get - ); - final List segmentsOfServer = l.stream( ).map( ServerMaybeSegmentMaybeCache::getSegmentDescriptor @@ -355,8 +354,24 @@ Sequence run(final UnaryOperator> time 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()) { - return Sequences.simple(cachedResults); + // 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 + return new MergeSequence<>(query.getResultOrdering(), Sequences.simple( + l.stream( + ).map( + ServerMaybeSegmentMaybeCache::getCachedValue + ).filter( + Optional::isPresent + ).map( + Optional::get + ).map( + Collections::singletonList + ).map( + Sequences::simple + ) + )); } final DruidServer server = l.get(0).getServer(); @@ -380,11 +395,9 @@ Sequence run(final UnaryOperator> time return serverResults; } ); - final Sequence> resultSeq = Sequences.simple(resultStream); - return resultSeq.flatMerge( - Function.identity(), - query.getResultOrdering() - ); + + // Do the actual merge + return new MergeSequence<>(query.getResultOrdering(), Sequences.simple(resultStream)); } private Stream computeSegmentsToQuery(TimelineLookup timeline) From e86c6905266962629135ddadeee8ad04dd15fa93 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 21 Jun 2018 09:40:33 -0700 Subject: [PATCH 06/58] General cleanup and refactoring for readability --- .../java/util/common/guava/Sequences.java | 2 +- .../druid/client/CachingClusteredClient.java | 262 +++++++++--------- 2 files changed, 138 insertions(+), 126 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 4a0c20181248..d5103d6112e2 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -73,7 +73,7 @@ public Iterator make() @Override public void cleanup(Iterator iterFromMake) { - // stream.close() maybe?? + stream.close(); } } ); diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 6b90bb4cc01c..1eabc1a344b4 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -156,14 +156,11 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data @Override public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { - return new QueryRunner() - { - @Override - public Sequence run(final QueryPlus queryPlus, final Map responseContext) - { - return CachingClusteredClient.this.run(queryPlus, responseContext, timeline -> timeline); - } - }; + return (queryPlus, responseContext) -> CachingClusteredClient.this.run( + queryPlus, + responseContext, + timeline -> timeline + ); } /** @@ -182,31 +179,24 @@ private Sequence run( @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { - return new QueryRunner() - { - @Override - public Sequence run(final QueryPlus queryPlus, final Map responseContext) - { - return CachingClusteredClient.this.run( - queryPlus, - responseContext, - timeline -> { - final VersionedIntervalTimeline timeline2 = - new VersionedIntervalTimeline<>(Ordering.natural()); - for (SegmentDescriptor spec : specs) { - final PartitionHolder entry = timeline.findEntry(spec.getInterval(), spec.getVersion()); - if (entry != null) { - final PartitionChunk chunk = entry.getChunk(spec.getPartitionNumber()); - if (chunk != null) { - timeline2.add(spec.getInterval(), spec.getVersion(), chunk); - } - } + return (queryPlus, responseContext) -> CachingClusteredClient.this.run( + queryPlus, + responseContext, + timeline -> { + final VersionedIntervalTimeline timeline2 = + new VersionedIntervalTimeline<>(Ordering.natural()); + for (SegmentDescriptor spec : specs) { + final PartitionHolder entry = timeline.findEntry(spec.getInterval(), spec.getVersion()); + if (entry != null) { + final PartitionChunk chunk = entry.getChunk(spec.getPartitionNumber()); + if (chunk != null) { + timeline2.add(spec.getInterval(), spec.getVersion(), chunk); } - return timeline2; } - ); - } - }; + } + return timeline2; + } + ); } /** @@ -295,108 +285,22 @@ Sequence run(final UnaryOperator> time // 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 + // 3. Per server (including the ALREADY_CACHED_SERVER) create the appropriate Sequence results - cached results + // are handled in their own merge // 4. Wrap the whole thing up in a merge - final Stream> resultStream = deserializeFromCache( + final Stream>> cacheResolvedResults = deserializeFromCache( maybeFetchCacheResults( queryCacheKey, segments ) + ); + final Stream> resultStream = groupCachedResultsByServer( + cacheResolvedResults ).map( - tuple -> { - final ServerToSegment serverToSegment = tuple.getLhs(); - final Optional maybeResult = tuple.getRhs(); - if (maybeResult.isPresent()) { - return new ServerMaybeSegmentMaybeCache<>(ALREADY_CACHED_SERVER, Optional.empty(), maybeResult); - } - 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( - ALREADY_CACHED_SERVER, - Optional.empty(), - Optional.empty() - ); - } else { - final DruidServer server = queryableDruidServer.getServer(); - return new ServerMaybeSegmentMaybeCache<>( - server, - Optional.ofNullable(serverToSegment.getSegmentDescriptor()), - maybeResult - ); - } - } - ).collect( - Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) - ).entrySet( - ).stream( - // At this point we have the segments per server, and a special entry for the pre-cached stuff - ).map( - Map.Entry::getValue - ).filter( - l -> !l.isEmpty() - ).filter( - l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() - ).map( - l -> { - final List segmentsOfServer = l.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 - return new MergeSequence<>(query.getResultOrdering(), Sequences.simple( - l.stream( - ).map( - ServerMaybeSegmentMaybeCache::getCachedValue - ).filter( - Optional::isPresent - ).map( - Optional::get - ).map( - Collections::singletonList - ).map( - Sequences::simple - ) - )); - } - - final DruidServer server = l.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 MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); - - final Sequence serverResults; - if (isBySegment) { - serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); - } else if (!server.segmentReplicatable() || !populateCache) { - serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec); - } else { - serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); - } - return serverResults; - } + this::runOnServer ); - // Do the actual merge + // Set up the actual merge return new MergeSequence<>(query.getResultOrdering(), Sequences.simple(resultStream)); } @@ -504,6 +408,7 @@ private String computeCurrentEtag(final Iterable segments, @Nul } } + // This materializes the input segment stream in order to let the BulgGet stuff in the cache system work private Stream>> maybeFetchCacheResults( final byte[] queryCacheKey, final Stream segments @@ -590,6 +495,112 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter return cachePopulatorMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } + private Sequence runOnServer(List> segmentOrResult) + { + final List 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 + return new MergeSequence<>(query.getResultOrdering(), Sequences.simple( + segmentOrResult.stream( + ).map( + ServerMaybeSegmentMaybeCache::getCachedValue + ).filter( + Optional::isPresent + ).map( + Optional::get + ).map( + Collections::singletonList + ).map( + Sequences::simple + ) + )); + } + + 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 MultipleSpecificSegmentSpec segmentsOfServerSpec = new MultipleSpecificSegmentSpec(segmentsOfServer); + + final Sequence serverResults; + if (isBySegment) { + serverResults = getBySegmentServerResults(serverRunner, segmentsOfServerSpec); + } else if (!server.segmentReplicatable() || !populateCache) { + serverResults = getSimpleServerResults(serverRunner, segmentsOfServerSpec); + } else { + serverResults = getAndCacheServerResults(serverRunner, segmentsOfServerSpec); + } + return serverResults; + } + + private ServerMaybeSegmentMaybeCache pickServer(SerializablePair> tuple) + { + final Optional maybeResult = tuple.getRhs(); + if (maybeResult.isPresent()) { + return new ServerMaybeSegmentMaybeCache(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( + ALREADY_CACHED_SERVER, + Optional.empty(), + Optional.empty() + ); + } + final DruidServer server = queryableDruidServer.getServer(); + return new ServerMaybeSegmentMaybeCache( + server, + Optional.ofNullable(serverToSegment.getSegmentDescriptor()), + Optional.empty() + ); + } + + // This materializes the input stream in order to group it by server + // This method takes in the stream of cache resolved items and will group all the items by server. + // Each entry in the output stream contains a list whose entries' getServer is the same + // Each entry will either have a present segemnt descriptor or a present result, but not both + // Downstream consumers should check each and handle appropriately. + private Stream>> groupCachedResultsByServer(Stream>> cacheResolvedStream) + { + return cacheResolvedStream.map( + this::pickServer + ).collect( + Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) + ).entrySet( + ).stream( + // At this point we have the segments per server, and a special entry for the pre-cached stuff + ).map( + Map.Entry::getValue + ).filter( + l -> !l.isEmpty() + ).filter( + // Get rid of any alerted conditions missing queryableDruidServer + l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() + ); + } + private Stream>> deserializeFromCache( final Stream>> cachedResults ) @@ -695,6 +706,7 @@ private Sequence getAndCacheServerResults( } } + // POJO private static class ServerMaybeSegmentMaybeCache { private final DruidServer server; From 8a1942ae68424a7ace735d31aa4bf2ca06dbf3a6 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 21 Jun 2018 13:59:04 -0700 Subject: [PATCH 07/58] Streams all around --- .../druid/client/CachingClusteredClient.java | 76 +++++++++++++++---- .../java/io/druid/server/QueryResource.java | 3 + 2 files changed, 64 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 1eabc1a344b4..213e760d863c 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -156,10 +156,12 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data @Override public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { - return (queryPlus, responseContext) -> CachingClusteredClient.this.run( + return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( + query, queryPlus, responseContext, - timeline -> timeline + // No change, but Function.identity() doesn't work here for some reason + stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup ); } @@ -167,7 +169,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different * timeline, if desired. This is used by getQueryRunnerForSegments. */ - private Sequence run( + private Stream> run( final QueryPlus queryPlus, final Map responseContext, final UnaryOperator> timelineConverter @@ -176,21 +178,49 @@ private Sequence run( return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); } + private Sequence runAndMergeWithTimelineChange( + final Query query, + final QueryPlus queryPlus, + final Map responseContext, + final UnaryOperator> timelineConverter + ) + { + return new MergeSequence( + query.getResultOrdering(), + Sequences.simple( + CachingClusteredClient.this.run( + queryPlus, + responseContext, + timelineConverter + ) + ) + ); + } + @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { - return (queryPlus, responseContext) -> CachingClusteredClient.this.run( + return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( + query, queryPlus, responseContext, timeline -> { final VersionedIntervalTimeline timeline2 = new VersionedIntervalTimeline<>(Ordering.natural()); for (SegmentDescriptor spec : specs) { - final PartitionHolder entry = timeline.findEntry(spec.getInterval(), spec.getVersion()); + final PartitionHolder entry = timeline.findEntry( + spec.getInterval(), + spec.getVersion() + ); if (entry != null) { - final PartitionChunk chunk = entry.getChunk(spec.getPartitionNumber()); + final PartitionChunk chunk = entry.getChunk( + spec.getPartitionNumber()); if (chunk != null) { - timeline2.add(spec.getInterval(), spec.getVersion(), chunk); + timeline2.add( + spec.getInterval(), + spec.getVersion(), + chunk + ); } } } @@ -252,12 +282,12 @@ private Map makeDownstreamQueryContext() return Collections.unmodifiableMap(contextBuilder); } - Sequence run(final UnaryOperator> timelineConverter) + Stream> run(final UnaryOperator> timelineConverter) { @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); if (timeline == null) { - return Sequences.empty(); + return Stream.empty(); } timeline = timelineConverter.apply(timeline); if (uncoveredIntervalsLimit > 0) { @@ -277,7 +307,7 @@ Sequence run(final UnaryOperator> time @Nullable final String currentEtag = computeCurrentEtag(materializedSegments, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { - return Sequences.empty(); + return Stream.empty(); } } @@ -294,14 +324,11 @@ Sequence run(final UnaryOperator> time segments ) ); - final Stream> resultStream = groupCachedResultsByServer( + return groupCachedResultsByServer( cacheResolvedResults ).map( this::runOnServer ); - - // Set up the actual merge - return new MergeSequence<>(query.getResultOrdering(), Sequences.simple(resultStream)); } private Stream computeSegmentsToQuery(TimelineLookup timeline) @@ -408,7 +435,15 @@ private String computeCurrentEtag(final Iterable segments, @Nul } } - // This materializes the input segment stream in order to let the BulgGet stuff in the cache system work + /** + * This materializes the input segment stream in order to let the BulgGet stuff in the cache system work + * + * @param queryCacheKey The cache key that is for the query (not-segment) portion + * @param segments The segments to check if they are in cache + * + * @return A stream of the server and segment combinations as well as an optional that is present + * if a cached value was found + */ private Stream>> maybeFetchCacheResults( final byte[] queryCacheKey, final Stream segments @@ -495,6 +530,15 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter return cachePopulatorMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } + /** + * 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. + * + * @return A sequence of either the merged cached results, or the server results from any particular server + */ private Sequence runOnServer(List> segmentOrResult) { final List segmentsOfServer = segmentOrResult.stream( @@ -512,6 +556,8 @@ private Sequence runOnServer(List> segmentOrR 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( diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index ae4931e7aa86..1942e0780394 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -198,6 +198,9 @@ public Response doPost( QueryContexts.isSerializeDateTimeAsLong(query, false) || (!shouldFinalize && QueryContexts.isSerializeDateTimeAsLongInner(query, false)); final ObjectWriter jsonWriter = context.newOutputWriter(serializeDateTimeAsLong); + // Wishlist: https://github.com/FasterXML/jackson-databind/blob/master/src/main/java/com/fasterxml/jackson/databind/ext/jdk8/Jdk8StreamSerializer.java + // https://github.com/FasterXML/jackson-databind/commit/cd32cd5539752cb606c41f09b04d61f170edf323 + // Probably landing in jackson-databind 3.x Response.ResponseBuilder builder = Response .ok( new StreamingOutput() From 4c9be21976580e675f5bc37162c7160f046b49d7 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 22 Jun 2018 10:08:28 -0700 Subject: [PATCH 08/58] PR comments --- .../io/druid/java/util/common/guava/MergeIterator.java | 4 ++-- .../druid/query/select/SelectQueryQueryToolChest.java | 10 ++++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java index f946951a8133..68b974191811 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java @@ -28,7 +28,7 @@ import java.util.PriorityQueue; /** -*/ + */ public class MergeIterator implements Iterator { private final PriorityQueue> pQueue; @@ -40,7 +40,7 @@ public MergeIterator( { pQueue = new PriorityQueue<>( 16, - (lhs, rhs) -> comparator.compare(lhs.peek(), rhs.peek()) + Comparator.comparing(PeekingIterator::peek, comparator) ); for (Iterator iterator : iterators) { 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 bf09b5ea9a80..ccf27ffa770e 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -164,10 +164,12 @@ 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.stream( + ).map( + DimensionSpec::getOutputName + ).collect( + Collectors.toList() + ); @Override public boolean isCacheable(SelectQuery query, boolean willMergeRunners) From 0eece4bbea39f9640b7e8db38aa90526611fd28a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 12:31:05 -0700 Subject: [PATCH 09/58] Add in parallel execution via FJP --- .../java/util/common/concurrent/Execs.java | 29 +++ .../java/util/common/guava/MergeWorkTask.java | 128 +++++++++++++ .../java/util/common/guava/Sequences.java | 5 +- .../util/common/guava/MergeSequenceTest.java | 2 +- .../util/common/guava/MergeWorkTaskTest.java | 175 ++++++++++++++++++ .../io/druid/guice/ForkJoinPoolProvider.java | 53 ++++++ .../io/druid/guice/LifecycleForkJoinPool.java | 58 ++++++ .../java/io/druid/query/QueryContexts.java | 28 +++ .../select/SelectQueryQueryToolChest.java | 1 - .../druid/client/CachingClusteredClient.java | 54 ++++-- ...chingClusteredClientFunctionalityTest.java | 4 +- .../client/CachingClusteredClientTest.java | 147 +++++++++++++-- .../src/main/java/io/druid/cli/CliBroker.java | 13 ++ 13 files changed, 655 insertions(+), 42 deletions(-) create mode 100644 java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java create mode 100644 java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java create mode 100644 processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java create mode 100644 processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 3c5a63755a66..302d36329fdc 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -22,6 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.druid.java.util.common.StringUtils; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; @@ -29,6 +30,8 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinWorkerThread; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ScheduledExecutorService; @@ -36,6 +39,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -147,4 +151,29 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) } ); } + + private static final AtomicLong fjpWorkerThreadCount = new AtomicLong(0L); + + public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool pool) + { + final FJPWorkerThread t = new FJPWorkerThread(pool); + t.setDaemon(true); + t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.getAndIncrement())); + return t; + } + + static class FJPWorkerThread extends ForkJoinWorkerThread + { + /** + * Creates a ForkJoinWorkerThread operating in the given pool. + * + * @param pool the pool this thread works in + * + * @throws NullPointerException if pool is null + */ + FJPWorkerThread(ForkJoinPool pool) + { + super(pool); + } + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java new file mode 100644 index 000000000000..b45eaaef81f7 --- /dev/null +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -0,0 +1,128 @@ +/* + * 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.java.util.common.guava; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Ordering; + +import java.util.ArrayList; +import java.util.Deque; +import java.util.LinkedList; +import java.util.List; +import java.util.Spliterator; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinTask; +import java.util.function.Consumer; +import java.util.stream.Stream; + +public class MergeWorkTask extends ForkJoinTask> +{ + + /** + * Take a stream of sequences, split them as possible, and do intermediate merges. If the input stream is not + * a parallel stream, ignore it. The stream attempts to use groups of {@code batchSize} to do its work, but this + * goal is on a best effort basis. Input streams that cannot be split or are not sized or not subsized might not be + * elligable for this parallelization. The intermediate merges are done in the passed in ForkJoinPool, but the final + * merge is still done when the returned sequence accumulated. + * + * @param ordering The ordering to pass into MergeSequence + * @param baseSequences The sequences that need merged + * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work + * @param fjp The ForkJoinPool to do the intermediate merges in. + * @param The result type + * + * @return A Sequence that will be the merged results of the sub-sequences + */ + public static Sequence parallelMerge( + Ordering ordering, + Stream> baseSequences, + long batchSize, + ForkJoinPool fjp + ) + { + if (!baseSequences.isParallel()) { + // Don't even try. + return new MergeSequence<>(ordering, Sequences.simple(baseSequences)); + } + @SuppressWarnings("unchecked") // Wildcard erasure is fine here + final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); + + final List>> tasks = new ArrayList<>(); + final Deque>> spliteratorStack = new LinkedList<>(); + + // Push the base spliterator onto the stack, keep splitting until we can't or splits are small + spliteratorStack.push(baseSpliterator); + while (!spliteratorStack.isEmpty()) { + + final Spliterator> pop = spliteratorStack.pop(); + if (pop.estimateSize() <= batchSize) { + // Batch is small enough, yay! + tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + continue; + } + + final Spliterator> other = pop.trySplit(); + if (other == null) { + // splits are too big, but we can't split any more + tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + continue; + } + spliteratorStack.push(pop); + spliteratorStack.push(other); + } + return new MergeSequence<>(ordering, Sequences.simple(tasks.stream().map(ForkJoinTask::join))); + } + + private final Ordering ordering; + private final Spliterator> baseSpliterator; + private Sequence result; + + @VisibleForTesting + MergeWorkTask(Ordering ordering, Spliterator> baseSpliterator) + { + this.ordering = ordering; + this.baseSpliterator = baseSpliterator; + } + + @Override + public Sequence getRawResult() + { + return result; + } + + @Override + protected void setRawResult(Sequence value) + { + result = value; + } + + @Override + protected boolean exec() + { + final long estSize = baseSpliterator.estimateSize(); + final List> sequences = new ArrayList<>(estSize > 0 ? (int) estSize : 8); + baseSpliterator.forEachRemaining((Consumer>) sequences::add); + // Force materialization "work" in this thread + // For singleton lists it is not clear it is even worth the optimization of short circuiting the merge for the + // extra code maintenance overhead + result = Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); + return true; + } +} diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index d5103d6112e2..2e437b6f912d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -59,15 +59,16 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence simple(final Stream stream) + public static Sequence simple(final Stream stream) { return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override + @SuppressWarnings("unchecked") public Iterator make() { - return stream.iterator(); + return (Iterator) stream.iterator(); } @Override diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index 1121a03dda04..3fa39a7b91ab 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -21,7 +21,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import junit.framework.Assert; +import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java new file mode 100644 index 000000000000..aad962294f99 --- /dev/null +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java @@ -0,0 +1,175 @@ +/* + * 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.java.util.common.guava; + +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import io.druid.java.util.common.concurrent.Execs; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +public class MergeWorkTaskTest +{ + @Test + public void testNotParallelSequence() throws Exception + { + final ArrayList> testSeqs = Lists.newArrayList( + TestSequence.create(1, 3, 5, 7, 9), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + testSeqs.stream(), + 999, + ForkJoinPool.commonPool() + ), expected); + } + + @Test + public void testOneBatchParallelSequence() throws Exception + { + final ArrayList> testSeqs = Lists.newArrayList( + TestSequence.create(1, 3, 5, 7, 9), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + testSeqs.stream().parallel(), + 999, + ForkJoinPool.commonPool() + ), expected); + } + + @Test + public void testAllBatchParallelSequence() throws Exception + { + final ArrayList> testSeqs = Lists.newArrayList( + TestSequence.create(1, 3, 5, 7, 9), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + testSeqs.stream().parallel(), + 1, + ForkJoinPool.commonPool() + ), expected); + } + + @Test + public void testSomeBatchParallelSequence() throws Exception + { + final ArrayList> testSeqs = Lists.newArrayList( + TestSequence.create(1, 3, 5, 7, 9), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + testSeqs.stream().parallel(), + 2, + ForkJoinPool.commonPool() + ), expected); + } + + + @Test + public void testFJPChoke() throws Exception + { + final ArrayList> testSeqs = Lists.newArrayList( + TestSequence.create(1, 3, 5, 7, 9), + TestSequence.create(2, 8), + TestSequence.create(4, 6, 8) + ); + final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); + final AtomicReference exception = new AtomicReference<>(null); + final ForkJoinPool fjp = new ForkJoinPool( + 1, + pool -> Execs.makeWorkerThread("test-%s", pool), + (t, e) -> exception.set(e), + false + ); + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + testSeqs.stream().parallel(), + 1, + fjp + ), expected); + fjp.shutdown(); + Assert.assertTrue(fjp.awaitTermination(5, TimeUnit.SECONDS)); + Assert.assertNull(exception.get()); + } + + @Test + public void testBigMerge() throws Exception + { + final AtomicReference exception = new AtomicReference<>(null); + final ForkJoinPool fjp = new ForkJoinPool( + 4, + pool -> Execs.makeWorkerThread("test-%s", pool), + (t, e) -> exception.set(e), + false + ); + + // Take a big list of numbers, scatter them among a bunch of different buckets, then make sure the parallel merge + // returns the original list + + final List intList = IntStream.range(0, 10000).boxed().collect(Collectors.toList()); + final List> listList = new ArrayList<>(); + for (int i = 0; i < 500; i++) { + listList.add(new ArrayList<>()); + } + final Random r = new Random(37489165L); + intList.forEach(i -> listList.get(r.nextInt(listList.size())).add(i)); + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + Ordering.natural(), + listList.stream( + ).map( + TestSequence::create + ).parallel(), + 10, + fjp + ), intList); + fjp.shutdown(); + Assert.assertTrue(fjp.awaitTermination(5, TimeUnit.SECONDS)); + Assert.assertNull(exception.get()); + } +} diff --git a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java new file mode 100644 index 000000000000..b47881b997a7 --- /dev/null +++ b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java @@ -0,0 +1,53 @@ +/* + * 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.guice; + +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.logger.Logger; + +import javax.inject.Provider; +import java.util.concurrent.ForkJoinPool; + +public class ForkJoinPoolProvider implements Provider +{ + private static final Logger LOG = new Logger(ForkJoinPoolProvider.class); + + private final String nameFormat; + + public ForkJoinPoolProvider(String nameFormat) + { + // Fail fast on bad name format + StringUtils.format(nameFormat, 3); + this.nameFormat = nameFormat; + } + + @Override + public LifecycleForkJoinPool get() + { + return new LifecycleForkJoinPool( + // This should probably be configurable. Until then, just piggyback off the common pool's parallelism + ForkJoinPool.commonPool().getParallelism(), + pool -> Execs.makeWorkerThread(nameFormat, pool), + (t, e) -> LOG.error(e, "Unhandled exception in thread [%s]", t), + false + ); + } +} diff --git a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java new file mode 100644 index 000000000000..31222c7ba351 --- /dev/null +++ b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java @@ -0,0 +1,58 @@ +/* + * 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.guice; + +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +public class LifecycleForkJoinPool extends ForkJoinPool +{ + private static final Logger LOG = new Logger(LifecycleForkJoinPool.class); + + public LifecycleForkJoinPool( + int parallelism, + ForkJoinWorkerThreadFactory factory, + Thread.UncaughtExceptionHandler handler, + boolean asyncMode + ) + { + super(parallelism, factory, handler, asyncMode); + } + + @LifecycleStop + public void stop() + { + LOG.info("Shutting down ForkJoinPool [%s]", this); + shutdown(); + try { + // Should this be configurable? + if (!awaitTermination(1, TimeUnit.MINUTES)) { + LOG.warn("Failed to complete all tasks in FJP [%s]", this); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("interrupted on shutdown", e); + } + } +} diff --git a/processing/src/main/java/io/druid/query/QueryContexts.java b/processing/src/main/java/io/druid/query/QueryContexts.java index 6a8ae0d5d8f6..8d639f5cb64b 100644 --- a/processing/src/main/java/io/druid/query/QueryContexts.java +++ b/processing/src/main/java/io/druid/query/QueryContexts.java @@ -25,6 +25,8 @@ import io.druid.java.util.common.IAE; import io.druid.java.util.common.Numbers; +import java.util.OptionalLong; + @PublicApi public class QueryContexts { @@ -33,6 +35,7 @@ public class QueryContexts public static final String MAX_SCATTER_GATHER_BYTES_KEY = "maxScatterGatherBytes"; public static final String DEFAULT_TIMEOUT_KEY = "defaultTimeout"; public static final String CHUNK_PERIOD_KEY = "chunkPeriod"; + public static final String INTERMEDIATE_MERGE_BATCH_THRESHOLD = "intermediateMergeBatchThreshold"; public static final boolean DEFAULT_BY_SEGMENT = false; public static final boolean DEFAULT_POPULATE_CACHE = true; @@ -169,6 +172,25 @@ public static Query verifyMaxQueryTimeout(Query query, long maxQueryTi } } + /** + * Return an optional long of the batch size. If the batch is less than 1 (0 or negative) then just return empty + * + * @param query The query whose context is to be used + * @param The query result type + * + * @return An optional long which, if present, will only be a positive long + */ + public static OptionalLong getIntermediateMergeBatchThreshold(Query query) + { + final OptionalLong optionalLong = parseLong(query, INTERMEDIATE_MERGE_BATCH_THRESHOLD); + if (!optionalLong.isPresent()) { + return optionalLong; + } + if (optionalLong.getAsLong() < 1) { + return OptionalLong.empty(); + } + return optionalLong; + } public static long getMaxScatterGatherBytes(Query query) @@ -216,6 +238,12 @@ static long parseLong(Query query, String key, long defaultValue) return val == null ? defaultValue : Numbers.parseLong(val); } + static OptionalLong parseLong(Query query, String key) + { + final Object val = query.getContextValue(key); + return val == null ? OptionalLong.empty() : OptionalLong.of(Numbers.parseLong(val)); + } + static int parseInt(Query query, String key, int defaultValue) { final Object val = query.getContextValue(key); 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 ccf27ffa770e..362a91bec3fa 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -26,7 +26,6 @@ import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 213e760d863c..5ee4bd4b78dd 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.hash.Hasher; @@ -37,12 +38,14 @@ import io.druid.client.selector.ServerSelector; import io.druid.collections.SerializablePair; import io.druid.guice.annotations.BackgroundCaching; +import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.MergeSequence; +import io.druid.java.util.common.guava.MergeWorkTask; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.emitter.EmittingLogger; @@ -82,9 +85,11 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.OptionalLong; import java.util.Spliterators; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; import java.util.function.Function; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -114,6 +119,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final ObjectMapper objectMapper; private final CacheConfig cacheConfig; private final ListeningExecutorService backgroundExecutorService; + private final ForkJoinPool mergeFjp; @Inject public CachingClusteredClient( @@ -122,7 +128,8 @@ public CachingClusteredClient( Cache cache, @Smile ObjectMapper objectMapper, @BackgroundCaching ExecutorService backgroundExecutorService, - CacheConfig cacheConfig + CacheConfig cacheConfig, + @Processing ForkJoinPool mergeFjp ) { this.warehouse = warehouse; @@ -131,7 +138,7 @@ public CachingClusteredClient( this.objectMapper = objectMapper; this.cacheConfig = cacheConfig; this.backgroundExecutorService = MoreExecutors.listeningDecorator(backgroundExecutorService); - + this.mergeFjp = mergeFjp; if (cacheConfig.isQueryCacheable(Query.GROUP_BY)) { log.warn( "Even though groupBy caching is enabled, v2 groupBys will not be cached. " @@ -169,7 +176,8 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different * timeline, if desired. This is used by getQueryRunnerForSegments. */ - private Stream> run( + @VisibleForTesting + Stream> run( final QueryPlus queryPlus, final Map responseContext, final UnaryOperator> timelineConverter @@ -185,16 +193,25 @@ private Sequence runAndMergeWithTimelineChange( final UnaryOperator> timelineConverter ) { - return new MergeSequence( - query.getResultOrdering(), - Sequences.simple( - CachingClusteredClient.this.run( - queryPlus, - responseContext, - timelineConverter - ) - ) + final Stream> sequences = CachingClusteredClient.this.run( + queryPlus, + responseContext, + timelineConverter ); + final OptionalLong mergeBatch = QueryContexts.getIntermediateMergeBatchThreshold(query); + if (mergeBatch.isPresent()) { + return MergeWorkTask.parallelMerge( + query.getResultOrdering(), + sequences, + mergeBatch.getAsLong(), + mergeFjp + ); + } else { + return new MergeSequence<>( + query.getResultOrdering(), + Sequences.simple(sequences) + ); + } } @Override @@ -630,21 +647,24 @@ private ServerMaybeSegmentMaybeCache pickServer(SerializablePair>> groupCachedResultsByServer(Stream>> cacheResolvedStream) { - return cacheResolvedStream.map( + final List>> listList = cacheResolvedStream.map( this::pickServer ).collect( Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) - ).entrySet( + ).values( + // Even though we have matrialized the map, and are iterating across the values, + // the java.util.HashMap.ValueSpliterator does not currently support things like sizing or subsetting, + // so we want to convert the collection into a form conducive with embarassing parallelization down the line ).stream( // At this point we have the segments per server, and a special entry for the pre-cached stuff - ).map( - Map.Entry::getValue ).filter( l -> !l.isEmpty() ).filter( // Get rid of any alerted conditions missing queryableDruidServer l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() - ); + ).collect(Collectors.toList()); + // We do a hard materialization here so that the resulting spliterators have properties that we want + return listList.stream(); } private Stream>> deserializeFromCache( diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 390bde55996b..6e86db0a6a23 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -58,6 +58,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; /** */ @@ -272,7 +273,8 @@ public int getCacheBulkMergeLimit() { return mergeLimit; } - } + }, + ForkJoinPool.commonPool() ); } diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index a9a2fa138b3b..35f6289e8940 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -65,6 +65,7 @@ import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; +import io.druid.java.util.common.guava.MergeWorkTask; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.TrinaryFn; @@ -159,10 +160,12 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.Spliterator; import java.util.TreeMap; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.stream.Stream; /** @@ -170,7 +173,7 @@ @RunWith(Parameterized.class) public class CachingClusteredClientTest { - public static final ImmutableMap CONTEXT = ImmutableMap.of( + public static final ImmutableMap CONTEXT = ImmutableMap.of( "finalize", false, // GroupBy v2 won't cache on the broker, so test with v1. @@ -316,14 +319,7 @@ public static Iterable constructorFeeder() { return Lists.transform( Lists.newArrayList(new RangeIterable(RANDOMNESS)), - new Function() - { - @Override - public Object[] apply(Integer input) - { - return new Object[]{input}; - } - } + input -> new Object[]{input} ); } @@ -2027,8 +2023,8 @@ public void testQueryCaching( .andReturn(expectations.getQueryRunner()) .once(); - final Capture capture = new Capture(); - final Capture context = new Capture(); + final Capture capture = EasyMock.newCapture(); + final Capture context = EasyMock.newCapture(); queryCaptures.add(capture); QueryRunner queryable = expectations.getQueryRunner(); @@ -2713,7 +2709,8 @@ public int getCacheBulkMergeLimit() { return mergeLimit; } - } + }, + ForkJoinPool.commonPool() ); } @@ -3112,14 +3109,124 @@ public void testIfNoneMatch() @SuppressWarnings("unchecked") private QueryRunner getDefaultQueryRunner() { - return new QueryRunner() + return (queryPlus, responseContext) -> client + .getQueryRunnerForIntervals(queryPlus.getQuery(), queryPlus.getQuery().getIntervals()) + .run(queryPlus, responseContext); + } + + @Test + public void testSpliterator() + { { - @Override - public Sequence run(final QueryPlus queryPlus, final Map responseContext) - { - return client.getQueryRunnerForIntervals(queryPlus.getQuery(), queryPlus.getQuery().getIntervals()) - .run(queryPlus, responseContext); - } - }; + // populate cache selectively + final Druids.TimeseriesQueryBuilder builder = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + QueryRunner runner = new FinalizeResultsQueryRunner( + getDefaultQueryRunner(), + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ) + ); + + testQueryCaching( + runner, + 1, + true, + builder.build(), + Intervals.of("2011-01-01/2011-01-02"), makeTimeResults(DateTimes.of("2011-01-01"), 50, 5000), + Intervals.of("2011-01-02/2011-01-03"), makeTimeResults(DateTimes.of("2011-01-02"), 30, 6000), + Intervals.of("2011-01-04/2011-01-05"), makeTimeResults(DateTimes.of("2011-01-04"), 23, 85312), + + Intervals.of("2011-01-05/2011-01-10"), + makeTimeResults( + DateTimes.of("2011-01-05"), 85, 102, + DateTimes.of("2011-01-06"), 412, 521, + DateTimes.of("2011-01-07"), 122, 21894, + DateTimes.of("2011-01-08"), 5, 20, + DateTimes.of("2011-01-09"), 18, 521 + ), + + Intervals.of("2011-01-05/2011-01-10"), + makeTimeResults( + DateTimes.of("2011-01-05T01"), 80, 100, + DateTimes.of("2011-01-06T01"), 420, 520, + DateTimes.of("2011-01-07T01"), 12, 2194, + DateTimes.of("2011-01-08T01"), 59, 201, + DateTimes.of("2011-01-09T01"), 181, 52 + ) + ); + } + final Druids.TimeseriesQueryBuilder builder = Druids + .newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(SEG_SPEC) + .filters(DIM_FILTER) + .granularity(GRANULARITY) + .aggregators(AGGS) + .postAggregators(POST_AGGS) + .context(CONTEXT); + + + final HashMap context = new HashMap<>(); + final TimeseriesQuery query = builder + .intervals("2011-01-01/2011-01-10") + .aggregators(RENAMED_AGGS) + .postAggregators(RENAMED_POST_AGGS) + .build() + .withOverriddenContext(Collections.singletonMap("populateCache", "false")); + + final Stream>> results = client.run( + QueryPlus.wrap(query), + context, + stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup + ); + + final Spliterator>> spliterator = results.spliterator(); + + Assert.assertNotNull(spliterator); + final int characteristics = spliterator.characteristics(); + Assert.assertEquals(characteristics & Spliterator.SIZED, Spliterator.SIZED); + Assert.assertEquals(characteristics & Spliterator.SUBSIZED, Spliterator.SUBSIZED); + final ArrayList>> sequences = new ArrayList<>(); + spliterator.forEachRemaining(sequences::add); + Assert.assertFalse(sequences.isEmpty()); + + + final Sequence> parallelMergeResults = MergeWorkTask.parallelMerge( + query.getResultOrdering(), + client.run( + QueryPlus.wrap(query), + context, + stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup + ).parallel(), + 1, + ForkJoinPool.commonPool() + ); + + TestHelper.assertExpectedResults( + makeRenamedTimeResults( + DateTimes.of("2011-01-01"), 50, 5000, + DateTimes.of("2011-01-02"), 30, 6000, + DateTimes.of("2011-01-04"), 23, 85312, + DateTimes.of("2011-01-05"), 85, 102, + DateTimes.of("2011-01-05T01"), 80, 100, + DateTimes.of("2011-01-06"), 412, 521, + DateTimes.of("2011-01-06T01"), 420, 520, + DateTimes.of("2011-01-07"), 122, 21894, + DateTimes.of("2011-01-07T01"), 12, 2194, + DateTimes.of("2011-01-08"), 5, 20, + DateTimes.of("2011-01-08T01"), 59, 201, + DateTimes.of("2011-01-09"), 18, 521, + DateTimes.of("2011-01-09T01"), 181, 52 + ), + parallelMergeResults + ); } } diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index e809ba3d6639..54356f7d3f67 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -39,12 +39,16 @@ import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; +import io.druid.guice.ForkJoinPoolProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleForkJoinPool; import io.druid.guice.LifecycleModule; +import io.druid.guice.ManageLifecycle; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; +import io.druid.guice.annotations.Processing; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -63,6 +67,7 @@ import org.eclipse.jetty.server.Server; import java.util.List; +import java.util.concurrent.ForkJoinPool; /** */ @@ -101,6 +106,14 @@ public void configure(Binder binder) binder.bind(CachingClusteredClient.class).in(LazySingleton.class); binder.bind(BrokerServerView.class).in(LazySingleton.class); binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); + binder.bind(Key.get(LifecycleForkJoinPool.class, Processing.class)) + .toProvider(new ForkJoinPoolProvider("processing-fjp-%s")) + .in(ManageLifecycle.class); + // Bind the lifecycle key, then bind the lifecycle to the forkjoinpool key so that any extensions that + // want to have their own fork join pool instead of this one can do so. + LifecycleModule.register(binder, LifecycleForkJoinPool.class, Processing.class); + binder.bind(Key.get(ForkJoinPool.class, Processing.class)) + .to(Key.get(LifecycleForkJoinPool.class, Processing.class)); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheConfig.class); binder.install(new CacheModule()); From f80a5bc137d759ce894b467ac33359f9fa5f7c4d Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 15:01:00 -0700 Subject: [PATCH 10/58] Move to completion service... but risk deadlocking --- ...geWorkTask.java => MergeWorkCallable.java} | 102 ++++++++++++------ ...skTest.java => MergeWorkCallableTest.java} | 14 +-- .../io/druid/guice/ForkJoinPoolProvider.java | 53 --------- .../io/druid/guice/LifecycleForkJoinPool.java | 58 ---------- .../druid/client/CachingClusteredClient.java | 12 +-- .../client/CachingClusteredClientTest.java | 4 +- .../src/main/java/io/druid/cli/CliBroker.java | 13 --- 7 files changed, 87 insertions(+), 169 deletions(-) rename java-util/src/main/java/io/druid/java/util/common/guava/{MergeWorkTask.java => MergeWorkCallable.java} (58%) rename java-util/src/test/java/io/druid/java/util/common/guava/{MergeWorkTaskTest.java => MergeWorkCallableTest.java} (91%) delete mode 100644 processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java delete mode 100644 processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java similarity index 58% rename from java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java rename to java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java index b45eaaef81f7..dae06c667a09 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java @@ -19,22 +19,27 @@ package io.druid.java.util.common.guava; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Ordering; import java.util.ArrayList; +import java.util.Collection; import java.util.Deque; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.NoSuchElementException; import java.util.Spliterator; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.ForkJoinTask; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; import java.util.function.Consumer; import java.util.stream.Stream; -public class MergeWorkTask extends ForkJoinTask> +public class MergeWorkCallable implements Callable> { - /** * Take a stream of sequences, split them as possible, and do intermediate merges. If the input stream is not * a parallel stream, ignore it. The stream attempts to use groups of {@code batchSize} to do its work, but this @@ -45,7 +50,7 @@ public class MergeWorkTask extends ForkJoinTask> * @param ordering The ordering to pass into MergeSequence * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work - * @param fjp The ForkJoinPool to do the intermediate merges in. + * @param mergeExecutor The ExecutorService to do the intermediate merges in. * @param The result type * * @return A Sequence that will be the merged results of the sub-sequences @@ -54,7 +59,7 @@ public static Sequence parallelMerge( Ordering ordering, Stream> baseSequences, long batchSize, - ForkJoinPool fjp + ExecutorService mergeExecutor ) { if (!baseSequences.isParallel()) { @@ -64,57 +69,95 @@ public static Sequence parallelMerge( @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); - final List>> tasks = new ArrayList<>(); final Deque>> spliteratorStack = new LinkedList<>(); + final CompletionService> completionService = new ExecutorCompletionService<>(mergeExecutor); + final Collection allFutures = new ArrayList<>(); + + long additions = 0; + // Push the base spliterator onto the stack, keep splitting until we can't or splits are small spliteratorStack.push(baseSpliterator); while (!spliteratorStack.isEmpty()) { - final Spliterator> pop = spliteratorStack.pop(); if (pop.estimateSize() <= batchSize) { // Batch is small enough, yay! - tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + additions++; + allFutures.add(completionService.submit(new MergeWorkCallable(ordering, pop))); continue; } final Spliterator> other = pop.trySplit(); if (other == null) { // splits are too big, but we can't split any more - tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + additions++; + allFutures.add(completionService.submit(new MergeWorkCallable<>(ordering, pop))); continue; } spliteratorStack.push(pop); spliteratorStack.push(other); } - return new MergeSequence<>(ordering, Sequences.simple(tasks.stream().map(ForkJoinTask::join))); + final long totalAdditions = additions; + return new MergeSequence<>( + ordering, + new BaseSequence<>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return new Iterator>() + { + long taken = 0L; + + @Override + public boolean hasNext() + { + return taken < totalAdditions; + } + + @Override + public Sequence next() throws NoSuchElementException + { + if (taken >= totalAdditions) { + throw new NoSuchElementException(); + } + try { + taken++; + return completionService.take().get(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted waiting for intermediate merge", e); + } + catch (ExecutionException e) { + throw new RuntimeException("Failed during intermediate merge", e); + } + } + }; + } + + @Override + public void cleanup(Iterator> iterFromMake) + { + allFutures.forEach(f -> f.cancel(true)); + } + } + ) + ); } private final Ordering ordering; private final Spliterator> baseSpliterator; - private Sequence result; - @VisibleForTesting - MergeWorkTask(Ordering ordering, Spliterator> baseSpliterator) + private MergeWorkCallable(Ordering ordering, Spliterator> baseSpliterator) { this.ordering = ordering; this.baseSpliterator = baseSpliterator; } @Override - public Sequence getRawResult() - { - return result; - } - - @Override - protected void setRawResult(Sequence value) - { - result = value; - } - - @Override - protected boolean exec() + public Sequence call() { final long estSize = baseSpliterator.estimateSize(); final List> sequences = new ArrayList<>(estSize > 0 ? (int) estSize : 8); @@ -122,7 +165,6 @@ protected boolean exec() // Force materialization "work" in this thread // For singleton lists it is not clear it is even worth the optimization of short circuiting the merge for the // extra code maintenance overhead - result = Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); - return true; + return Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); } } diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java similarity index 91% rename from java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java rename to java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java index aad962294f99..82b57122c366 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java @@ -36,7 +36,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -public class MergeWorkTaskTest +public class MergeWorkCallableTest { @Test public void testNotParallelSequence() throws Exception @@ -48,7 +48,7 @@ public void testNotParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), testSeqs.stream(), 999, @@ -66,7 +66,7 @@ public void testOneBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 999, @@ -84,7 +84,7 @@ public void testAllBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 1, @@ -102,7 +102,7 @@ public void testSomeBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 2, @@ -127,7 +127,7 @@ public void testFJPChoke() throws Exception (t, e) -> exception.set(e), false ); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 1, @@ -159,7 +159,7 @@ public void testBigMerge() throws Exception } final Random r = new Random(37489165L); intList.forEach(i -> listList.get(r.nextInt(listList.size())).add(i)); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( Ordering.natural(), listList.stream( ).map( diff --git a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java deleted file mode 100644 index b47881b997a7..000000000000 --- a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java +++ /dev/null @@ -1,53 +0,0 @@ -/* - * 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.guice; - -import io.druid.java.util.common.StringUtils; -import io.druid.java.util.common.concurrent.Execs; -import io.druid.java.util.common.logger.Logger; - -import javax.inject.Provider; -import java.util.concurrent.ForkJoinPool; - -public class ForkJoinPoolProvider implements Provider -{ - private static final Logger LOG = new Logger(ForkJoinPoolProvider.class); - - private final String nameFormat; - - public ForkJoinPoolProvider(String nameFormat) - { - // Fail fast on bad name format - StringUtils.format(nameFormat, 3); - this.nameFormat = nameFormat; - } - - @Override - public LifecycleForkJoinPool get() - { - return new LifecycleForkJoinPool( - // This should probably be configurable. Until then, just piggyback off the common pool's parallelism - ForkJoinPool.commonPool().getParallelism(), - pool -> Execs.makeWorkerThread(nameFormat, pool), - (t, e) -> LOG.error(e, "Unhandled exception in thread [%s]", t), - false - ); - } -} diff --git a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java deleted file mode 100644 index 31222c7ba351..000000000000 --- a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.guice; - -import io.druid.java.util.common.lifecycle.LifecycleStop; -import io.druid.java.util.common.logger.Logger; - -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.TimeUnit; - -public class LifecycleForkJoinPool extends ForkJoinPool -{ - private static final Logger LOG = new Logger(LifecycleForkJoinPool.class); - - public LifecycleForkJoinPool( - int parallelism, - ForkJoinWorkerThreadFactory factory, - Thread.UncaughtExceptionHandler handler, - boolean asyncMode - ) - { - super(parallelism, factory, handler, asyncMode); - } - - @LifecycleStop - public void stop() - { - LOG.info("Shutting down ForkJoinPool [%s]", this); - shutdown(); - try { - // Should this be configurable? - if (!awaitTermination(1, TimeUnit.MINUTES)) { - LOG.warn("Failed to complete all tasks in FJP [%s]", this); - } - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("interrupted on shutdown", e); - } - } -} diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 5ee4bd4b78dd..88a338334e28 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -45,7 +45,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.MergeSequence; -import io.druid.java.util.common.guava.MergeWorkTask; +import io.druid.java.util.common.guava.MergeWorkCallable; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.emitter.EmittingLogger; @@ -119,7 +119,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final ObjectMapper objectMapper; private final CacheConfig cacheConfig; private final ListeningExecutorService backgroundExecutorService; - private final ForkJoinPool mergeFjp; + private final ExecutorService mergeExecutor; @Inject public CachingClusteredClient( @@ -129,7 +129,7 @@ public CachingClusteredClient( @Smile ObjectMapper objectMapper, @BackgroundCaching ExecutorService backgroundExecutorService, CacheConfig cacheConfig, - @Processing ForkJoinPool mergeFjp + @Processing ExecutorService mergeExecutor ) { this.warehouse = warehouse; @@ -138,7 +138,7 @@ public CachingClusteredClient( this.objectMapper = objectMapper; this.cacheConfig = cacheConfig; this.backgroundExecutorService = MoreExecutors.listeningDecorator(backgroundExecutorService); - this.mergeFjp = mergeFjp; + this.mergeExecutor = mergeExecutor; if (cacheConfig.isQueryCacheable(Query.GROUP_BY)) { log.warn( "Even though groupBy caching is enabled, v2 groupBys will not be cached. " @@ -200,11 +200,11 @@ private Sequence runAndMergeWithTimelineChange( ); final OptionalLong mergeBatch = QueryContexts.getIntermediateMergeBatchThreshold(query); if (mergeBatch.isPresent()) { - return MergeWorkTask.parallelMerge( + return MergeWorkCallable.parallelMerge( query.getResultOrdering(), sequences, mergeBatch.getAsLong(), - mergeFjp + mergeExecutor ); } else { return new MergeSequence<>( diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 35f6289e8940..c07b0ef2eade 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -65,7 +65,7 @@ import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; -import io.druid.java.util.common.guava.MergeWorkTask; +import io.druid.java.util.common.guava.MergeWorkCallable; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.TrinaryFn; @@ -3199,7 +3199,7 @@ public void testSpliterator() Assert.assertFalse(sequences.isEmpty()); - final Sequence> parallelMergeResults = MergeWorkTask.parallelMerge( + final Sequence> parallelMergeResults = MergeWorkCallable.parallelMerge( query.getResultOrdering(), client.run( QueryPlus.wrap(query), diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 54356f7d3f67..e809ba3d6639 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -39,16 +39,12 @@ import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; -import io.druid.guice.ForkJoinPoolProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; -import io.druid.guice.LifecycleForkJoinPool; import io.druid.guice.LifecycleModule; -import io.druid.guice.ManageLifecycle; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; -import io.druid.guice.annotations.Processing; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -67,7 +63,6 @@ import org.eclipse.jetty.server.Server; import java.util.List; -import java.util.concurrent.ForkJoinPool; /** */ @@ -106,14 +101,6 @@ public void configure(Binder binder) binder.bind(CachingClusteredClient.class).in(LazySingleton.class); binder.bind(BrokerServerView.class).in(LazySingleton.class); binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); - binder.bind(Key.get(LifecycleForkJoinPool.class, Processing.class)) - .toProvider(new ForkJoinPoolProvider("processing-fjp-%s")) - .in(ManageLifecycle.class); - // Bind the lifecycle key, then bind the lifecycle to the forkjoinpool key so that any extensions that - // want to have their own fork join pool instead of this one can do so. - LifecycleModule.register(binder, LifecycleForkJoinPool.class, Processing.class); - binder.bind(Key.get(ForkJoinPool.class, Processing.class)) - .to(Key.get(LifecycleForkJoinPool.class, Processing.class)); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheConfig.class); binder.install(new CacheModule()); From 0bdaed2ddb3031d4f114d65c21ce746e485fcfb3 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 15:24:35 -0700 Subject: [PATCH 11/58] Revert "Move to completion service... but risk deadlocking" This reverts commit f80a5bc137d759ce894b467ac33359f9fa5f7c4d. --- ...geWorkCallable.java => MergeWorkTask.java} | 102 ++++++------------ ...llableTest.java => MergeWorkTaskTest.java} | 14 +-- .../io/druid/guice/ForkJoinPoolProvider.java | 53 +++++++++ .../io/druid/guice/LifecycleForkJoinPool.java | 58 ++++++++++ .../druid/client/CachingClusteredClient.java | 12 +-- .../client/CachingClusteredClientTest.java | 4 +- .../src/main/java/io/druid/cli/CliBroker.java | 13 +++ 7 files changed, 169 insertions(+), 87 deletions(-) rename java-util/src/main/java/io/druid/java/util/common/guava/{MergeWorkCallable.java => MergeWorkTask.java} (58%) rename java-util/src/test/java/io/druid/java/util/common/guava/{MergeWorkCallableTest.java => MergeWorkTaskTest.java} (91%) create mode 100644 processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java create mode 100644 processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java similarity index 58% rename from java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java rename to java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index dae06c667a09..b45eaaef81f7 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkCallable.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -19,27 +19,22 @@ package io.druid.java.util.common.guava; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Ordering; import java.util.ArrayList; -import java.util.Collection; import java.util.Deque; -import java.util.Iterator; import java.util.LinkedList; import java.util.List; -import java.util.NoSuchElementException; import java.util.Spliterator; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletionService; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.ForkJoinTask; import java.util.function.Consumer; import java.util.stream.Stream; -public class MergeWorkCallable implements Callable> +public class MergeWorkTask extends ForkJoinTask> { + /** * Take a stream of sequences, split them as possible, and do intermediate merges. If the input stream is not * a parallel stream, ignore it. The stream attempts to use groups of {@code batchSize} to do its work, but this @@ -50,7 +45,7 @@ public class MergeWorkCallable implements Callable> * @param ordering The ordering to pass into MergeSequence * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work - * @param mergeExecutor The ExecutorService to do the intermediate merges in. + * @param fjp The ForkJoinPool to do the intermediate merges in. * @param The result type * * @return A Sequence that will be the merged results of the sub-sequences @@ -59,7 +54,7 @@ public static Sequence parallelMerge( Ordering ordering, Stream> baseSequences, long batchSize, - ExecutorService mergeExecutor + ForkJoinPool fjp ) { if (!baseSequences.isParallel()) { @@ -69,95 +64,57 @@ public static Sequence parallelMerge( @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); + final List>> tasks = new ArrayList<>(); final Deque>> spliteratorStack = new LinkedList<>(); - final CompletionService> completionService = new ExecutorCompletionService<>(mergeExecutor); - final Collection allFutures = new ArrayList<>(); - - long additions = 0; - // Push the base spliterator onto the stack, keep splitting until we can't or splits are small spliteratorStack.push(baseSpliterator); while (!spliteratorStack.isEmpty()) { + final Spliterator> pop = spliteratorStack.pop(); if (pop.estimateSize() <= batchSize) { // Batch is small enough, yay! - additions++; - allFutures.add(completionService.submit(new MergeWorkCallable(ordering, pop))); + tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); continue; } final Spliterator> other = pop.trySplit(); if (other == null) { // splits are too big, but we can't split any more - additions++; - allFutures.add(completionService.submit(new MergeWorkCallable<>(ordering, pop))); + tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); continue; } spliteratorStack.push(pop); spliteratorStack.push(other); } - final long totalAdditions = additions; - return new MergeSequence<>( - ordering, - new BaseSequence<>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return new Iterator>() - { - long taken = 0L; - - @Override - public boolean hasNext() - { - return taken < totalAdditions; - } - - @Override - public Sequence next() throws NoSuchElementException - { - if (taken >= totalAdditions) { - throw new NoSuchElementException(); - } - try { - taken++; - return completionService.take().get(); - } - catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted waiting for intermediate merge", e); - } - catch (ExecutionException e) { - throw new RuntimeException("Failed during intermediate merge", e); - } - } - }; - } - - @Override - public void cleanup(Iterator> iterFromMake) - { - allFutures.forEach(f -> f.cancel(true)); - } - } - ) - ); + return new MergeSequence<>(ordering, Sequences.simple(tasks.stream().map(ForkJoinTask::join))); } private final Ordering ordering; private final Spliterator> baseSpliterator; + private Sequence result; - private MergeWorkCallable(Ordering ordering, Spliterator> baseSpliterator) + @VisibleForTesting + MergeWorkTask(Ordering ordering, Spliterator> baseSpliterator) { this.ordering = ordering; this.baseSpliterator = baseSpliterator; } @Override - public Sequence call() + public Sequence getRawResult() + { + return result; + } + + @Override + protected void setRawResult(Sequence value) + { + result = value; + } + + @Override + protected boolean exec() { final long estSize = baseSpliterator.estimateSize(); final List> sequences = new ArrayList<>(estSize > 0 ? (int) estSize : 8); @@ -165,6 +122,7 @@ public Sequence call() // Force materialization "work" in this thread // For singleton lists it is not clear it is even worth the optimization of short circuiting the merge for the // extra code maintenance overhead - return Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); + result = Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); + return true; } } diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java similarity index 91% rename from java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java rename to java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java index 82b57122c366..aad962294f99 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkCallableTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java @@ -36,7 +36,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; -public class MergeWorkCallableTest +public class MergeWorkTaskTest { @Test public void testNotParallelSequence() throws Exception @@ -48,7 +48,7 @@ public void testNotParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), testSeqs.stream(), 999, @@ -66,7 +66,7 @@ public void testOneBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 999, @@ -84,7 +84,7 @@ public void testAllBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 1, @@ -102,7 +102,7 @@ public void testSomeBatchParallelSequence() throws Exception ); final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 2, @@ -127,7 +127,7 @@ public void testFJPChoke() throws Exception (t, e) -> exception.set(e), false ); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), testSeqs.stream().parallel(), 1, @@ -159,7 +159,7 @@ public void testBigMerge() throws Exception } final Random r = new Random(37489165L); intList.forEach(i -> listList.get(r.nextInt(listList.size())).add(i)); - SequenceTestHelper.testAll(() -> MergeWorkCallable.parallelMerge( + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( Ordering.natural(), listList.stream( ).map( diff --git a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java new file mode 100644 index 000000000000..b47881b997a7 --- /dev/null +++ b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java @@ -0,0 +1,53 @@ +/* + * 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.guice; + +import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; +import io.druid.java.util.common.logger.Logger; + +import javax.inject.Provider; +import java.util.concurrent.ForkJoinPool; + +public class ForkJoinPoolProvider implements Provider +{ + private static final Logger LOG = new Logger(ForkJoinPoolProvider.class); + + private final String nameFormat; + + public ForkJoinPoolProvider(String nameFormat) + { + // Fail fast on bad name format + StringUtils.format(nameFormat, 3); + this.nameFormat = nameFormat; + } + + @Override + public LifecycleForkJoinPool get() + { + return new LifecycleForkJoinPool( + // This should probably be configurable. Until then, just piggyback off the common pool's parallelism + ForkJoinPool.commonPool().getParallelism(), + pool -> Execs.makeWorkerThread(nameFormat, pool), + (t, e) -> LOG.error(e, "Unhandled exception in thread [%s]", t), + false + ); + } +} diff --git a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java new file mode 100644 index 000000000000..31222c7ba351 --- /dev/null +++ b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java @@ -0,0 +1,58 @@ +/* + * 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.guice; + +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; + +import java.util.concurrent.ForkJoinPool; +import java.util.concurrent.TimeUnit; + +public class LifecycleForkJoinPool extends ForkJoinPool +{ + private static final Logger LOG = new Logger(LifecycleForkJoinPool.class); + + public LifecycleForkJoinPool( + int parallelism, + ForkJoinWorkerThreadFactory factory, + Thread.UncaughtExceptionHandler handler, + boolean asyncMode + ) + { + super(parallelism, factory, handler, asyncMode); + } + + @LifecycleStop + public void stop() + { + LOG.info("Shutting down ForkJoinPool [%s]", this); + shutdown(); + try { + // Should this be configurable? + if (!awaitTermination(1, TimeUnit.MINUTES)) { + LOG.warn("Failed to complete all tasks in FJP [%s]", this); + } + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("interrupted on shutdown", e); + } + } +} diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 88a338334e28..5ee4bd4b78dd 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -45,7 +45,7 @@ import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.MergeSequence; -import io.druid.java.util.common.guava.MergeWorkCallable; +import io.druid.java.util.common.guava.MergeWorkTask; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.emitter.EmittingLogger; @@ -119,7 +119,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final ObjectMapper objectMapper; private final CacheConfig cacheConfig; private final ListeningExecutorService backgroundExecutorService; - private final ExecutorService mergeExecutor; + private final ForkJoinPool mergeFjp; @Inject public CachingClusteredClient( @@ -129,7 +129,7 @@ public CachingClusteredClient( @Smile ObjectMapper objectMapper, @BackgroundCaching ExecutorService backgroundExecutorService, CacheConfig cacheConfig, - @Processing ExecutorService mergeExecutor + @Processing ForkJoinPool mergeFjp ) { this.warehouse = warehouse; @@ -138,7 +138,7 @@ public CachingClusteredClient( this.objectMapper = objectMapper; this.cacheConfig = cacheConfig; this.backgroundExecutorService = MoreExecutors.listeningDecorator(backgroundExecutorService); - this.mergeExecutor = mergeExecutor; + this.mergeFjp = mergeFjp; if (cacheConfig.isQueryCacheable(Query.GROUP_BY)) { log.warn( "Even though groupBy caching is enabled, v2 groupBys will not be cached. " @@ -200,11 +200,11 @@ private Sequence runAndMergeWithTimelineChange( ); final OptionalLong mergeBatch = QueryContexts.getIntermediateMergeBatchThreshold(query); if (mergeBatch.isPresent()) { - return MergeWorkCallable.parallelMerge( + return MergeWorkTask.parallelMerge( query.getResultOrdering(), sequences, mergeBatch.getAsLong(), - mergeExecutor + mergeFjp ); } else { return new MergeSequence<>( diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index c07b0ef2eade..35f6289e8940 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -65,7 +65,7 @@ import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; -import io.druid.java.util.common.guava.MergeWorkCallable; +import io.druid.java.util.common.guava.MergeWorkTask; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.TrinaryFn; @@ -3199,7 +3199,7 @@ public void testSpliterator() Assert.assertFalse(sequences.isEmpty()); - final Sequence> parallelMergeResults = MergeWorkCallable.parallelMerge( + final Sequence> parallelMergeResults = MergeWorkTask.parallelMerge( query.getResultOrdering(), client.run( QueryPlus.wrap(query), diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index e809ba3d6639..54356f7d3f67 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -39,12 +39,16 @@ import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; +import io.druid.guice.ForkJoinPoolProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; +import io.druid.guice.LifecycleForkJoinPool; import io.druid.guice.LifecycleModule; +import io.druid.guice.ManageLifecycle; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; +import io.druid.guice.annotations.Processing; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -63,6 +67,7 @@ import org.eclipse.jetty.server.Server; import java.util.List; +import java.util.concurrent.ForkJoinPool; /** */ @@ -101,6 +106,14 @@ public void configure(Binder binder) binder.bind(CachingClusteredClient.class).in(LazySingleton.class); binder.bind(BrokerServerView.class).in(LazySingleton.class); binder.bind(TimelineServerView.class).to(BrokerServerView.class).in(LazySingleton.class); + binder.bind(Key.get(LifecycleForkJoinPool.class, Processing.class)) + .toProvider(new ForkJoinPoolProvider("processing-fjp-%s")) + .in(ManageLifecycle.class); + // Bind the lifecycle key, then bind the lifecycle to the forkjoinpool key so that any extensions that + // want to have their own fork join pool instead of this one can do so. + LifecycleModule.register(binder, LifecycleForkJoinPool.class, Processing.class); + binder.bind(Key.get(ForkJoinPool.class, Processing.class)) + .to(Key.get(LifecycleForkJoinPool.class, Processing.class)); JsonConfigProvider.bind(binder, "druid.broker.cache", CacheConfig.class); binder.install(new CacheModule()); From 1a6b3b61c2c394b0995f7212e79066b9de32fb50 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 16:22:57 -0700 Subject: [PATCH 12/58] Actually make the stream parallel --- .../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 5ee4bd4b78dd..94e957c39c1e 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -202,7 +202,7 @@ private Sequence runAndMergeWithTimelineChange( if (mergeBatch.isPresent()) { return MergeWorkTask.parallelMerge( query.getResultOrdering(), - sequences, + sequences.parallel(), mergeBatch.getAsLong(), mergeFjp ); From e86227b55e4c433259a142579558670b4b465813 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 17:53:42 -0700 Subject: [PATCH 13/58] Make the returned merge sequence fetch intermediate results asap --- .../java/util/common/guava/MergeWorkTask.java | 62 ++++++++++++++++++- 1 file changed, 59 insertions(+), 3 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index b45eaaef81f7..ddb08e4866e6 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -24,9 +24,13 @@ import java.util.ArrayList; import java.util.Deque; +import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.NoSuchElementException; import java.util.Spliterator; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinTask; import java.util.function.Consumer; @@ -37,10 +41,11 @@ public class MergeWorkTask extends ForkJoinTask> /** * Take a stream of sequences, split them as possible, and do intermediate merges. If the input stream is not - * a parallel stream, ignore it. The stream attempts to use groups of {@code batchSize} to do its work, but this + * a parallel stream, do a traditional merge. The stream attempts to use groups of {@code batchSize} to do its work, but this * goal is on a best effort basis. Input streams that cannot be split or are not sized or not subsized might not be * elligable for this parallelization. The intermediate merges are done in the passed in ForkJoinPool, but the final - * merge is still done when the returned sequence accumulated. + * merge is still done when the returned sequence accumulated. The intermediate merges are yielded in the order + * in which they are ready. * * @param ordering The ordering to pass into MergeSequence * @param baseSequences The sequences that need merged @@ -64,6 +69,7 @@ public static Sequence parallelMerge( @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); + // Accumulate a list of forked off tasks final List>> tasks = new ArrayList<>(); final Deque>> spliteratorStack = new LinkedList<>(); @@ -87,7 +93,57 @@ public static Sequence parallelMerge( spliteratorStack.push(pop); spliteratorStack.push(other); } - return new MergeSequence<>(ordering, Sequences.simple(tasks.stream().map(ForkJoinTask::join))); + + final BlockingQueue> readyForFinalMerge = new ArrayBlockingQueue<>(tasks.size()); + tasks.forEach(t -> fjp.submit(() -> { + readyForFinalMerge.add(t.join()); + })); + + final long totalAdditions = tasks.size(); + return new MergeSequence<>( + ordering, + new BaseSequence<>( + new BaseSequence.IteratorMaker, Iterator>>() + { + @Override + public Iterator> make() + { + return new Iterator>() + { + long taken = 0L; + + @Override + public boolean hasNext() + { + return taken < totalAdditions; + } + + @Override + public Sequence next() throws NoSuchElementException + { + if (taken >= totalAdditions) { + throw new NoSuchElementException(); + } + try { + taken++; + return readyForFinalMerge.take(); + } + catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted waiting for intermediate merge", e); + } + } + }; + } + + @Override + public void cleanup(Iterator> iterFromMake) + { + tasks.forEach(t -> t.cancel(true)); + } + } + ) + ); } private final Ordering ordering; From ae6ae4bd2d7b28ecef050139ac0fd2b35ec0289a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 18:06:29 -0700 Subject: [PATCH 14/58] Pass Throwables --- .../java/util/common/guava/MergeWorkTask.java | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index ddb08e4866e6..4c9ae29b122f 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Ordering; +import io.druid.java.util.common.Pair; import java.util.ArrayList; import java.util.Deque; @@ -47,12 +48,18 @@ public class MergeWorkTask extends ForkJoinTask> * merge is still done when the returned sequence accumulated. The intermediate merges are yielded in the order * in which they are ready. * + * Exceptions that happen during execution of the merge are passed through and bubbled up during the resulting sequence + * iteration + * * @param ordering The ordering to pass into MergeSequence * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work * @param fjp The ForkJoinPool to do the intermediate merges in. * @param The result type * + * @throws RuntimeException Will throw a RuntimeException in during iterating through the returned Sequence if a Throwable + * was encountered in an intermediate merge + * * @return A Sequence that will be the merged results of the sub-sequences */ public static Sequence parallelMerge( @@ -94,9 +101,16 @@ public static Sequence parallelMerge( spliteratorStack.push(other); } - final BlockingQueue> readyForFinalMerge = new ArrayBlockingQueue<>(tasks.size()); - tasks.forEach(t -> fjp.submit(() -> { - readyForFinalMerge.add(t.join()); + // We guarantee enough space to put all the results so that the FJP doesn't block waiting for results to come in + final BlockingQueue, Throwable>> readyForFinalMerge = new ArrayBlockingQueue<>(tasks.size()); + tasks.forEach(task -> fjp.submit(() -> { + try { + readyForFinalMerge.add(Pair.of(task.join(), null)); + } + catch (Throwable t) { + // FJP.join exceptions are different than executor service's `.get()` + readyForFinalMerge.add(Pair.of(null, t)); + } })); final long totalAdditions = tasks.size(); @@ -126,7 +140,11 @@ public Sequence next() throws NoSuchElementException } try { taken++; - return readyForFinalMerge.take(); + final Pair, Throwable> result = readyForFinalMerge.take(); + if (result.rhs != null) { + throw new RuntimeException("failed in executing merge task", result.rhs); + } + return result.lhs; } catch (InterruptedException e) { Thread.currentThread().interrupt(); From 888c42e53aed2b31d0cc7f1a0659567250df3aec Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 26 Jun 2018 18:06:45 -0700 Subject: [PATCH 15/58] Code format --- .../java/io/druid/java/util/common/guava/MergeWorkTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index 4c9ae29b122f..e7f9e152741d 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -57,10 +57,10 @@ public class MergeWorkTask extends ForkJoinTask> * @param fjp The ForkJoinPool to do the intermediate merges in. * @param The result type * - * @throws RuntimeException Will throw a RuntimeException in during iterating through the returned Sequence if a Throwable - * was encountered in an intermediate merge - * * @return A Sequence that will be the merged results of the sub-sequences + * + * @throws RuntimeException Will throw a RuntimeException in during iterating through the returned Sequence if a Throwable + * was encountered in an intermediate merge */ public static Sequence parallelMerge( Ordering ordering, From 7377fc24beb39677df23814cc473345213e45e15 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 27 Jun 2018 11:48:36 -0700 Subject: [PATCH 16/58] Eager materialize direct druid client sequences --- .../druid/client/CachingClusteredClient.java | 20 +++++++++++-------- .../client/CachingClusteredClientTest.java | 13 ++++++++++-- 2 files changed, 23 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 94e957c39c1e..b9d6db7fe5a8 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -193,7 +193,7 @@ private Sequence runAndMergeWithTimelineChange( final UnaryOperator> timelineConverter ) { - final Stream> sequences = CachingClusteredClient.this.run( + final Stream> sequences = run( queryPlus, responseContext, timelineConverter @@ -334,7 +334,6 @@ Stream> run(final UnaryOperator>> cacheResolvedResults = deserializeFromCache( maybeFetchCacheResults( queryCacheKey, @@ -345,7 +344,13 @@ Stream> run(final UnaryOperator computeSegmentsToQuery(TimelineLookup timeline) @@ -552,7 +557,8 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter * 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. + * @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 */ @@ -647,7 +653,7 @@ private ServerMaybeSegmentMaybeCache pickServer(SerializablePair>> groupCachedResultsByServer(Stream>> cacheResolvedStream) { - final List>> listList = cacheResolvedStream.map( + return cacheResolvedStream.map( this::pickServer ).collect( Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) @@ -662,9 +668,7 @@ private Stream>> groupCachedResultsByServer ).filter( // Get rid of any alerted conditions missing queryableDruidServer l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() - ).collect(Collectors.toList()); - // We do a hard materialization here so that the resulting spliterators have properties that we want - return listList.stream(); + ); } private Stream>> deserializeFromCache( diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index 35f6289e8940..ccbd28924be9 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -604,9 +604,18 @@ public void testCachingOverBulkLimitEnforcesLimit() .once(); EasyMock.replay(cache); client = makeClient(MoreExecutors.sameThreadExecutor(), cache, 0); + + // Direct druid client runners are eagerly forked off + QueryRunner runner = EasyMock.createStrictMock(QueryRunner.class); + EasyMock.expect(runner.run(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(Sequences.empty()).once(); + + EasyMock.reset(serverView); + EasyMock.expect(serverView.getQueryRunner(lastServer)).andReturn(runner).once(); + EasyMock.replay(runner, serverView); + getDefaultQueryRunner().run(QueryPlus.wrap(query), context); - EasyMock.verify(cache); - EasyMock.verify(dataSegment); + + EasyMock.verify(cache, dataSegment, serverView, runner); Assert.assertTrue("Capture cache keys", cacheKeyCapture.hasCaptured()); Assert.assertTrue("Cache Keys empty", cacheKeyCapture.getValue().count() == 0); } From 40fce2440ebebc5bbf76a205695969896eda5fb7 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 27 Jun 2018 14:49:02 -0700 Subject: [PATCH 17/58] Fix functionality test --- .../CachingClusteredClientFunctionalityTest.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 6e86db0a6a23..53561ad19b42 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -31,6 +31,7 @@ import io.druid.client.selector.TierSelectorStrategy; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.guava.Sequence; +import io.druid.java.util.common.guava.Sequences; import io.druid.query.DataSource; import io.druid.query.Druids; import io.druid.query.Query; @@ -76,6 +77,21 @@ public void setUp() { timeline = new VersionedIntervalTimeline<>(Ordering.natural()); serverView = EasyMock.createNiceMock(TimelineServerView.class); + + final QueryRunner emptyQueryRunner = EasyMock.createStrictMock(QueryRunner.class); + + EasyMock.expect( + emptyQueryRunner.run(EasyMock.anyObject(), EasyMock.anyObject()) + ).andReturn( + Sequences.empty() + ).anyTimes(); + EasyMock.expect( + serverView.getQueryRunner(EasyMock.anyObject()) + ).andReturn( + emptyQueryRunner + ).anyTimes(); + + EasyMock.replay(serverView, emptyQueryRunner); cache = MapCache.create(100000); client = makeClient(MoreExecutors.sameThreadExecutor()); } From 445de11b2761a528b95898ecce4ae383e6b9505a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 27 Jun 2018 15:51:55 -0700 Subject: [PATCH 18/58] Remove non-germane changes --- .../java/util/common/guava/MergeIterator.java | 11 +- .../java/util/common/guava/MergeSequence.java | 12 +- .../DataSourceQueryQueryToolChest.java | 21 ++- .../SegmentMetadataQueryQueryToolChest.java | 25 +-- .../select/SelectQueryQueryToolChest.java | 142 ++++++++---------- .../TimeBoundaryQueryQueryToolChest.java | 22 ++- .../java/io/druid/server/QueryResource.java | 3 - 7 files changed, 132 insertions(+), 104 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java index 68b974191811..ada5577b6840 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeIterator.java @@ -28,7 +28,7 @@ import java.util.PriorityQueue; /** - */ +*/ public class MergeIterator implements Iterator { private final PriorityQueue> pQueue; @@ -40,7 +40,14 @@ public MergeIterator( { pQueue = new PriorityQueue<>( 16, - Comparator.comparing(PeekingIterator::peek, comparator) + new Comparator>() + { + @Override + public int compare(PeekingIterator lhs, PeekingIterator rhs) + { + return comparator.compare(lhs.peek(), rhs.peek()); + } + } ); for (Iterator iterator : iterators) { diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java index e9f0628fccd3..27277820c599 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeSequence.java @@ -19,6 +19,7 @@ package io.druid.java.util.common.guava; +import com.google.common.base.Function; import com.google.common.base.Throwables; import com.google.common.collect.Ordering; import io.druid.java.util.common.io.Closer; @@ -47,7 +48,16 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat { PriorityQueue> pQueue = new PriorityQueue<>( 32, - ordering.onResultOf(Yielder::get) + ordering.onResultOf( + new Function, T>() + { + @Override + public T apply(Yielder input) + { + return input.get(); + } + } + ) ); pQueue = baseSequences.accumulate( diff --git a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java index ce37d4e97d23..9a9c5c16659a 100644 --- a/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/datasourcemetadata/DataSourceQueryQueryToolChest.java @@ -22,6 +22,9 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.inject.Inject; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; @@ -39,7 +42,6 @@ import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** */ @@ -67,11 +69,18 @@ public List filterSegments(DataSourceMetadataQuery final T max = segments.get(segments.size() - 1); - return segments.stream( - ).filter( - segment -> max != null && segment.getInterval().overlaps(max.getInterval()) - ).collect( - Collectors.toList() + return Lists.newArrayList( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(T input) + { + return max != null && input.getInterval().overlaps(max.getInterval()); + } + } + ) ); } 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 875535c426dd..5025d9dbc975 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -23,15 +23,17 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; import io.druid.common.guava.CombiningSequence; -import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.JodaUtils; +import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.granularity.Granularity; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.MappedSequence; @@ -62,7 +64,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { @@ -110,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( @@ -240,11 +240,18 @@ public List filterSegments(SegmentMetadataQuery qu DateTime targetEnd = max.getInterval().getEnd(); final Interval targetInterval = new Interval(config.getDefaultHistory(), targetEnd); - return segments.stream( - ).filter( - segment -> segment.getInterval().overlaps(targetInterval) - ).collect( - Collectors.toList() + return Lists.newArrayList( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(T input) + { + return (input.getInterval().overlaps(targetInterval)); + } + } + ) ); } 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 362a91bec3fa..c0c3d8278502 100644 --- a/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/select/SelectQueryQueryToolChest.java @@ -24,8 +24,11 @@ import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.common.base.Preconditions; +import com.google.common.base.Predicate; import com.google.common.base.Supplier; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; @@ -59,9 +62,6 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; -import java.util.function.Predicate; -import java.util.stream.Collector; -import java.util.stream.Collectors; /** */ @@ -163,12 +163,8 @@ public CacheStrategy, Object, SelectQuery> getCacheStr { private final List dimensionSpecs = query.getDimensions() != null ? query.getDimensions() : Collections.emptyList(); - private final List dimOutputNames = dimensionSpecs.stream( - ).map( - DimensionSpec::getOutputName - ).collect( - Collectors.toList() - ); + private final List dimOutputNames = dimensionSpecs.size() > 0 ? + Lists.transform(dimensionSpecs, DimensionSpec::getOutputName) : Collections.emptyList(); @Override public boolean isCacheable(SelectQuery query, boolean willMergeRunners) @@ -291,32 +287,22 @@ 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 @@ -383,58 +369,62 @@ public List filterSegments(SelectQuery query, List // A paged select query using a UnionDataSource will return pagingIdentifiers from segments in more than one // dataSource which confuses subsequent queries and causes a failure. To avoid this, filter only the paging keys // that are applicable to this dataSource so that each dataSource in a union query gets the appropriate keys. + final Iterable filteredPagingKeys = Iterables.filter( + paging.keySet(), new Predicate() + { + @Override + public boolean apply(String input) + { + return DataSegmentUtils.valueOf(dataSource, input) != null; + } + } + ); - // The tree map needs to either be based off of max end or min start, depending on if we are ASC or DESC - final Collector> mapCollector = - query.isDescending() - ? Collectors.toMap( - interval -> granularity.bucketStart(interval.getEnd()).getMillis(), - Interval::getEndMillis, - Math::max, - TreeMap::new - ) - : Collectors.toMap( - interval -> granularity.bucketStart(interval.getStart()).getMillis(), - Interval::getStartMillis, - Math::min, - TreeMap::new - ); - - final TreeMap granularThresholds = paging.keySet( - ).stream( - ).filter( - key -> DataSegmentUtils.valueOf(dataSource, key) != null - ).map( - DataSegmentUtils.INTERVAL_EXTRACTOR(dataSource)::apply - ).sorted( - query.isDescending() ? Comparators.intervalsByEndThenStart() - : Comparators.intervalsByStartThenEnd() - ).collect( - mapCollector + List intervals = Lists.newArrayList( + Iterables.transform(filteredPagingKeys, DataSegmentUtils.INTERVAL_EXTRACTOR(dataSource)) + ); + Collections.sort( + intervals, query.isDescending() ? Comparators.intervalsByEndThenStart() + : Comparators.intervalsByStartThenEnd() ); - // Out of bounds for time if descending and too low, or ascending and too high - final Predicate outOfTimeBounds = query.isDescending( - ) ? segment -> { - final Interval interval = segment.getInterval(); - final Map.Entry ceiling = granularThresholds.ceilingEntry( - granularity.bucketStart(interval.getEnd()) - .getMillis()); - return ceiling != null - && interval.getStartMillis() < ceiling.getValue(); - } : segment -> { - final Interval interval = segment.getInterval(); - final Map.Entry floor = granularThresholds.floorEntry( - granularity.bucketStart(interval.getStart()) - .getMillis()); - return floor != null && interval.getEndMillis() > floor.getValue(); - }; + TreeMap granularThresholds = Maps.newTreeMap(); + for (Interval interval : intervals) { + if (query.isDescending()) { + long granularEnd = granularity.bucketStart(interval.getEnd()).getMillis(); + Long currentEnd = granularThresholds.get(granularEnd); + if (currentEnd == null || interval.getEndMillis() > currentEnd) { + granularThresholds.put(granularEnd, interval.getEndMillis()); + } + } else { + long granularStart = granularity.bucketStart(interval.getStart()).getMillis(); + Long currentStart = granularThresholds.get(granularStart); + if (currentStart == null || interval.getStartMillis() < currentStart) { + granularThresholds.put(granularStart, interval.getStartMillis()); + } + } + } - return segments.stream( - ).filter( - outOfTimeBounds - ).collect( - Collectors.toList() - ); + List queryIntervals = Lists.newArrayList(segments); + + Iterator it = queryIntervals.iterator(); + if (query.isDescending()) { + while (it.hasNext()) { + Interval interval = it.next().getInterval(); + Map.Entry ceiling = granularThresholds.ceilingEntry(granularity.bucketStart(interval.getEnd()).getMillis()); + if (ceiling == null || interval.getStartMillis() >= ceiling.getValue()) { + it.remove(); + } + } + } else { + while (it.hasNext()) { + Interval interval = it.next().getInterval(); + Map.Entry floor = granularThresholds.floorEntry(granularity.bucketStart(interval.getStart()).getMillis()); + if (floor == null || interval.getEndMillis() <= floor.getValue()) { + it.remove(); + } + } + } + return queryIntervals; } } 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 f8a0b95b360c..a1046a00c05b 100644 --- a/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/timeboundary/TimeBoundaryQueryQueryToolChest.java @@ -23,6 +23,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.inject.Inject; import io.druid.java.util.common.DateTimes; @@ -44,7 +46,6 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** */ @@ -84,12 +85,19 @@ public List filterSegments(TimeBoundaryQuery query final T min = query.isMaxTime() ? null : segments.get(0); final T max = query.isMinTime() ? null : segments.get(segments.size() - 1); - return segments.stream( - ).filter( - segment -> (min != null && segment.getInterval().overlaps(min.getInterval())) || - (max != null && segment.getInterval().overlaps(max.getInterval())) - ).collect( - Collectors.toList() + return Lists.newArrayList( + Iterables.filter( + segments, + new Predicate() + { + @Override + public boolean apply(T input) + { + return (min != null && input.getInterval().overlaps(min.getInterval())) || + (max != null && input.getInterval().overlaps(max.getInterval())); + } + } + ) ); } diff --git a/server/src/main/java/io/druid/server/QueryResource.java b/server/src/main/java/io/druid/server/QueryResource.java index 1942e0780394..ae4931e7aa86 100644 --- a/server/src/main/java/io/druid/server/QueryResource.java +++ b/server/src/main/java/io/druid/server/QueryResource.java @@ -198,9 +198,6 @@ public Response doPost( QueryContexts.isSerializeDateTimeAsLong(query, false) || (!shouldFinalize && QueryContexts.isSerializeDateTimeAsLongInner(query, false)); final ObjectWriter jsonWriter = context.newOutputWriter(serializeDateTimeAsLong); - // Wishlist: https://github.com/FasterXML/jackson-databind/blob/master/src/main/java/com/fasterxml/jackson/databind/ext/jdk8/Jdk8StreamSerializer.java - // https://github.com/FasterXML/jackson-databind/commit/cd32cd5539752cb606c41f09b04d61f170edf323 - // Probably landing in jackson-databind 3.x Response.ResponseBuilder builder = Response .ok( new StreamingOutput() From 7b663783aa871db2c5898e38a0f9ae726ce59bae Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 09:37:12 -0700 Subject: [PATCH 19/58] LongAdder --- .../java/io/druid/java/util/common/concurrent/Execs.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 302d36329fdc..12549b061d93 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -40,6 +40,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; /** */ @@ -152,13 +153,14 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) ); } - private static final AtomicLong fjpWorkerThreadCount = new AtomicLong(0L); + private static final LongAdder fjpWorkerThreadCount = new LongAdder(); public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool pool) { final FJPWorkerThread t = new FJPWorkerThread(pool); t.setDaemon(true); - t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.getAndIncrement())); + fjpWorkerThreadCount.increment(); + t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.longValue())); return t; } From 974f3b2e91dca04dcd59966a741343883690ce05 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 09:38:43 -0700 Subject: [PATCH 20/58] Remove type arguments in test --- .../test/java/io/druid/client/CachingClusteredClientTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index ccbd28924be9..84f98be3639b 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -3102,7 +3102,7 @@ public void testIfNoneMatch() TimeBoundaryQuery query = Druids.newTimeBoundaryQueryBuilder() .dataSource(DATA_SOURCE) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(interval))) - .context(ImmutableMap.of( + .context(ImmutableMap.of( "If-None-Match", "aVJV29CJY93rszVW/QBy0arWZo0=" )) From 7efd96e2af8152ac5438d12449563308a602fe13 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 10:00:23 -0700 Subject: [PATCH 21/58] Add HybridCache tests --- .../io/druid/client/cache/HybridCache.java | 28 +++++--- .../druid/client/cache/HybridCacheTest.java | 69 +++++++++++++++---- 2 files changed, 73 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/io/druid/client/cache/HybridCache.java b/server/src/main/java/io/druid/client/cache/HybridCache.java index 0a42625a4621..ee070f69e71d 100644 --- a/server/src/main/java/io/druid/client/cache/HybridCache.java +++ b/server/src/main/java/io/druid/client/cache/HybridCache.java @@ -31,7 +31,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -43,8 +43,8 @@ public class HybridCache implements Cache private final Cache level1; private final Cache level2; - private final AtomicLong hitCount = new AtomicLong(0); - private final AtomicLong missCount = new AtomicLong(0); + private final LongAdder hitCount = new LongAdder(); + private final LongAdder missCount = new LongAdder(); public HybridCache(HybridCacheConfig config, Cache level1, Cache level2) { @@ -66,10 +66,10 @@ public byte[] get(NamedKey key) } } if (res != null) { - hitCount.incrementAndGet(); + hitCount.increment(); return res; } else { - missCount.incrementAndGet(); + missCount.increment(); return null; } } @@ -98,7 +98,7 @@ public Map getBulk(Iterable keys) { Set remaining = Sets.newHashSet(keys); Map res = level1.getBulk(keys); - hitCount.addAndGet(res.size()); + hitCount.add(res.size()); remaining = Sets.difference(remaining, res.keySet()); @@ -109,8 +109,8 @@ public Map getBulk(Iterable keys) } int size = res2.size(); - hitCount.addAndGet(size); - missCount.addAndGet(remaining.size() - size); + hitCount.add(size); + missCount.add(remaining.size() - size); if (size != 0) { res = Maps.newHashMap(res); @@ -161,6 +161,14 @@ public Stream>> getBulk(Stream { + if (sp.getRhs().isPresent()) { + hitCount.increment(); + } else { + missCount.increment(); + } + }); return materializedL1Results.stream(); } @@ -177,8 +185,8 @@ public CacheStats getStats() CacheStats stats1 = level1.getStats(); CacheStats stats2 = level2.getStats(); return new CacheStats( - hitCount.get(), - missCount.get(), + hitCount.longValue(), + missCount.longValue(), stats1.getNumEntries() + stats2.getNumEntries(), stats1.getSizeInBytes() + stats2.getSizeInBytes(), stats1.getNumEvictions() + stats2.getNumEvictions(), diff --git a/server/src/test/java/io/druid/client/cache/HybridCacheTest.java b/server/src/test/java/io/druid/client/cache/HybridCacheTest.java index f129822ec747..70731cf3026a 100644 --- a/server/src/test/java/io/druid/client/cache/HybridCacheTest.java +++ b/server/src/test/java/io/druid/client/cache/HybridCacheTest.java @@ -22,11 +22,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Sets; import com.google.common.primitives.Ints; -import com.google.inject.Binder; import com.google.inject.Injector; import com.google.inject.Key; -import com.google.inject.Module; import com.google.inject.name.Names; +import io.druid.collections.SerializablePair; import io.druid.guice.CacheModule; import io.druid.guice.GuiceInjectors; import io.druid.guice.annotations.Global; @@ -35,8 +34,12 @@ import org.junit.Assert; import org.junit.Test; +import java.util.Arrays; import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; public class HybridCacheTest { @@ -53,17 +56,12 @@ public void testInjection() System.setProperty(prefix + ".l2.hosts", "localhost:11711"); final Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bindConstant().annotatedWith(Names.named("serviceName")).to("hybridTest"); - binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); - binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); - binder.install(new CacheModule(prefix)); - } + GuiceInjectors.makeStartupInjector(), ImmutableList.of( + binder -> { + binder.bindConstant().annotatedWith(Names.named("serviceName")).to("hybridTest"); + binder.bindConstant().annotatedWith(Names.named("servicePort")).to(0); + binder.bindConstant().annotatedWith(Names.named("tlsServicePort")).to(-1); + binder.install(new CacheModule(prefix)); } ) ); @@ -98,7 +96,6 @@ public void testSanity() final byte[] value3 = Ints.toByteArray(3); - // test put puts to both cache.put(key1, value1); Assert.assertEquals(value1, l1.get(key1)); @@ -137,6 +134,24 @@ public void testSanity() Assert.assertEquals(hits, cache.getStats().getNumHits()); } + // test streaming bulk get with l1 and l2 + { + final List keys = ImmutableList.of(key1, key2, key3); + final List>> res = cache + .getBulk(keys.stream()).collect(Collectors.toList()); + Assert.assertNotNull(res); + Assert.assertEquals(Arrays.asList( + new SerializablePair<>(key1, Optional.of(value1)), + new SerializablePair<>(key2, Optional.of(value2)), + new SerializablePair<>(key3, Optional.of(value3)) + ), res); + + hits += 3; + Assert.assertEquals(0, cache.getStats().getNumMisses()); + Assert.assertEquals(hits, cache.getStats().getNumHits()); + } + + // test bulk get with l1 entries only { final HashSet keys = Sets.newHashSet(key1, key2); @@ -175,5 +190,31 @@ public void testSanity() Assert.assertEquals(++hits, cache.getStats().getNumHits()); Assert.assertEquals(++misses, cache.getStats().getNumMisses()); } + + { + final List keys = ImmutableList.of(key3, key4); + final List>> res = cache + .getBulk(keys.stream()).collect(Collectors.toList()); + Assert.assertNotNull(res); + Assert.assertEquals(Arrays.asList( + new SerializablePair<>(key3, Optional.of(value3)), + new SerializablePair<>(key4, Optional.empty()) + ), res); + Assert.assertEquals(++misses, cache.getStats().getNumMisses()); + Assert.assertEquals(++hits, cache.getStats().getNumHits()); + } + + { + final List keys = ImmutableList.of(key1, key4); + final List>> res = cache + .getBulk(keys.stream()).collect(Collectors.toList()); + Assert.assertNotNull(res); + Assert.assertEquals(Arrays.asList( + new SerializablePair<>(key1, Optional.of(value1)), + new SerializablePair<>(key4, Optional.empty()) + ), res); + Assert.assertEquals(++misses, cache.getStats().getNumMisses()); + Assert.assertEquals(++hits, cache.getStats().getNumHits()); + } } } From 6341e1cb30f3f6bebc2cc22de3f53b03deb401d6 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 10:20:59 -0700 Subject: [PATCH 22/58] Remove unused import --- .../main/java/io/druid/java/util/common/concurrent/Execs.java | 1 - 1 file changed, 1 deletion(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 12549b061d93..45b95aef2ed7 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -39,7 +39,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.LongAdder; /** From 981a0d6a6c7cc13b3da165a82f5960b6959b733b Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 10:31:26 -0700 Subject: [PATCH 23/58] Fix copyright --- .../druid/java/util/common/guava/MergeWorkTask.java | 12 ++++++------ .../java/util/common/guava/MergeWorkTaskTest.java | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index e7f9e152741d..363435bdcefe 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java index aad962294f99..eb901df36ce6 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ From a1d286309ab645adf590dfd383d58060029bd428 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Fri, 13 Jul 2018 10:43:12 -0700 Subject: [PATCH 24/58] Copyrights --- .../java/io/druid/guice/ForkJoinPoolProvider.java | 12 ++++++------ .../java/io/druid/guice/LifecycleForkJoinPool.java | 12 ++++++------ 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java index b47881b997a7..c6fa29b0bd5b 100644 --- a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java +++ b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ diff --git a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java index 31222c7ba351..960a340f1816 100644 --- a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java +++ b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java @@ -1,18 +1,18 @@ /* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file + * Licensed to the Apache Software Foundation (ASF) 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 + * regarding copyright ownership. The ASF 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 + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * 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 + * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. */ From b3916afe729aba48592ee57f06dc1f06ea5cc4a0 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:05:53 -0700 Subject: [PATCH 25/58] Annonymous fork join pool worker task --- .../java/util/common/concurrent/Execs.java | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 070b6d5c9176..cdf9d35a2471 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -156,25 +156,13 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool pool) { - final FJPWorkerThread t = new FJPWorkerThread(pool); + final ForkJoinWorkerThread t = new ForkJoinWorkerThread(pool) + { + // No special handling in subclass + }; t.setDaemon(true); fjpWorkerThreadCount.increment(); t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.longValue())); return t; } - - static class FJPWorkerThread extends ForkJoinWorkerThread - { - /** - * Creates a ForkJoinWorkerThread operating in the given pool. - * - * @param pool the pool this thread works in - * - * @throws NullPointerException if pool is null - */ - FJPWorkerThread(ForkJoinPool pool) - { - super(pool); - } - } } From ae3d7ef2e57c7f1fe765495c0ab6afcdc9759c1e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:07:55 -0700 Subject: [PATCH 26/58] simple --> fromStream --- .../druid/java/util/common/guava/MergeWorkTask.java | 2 +- .../io/druid/java/util/common/guava/Sequences.java | 2 +- .../java/util/common/guava/MergeSequenceTest.java | 12 ++++++------ .../java/io/druid/client/CachingClusteredClient.java | 4 ++-- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index 363435bdcefe..7476612734c3 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -71,7 +71,7 @@ public static Sequence parallelMerge( { if (!baseSequences.isParallel()) { // Don't even try. - return new MergeSequence<>(ordering, Sequences.simple(baseSequences)); + return new MergeSequence<>(ordering, Sequences.fromStream(baseSequences)); } @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java index 7d36b6251a4a..07ac261b7a41 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/Sequences.java @@ -59,7 +59,7 @@ public void cleanup(Iterator iterFromMake) ); } - public static Sequence simple(final Stream stream) + public static Sequence fromStream(final Stream stream) { return new BaseSequence<>( new BaseSequence.IteratorMaker>() diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index 6a22367b9f9f..3b93b5567ff1 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -55,7 +55,7 @@ public void testSanity() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), @@ -85,7 +85,7 @@ public void testWorksWhenBeginningOutOfOrder() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), @@ -118,7 +118,7 @@ public void testMergeEmpties() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), @@ -151,7 +151,7 @@ public void testMergeEmpties1() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), @@ -185,7 +185,7 @@ public void testMergeEmpties2() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), @@ -214,7 +214,7 @@ public void testScrewsUpOnOutOfOrder() throws Exception SequenceTestHelper.testAll( () -> new MergeSequence<>( Ordering.natural(), - Sequences.simple( + Sequences.fromStream( testSeqs.stream() ) ), diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index efea4efdddf1..bd4a7b5b6047 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -209,7 +209,7 @@ private Sequence runAndMergeWithTimelineChange( } else { return new MergeSequence<>( query.getResultOrdering(), - Sequences.simple(sequences) + Sequences.fromStream(sequences) ); } } @@ -581,7 +581,7 @@ private Sequence runOnServer(List> segmentOrR // 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( + return new MergeSequence<>(query.getResultOrdering(), Sequences.fromStream( segmentOrResult.stream( ).map( ServerMaybeSegmentMaybeCache::getCachedValue From 19290ecd3e50b476aa5b6aa148047062ec106cea Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:10:40 -0700 Subject: [PATCH 27/58] Simple identity rename --- .../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 bd4a7b5b6047..403b925af30a 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -168,7 +168,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final queryPlus, responseContext, // No change, but Function.identity() doesn't work here for some reason - stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup + serverSelectorTimelineLookup -> serverSelectorTimelineLookup ); } From f44f9cfa17783dd44d7ca865cb1c5079153046ac Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:26:41 -0700 Subject: [PATCH 28/58] Method ordering refactoring --- .../druid/client/CachingClusteredClient.java | 88 +++++++++---------- 1 file changed, 43 insertions(+), 45 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 403b925af30a..0a9b8163d1ab 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -161,38 +161,54 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data } @Override - public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( - query, queryPlus, responseContext, - // No change, but Function.identity() doesn't work here for some reason - serverSelectorTimelineLookup -> serverSelectorTimelineLookup + timeline -> { + final VersionedIntervalTimeline timeline2 = + new VersionedIntervalTimeline<>(Ordering.natural()); + for (SegmentDescriptor spec : specs) { + final PartitionHolder entry = timeline.findEntry( + spec.getInterval(), + spec.getVersion() + ); + if (entry != null) { + final PartitionChunk chunk = entry.getChunk( + spec.getPartitionNumber()); + if (chunk != null) { + timeline2.add( + spec.getInterval(), + spec.getVersion(), + chunk + ); + } + } + } + return timeline2; + } ); } - /** - * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different - * timeline, if desired. This is used by getQueryRunnerForSegments. - */ - @VisibleForTesting - Stream> run( - final QueryPlus queryPlus, - final Map responseContext, - final UnaryOperator> timelineConverter - ) + @Override + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { - return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); + return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( + queryPlus, + responseContext, + // No change, but Function.identity() doesn't work here for some reason + serverSelectorTimelineLookup -> serverSelectorTimelineLookup + ); } private Sequence runAndMergeWithTimelineChange( - final Query query, final QueryPlus queryPlus, final Map responseContext, final UnaryOperator> timelineConverter ) { + final Query query = queryPlus.getQuery(); final Stream> sequences = run( queryPlus, responseContext, @@ -214,36 +230,18 @@ private Sequence runAndMergeWithTimelineChange( } } - @Override - public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + /** + * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different + * timeline, if desired. This is used by getQueryRunnerForSegments. + */ + @VisibleForTesting + Stream> run( + final QueryPlus queryPlus, + final Map responseContext, + final UnaryOperator> timelineConverter + ) { - return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( - query, - queryPlus, - responseContext, - timeline -> { - final VersionedIntervalTimeline timeline2 = - new VersionedIntervalTimeline<>(Ordering.natural()); - for (SegmentDescriptor spec : specs) { - final PartitionHolder entry = timeline.findEntry( - spec.getInterval(), - spec.getVersion() - ); - if (entry != null) { - final PartitionChunk chunk = entry.getChunk( - spec.getPartitionNumber()); - if (chunk != null) { - timeline2.add( - spec.getInterval(), - spec.getVersion(), - chunk - ); - } - } - } - return timeline2; - } - ); + return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); } /** From 6d90500729a3d46de55b01e659fc69f210d076e5 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:55:28 -0700 Subject: [PATCH 29/58] Refactoring a bit and better javadoc descriptions --- .../druid/collections/SerializablePair.java | 12 ++ .../druid/client/CachingClusteredClient.java | 177 +++++++++--------- 2 files changed, 104 insertions(+), 85 deletions(-) diff --git a/common/src/main/java/io/druid/collections/SerializablePair.java b/common/src/main/java/io/druid/collections/SerializablePair.java index b3bcf0af4066..6646a45962aa 100644 --- a/common/src/main/java/io/druid/collections/SerializablePair.java +++ b/common/src/main/java/io/druid/collections/SerializablePair.java @@ -23,6 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.java.util.common.Pair; +import java.util.Map; +import java.util.stream.Collector; +import java.util.stream.Collectors; + public class SerializablePair extends Pair { @JsonCreator @@ -42,4 +46,12 @@ public T2 getRhs() { return rhs; } + + public static Collector, ?, Map> mapCollector() + { + return Collectors.toMap( + SerializablePair::getLhs, + SerializablePair::getRhs + ); + } } diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 0a9b8163d1ab..a50429f3c478 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -338,17 +338,14 @@ Stream> run(final UnaryOperator computeSegmentsToQuery(TimelineLookup timeline) @@ -356,15 +353,13 @@ private Stream computeSegmentsToQuery(TimelineLookup timeline.lookup(i).stream()).collect(Collectors.toList()) - ).stream( - ).flatMap( + ).stream().flatMap( holder -> DimFilterUtils.filterShards( query.getFilter(), holder.getObject(), partitionChunk -> partitionChunk.getObject().getSegment().getShardSpec(), Maps.newHashMap() - ).stream( - ).map( + ).stream().map( chunk -> { ServerSelector server = chunk.getObject(); final SegmentDescriptor segment = new SegmentDescriptor( @@ -456,7 +451,7 @@ private String computeCurrentEtag(final Iterable segments, @Nul } /** - * This materializes the input segment stream in order to let the BulgGet stuff in the cache system work + * This materializes the input segment stream in order to let the BulkGet stuff in the cache system work * * @param queryCacheKey The cache key that is for the query (not-segment) portion * @param segments The segments to check if they are in cache @@ -482,19 +477,20 @@ private Stream>> maybeFetchCa final Map> cachedValues = computeCachedValues( materializedKeyList.stream() ).collect( - Collectors.toMap( - SerializablePair::getLhs, - SerializablePair::getRhs - ) + SerializablePair.mapCollector() ); - // Hash join to return the stream + // A limitation of the cache system is that the cached values are returned without passing through the original + // objects. This hash join is a way to get the ServerToSegment and Optional matched up again return materializedKeyList.stream().map( - psck -> { - final ServerToSegment segment = psck.getLhs(); - final Cache.NamedKey segmentCacheKey = psck.getRhs(); + serializedPairSegmentAndKey -> { + final ServerToSegment segment = serializedPairSegmentAndKey.getLhs(); + final Cache.NamedKey segmentCacheKey = serializedPairSegmentAndKey.getRhs(); final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); - final Optional cachedValue = Optional.ofNullable(cachedValues.get(segmentCacheKey)) - .orElse(Optional.empty()); // Shouldn't happen in practice, but can screw up unit tests + final Optional cachedValue = Optional + .ofNullable(cachedValues.get(segmentCacheKey)) + // Shouldn't happen in practice, but can screw up unit tests where cache state is mutated in crazy + // ways + .orElse(Optional.empty()); if (!cachedValue.isPresent()) { // if populating cache, add segment to list of segments to cache if it is not cached final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); @@ -523,7 +519,9 @@ private Stream> computePerSegm ); } - private Stream>> computeCachedValues(Stream> cacheKeys) + private Stream>> computeCachedValues( + Stream> cacheKeys + ) { if (useCache) { return cache.getBulk(cacheKeys.limit(cacheConfig.getCacheBulkMergeLimit()).map(SerializablePair::getRhs)); @@ -551,51 +549,15 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter } /** - * 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 + * Run all the queries for the specific segments on the specific server * - * @param segmentOrResult A list that is traversed in order to determine what should be sent back. All segments - * should be on the same server. + * @param server The server to target + * @param segmentsOfServer The segments on the server * - * @return A sequence of either the merged cached results, or the server results from any particular server + * @return The sequence of results for this specific server */ - private Sequence runOnServer(List> segmentOrResult) + private Sequence runOnServer(final DruidServer server, final List segmentsOfServer) { - final List 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.fromStream( - segmentOrResult.stream( - ).map( - ServerMaybeSegmentMaybeCache::getCachedValue - ).filter( - Optional::isPresent - ).map( - Optional::get - ).map( - Collections::singletonList - ).map( - Sequences::simple - ) - )); - } - - final DruidServer server = segmentOrResult.get(0).getServer(); final QueryRunner serverRunner = serverView.getQueryRunner(server); if (serverRunner == null) { @@ -616,6 +578,46 @@ private Sequence runOnServer(List> segmentOrR return serverResults; } + /** + * 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 mergeCacheOrRunOnServer(List> segmentOrResult) + { + final List segmentsOfServer = segmentOrResult + .stream() + .map(ServerMaybeSegmentMaybeCache::getSegmentDescriptor) + // Later versions of Java have better Optional + Stream builtin methods + .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.fromStream( + segmentOrResult + .stream() + .map(ServerMaybeSegmentMaybeCache::getCachedValue) + .filter(Optional::isPresent) + .map(Optional::get) + .map(Collections::singletonList) + .map(Sequences::simple) + )); + } else { + return runOnServer(segmentOrResult.get(0).getServer(), segmentsOfServer); + } + } + private ServerMaybeSegmentMaybeCache pickServer(SerializablePair> tuple) { final Optional maybeResult = tuple.getRhs(); @@ -644,29 +646,34 @@ private ServerMaybeSegmentMaybeCache pickServer(SerializablePair>> groupCachedResultsByServer(Stream>> cacheResolvedStream) + /** + * This materializes the input stream in order to group it by server. This method takes in the stream of cache + * resolved items and will group all the items by server. Each entry in the output stream contains a list whose + * entries' getServer is the same. Each entry will either have a present segemnt descriptor or a present result, + * but not both. Downstream consumers should check each and handle appropriately. + * + * @param cacheResolvedStream A stream of the cached results for different segment queries + * + * @return A stream of potentially cached results per server + */ + + private Stream>> groupCachedResultsByServer( + Stream>> cacheResolvedStream + ) { - return cacheResolvedStream.map( - this::pickServer - ).collect( - Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer) - ).values( - // Even though we have matrialized the map, and are iterating across the values, - // the java.util.HashMap.ValueSpliterator does not currently support things like sizing or subsetting, - // so we want to convert the collection into a form conducive with embarassing parallelization down the line - ).stream( - // At this point we have the segments per server, and a special entry for the pre-cached stuff - ).filter( - l -> !l.isEmpty() - ).filter( + return cacheResolvedStream + .map(this::pickServer) + .collect(Collectors.groupingBy(ServerMaybeSegmentMaybeCache::getServer)) + .values() + // At this point we have the segments per server, and a special entry for the pre-cached results. + // As of the time of this writing, this results in a java.util.HashMap.ValueSpliterator which + // does not have great properties for splitting in parallel since it does not have total size awareness + // yet. I hope future implementations of the grouping collector can handle such a scenario where the + // grouping result is immutable and can be split very easily into parallel spliterators + .stream() + .filter(l -> !l.isEmpty()) // Get rid of any alerted conditions missing queryableDruidServer - l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent() - ); + .filter(l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent()); } private Stream>> deserializeFromCache( From 066716eabc5c3c32ef19d97e179a0282a4b69c8e Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 16 Jul 2018 10:58:38 -0700 Subject: [PATCH 30/58] Formatting --- .../druid/client/CachingClusteredClient.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index a50429f3c478..7aee4749bef8 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -474,11 +474,9 @@ private Stream>> maybeFetchCa ).collect(Collectors.toList()); // Do bulk fetch - final Map> cachedValues = computeCachedValues( - materializedKeyList.stream() - ).collect( - SerializablePair.mapCollector() - ); + final Map> cachedValues = computeCachedValues(materializedKeyList.stream()) + .collect(SerializablePair.mapCollector()); + // A limitation of the cache system is that the cached values are returned without passing through the original // objects. This hash join is a way to get the ServerToSegment and Optional matched up again return materializedKeyList.stream().map( @@ -604,15 +602,18 @@ private Sequence mergeCacheOrRunOnServer(List // 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.fromStream( - segmentOrResult - .stream() - .map(ServerMaybeSegmentMaybeCache::getCachedValue) - .filter(Optional::isPresent) - .map(Optional::get) - .map(Collections::singletonList) - .map(Sequences::simple) - )); + return new MergeSequence<>( + query.getResultOrdering(), + Sequences.fromStream( + segmentOrResult + .stream() + .map(ServerMaybeSegmentMaybeCache::getCachedValue) + .filter(Optional::isPresent) + .map(Optional::get) + .map(Collections::singletonList) + .map(Sequences::simple) + ) + ); } else { return runOnServer(segmentOrResult.get(0).getServer(), segmentsOfServer); } From 9ddc295bb8906a21c5871775086194b10b5f7ba1 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 17 Jul 2018 15:50:10 -0700 Subject: [PATCH 31/58] Add query context docs --- docs/content/querying/query-context.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/content/querying/query-context.md b/docs/content/querying/query-context.md index d4e2be28f123..f396b460b2cf 100644 --- a/docs/content/querying/query-context.md +++ b/docs/content/querying/query-context.md @@ -22,6 +22,7 @@ The query context is used for various query configuration parameters. The follow |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. | |serializeDateTimeAsLong| `false` | If true, DateTime is serialized as long in the result returned by broker and the data transportation between broker and compute node| |serializeDateTimeAsLongInner| `false` | If true, DateTime is serialized as long in the data transportation between broker and compute node| +|intermediateMergeBatchThreshold|none, do not use|(EXPERIMENTAL) (positive integer) If present, will attempt to do parallel intermediate merges at the broker with a batch size of `intermediateMergeBatchThreshold` server results. This can greatly speed up the merging of large result sets across a large quantity of servers. The maximum number of active parallel merges across all queries is currently tied to the same count as the Common ForkJoinPool and can be set for both by the system property `java.util.concurrent.ForkJoinPool.common.parallelism`| In addition, some query types offer context parameters specific to that query type. From 1875786bbc213c9a819de48405ab90b5e57ac704 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 23 Jul 2018 15:27:03 -0700 Subject: [PATCH 32/58] Add FJP friendly methods --- .../io/druid/java/util/common/JodaUtils.java | 21 +++++++++++++ .../query/ChainedExecutionQueryRunner.java | 30 +++++++++++++++++-- .../io/druid/client/DirectDruidClient.java | 22 ++++++++++++++ 3 files changed, 70 insertions(+), 3 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java index 85cd9e325d1d..a210d91f60f9 100644 --- a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java @@ -28,6 +28,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.SortedSet; +import java.util.concurrent.TimeoutException; /** */ @@ -138,4 +139,24 @@ public static DateTime maxDateTime(DateTime... times) return max; } } + + /** + * Return a qty of millisconds approximately until deadline. If deadline has passed, throw TimeoutException + * + * @param deadline The time on or after which things should be considered "timed out" + * + * @return A millisecond number where, if one were to wait that many milliseconds, the deadline would + * probably have passed. Always greater than zero + * + * @throws TimeoutException If the deadline has already passed (ties are treated as having passed the deadline) + */ + public static long timeoutForDeadline(DateTime deadline) throws TimeoutException + { + final DateTime now = DateTime.now(); + final long diff = deadline.getMillis() - now.getMillis(); + if (now.isAfter(deadline) || diff == 0) { + throw new TimeoutException(StringUtils.format("Deadline passed: [%s]", deadline)); + } + return diff; + } } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index f0b1cf3f88be..eaf7bc1900db 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -28,10 +28,12 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import io.druid.java.util.common.ISE; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.logger.Logger; +import org.joda.time.DateTime; import java.util.Arrays; import java.util.Iterator; @@ -40,6 +42,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -146,11 +149,32 @@ public Iterable call() queryWatcher.registerQuery(query, futures); try { + final DateTime deadline = QueryContexts.hasTimeout(query) ? + new DateTime(DateTime.now().getMillis() + QueryContexts.getTimeout(query)) : + new DateTime(JodaUtils.MAX_INSTANT); + ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() + { + @Override + public boolean block() throws InterruptedException + { + try { + futures.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS); + } + catch (ExecutionException | TimeoutException e) { + // Will get caught later + } + return true; + } + + @Override + public boolean isReleasable() + { + return futures.isDone() || deadline.isBefore(DateTime.now()); + } + }); return new MergeIterable<>( ordering.nullsFirst(), - QueryContexts.hasTimeout(query) ? - futures.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS) : - futures.get() + futures.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS) ).iterator(); } catch (InterruptedException e) { diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 72607f91bf4c..15fca245c54e 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -84,6 +84,7 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -590,6 +591,27 @@ private void init() { if (jp == null) { try { + // Safety for if we are in a FJP + ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() + { + @Override + public boolean block() throws InterruptedException + { + try { + future.get(); + } + catch (ExecutionException e) { + // Ignore, will be caught when get is called below + } + return true; + } + + @Override + public boolean isReleasable() + { + return future.isDone(); + } + }); InputStream is = future.get(); if (is == null) { throw new QueryInterruptedException( From f5ecd7e271326f7c6e726e4a3cdb2d9395708d65 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Sat, 21 Jul 2018 16:01:56 -0700 Subject: [PATCH 33/58] Change streams to use fjp for merge work --- .../io/druid/common/guava/GuavaUtils.java | 9 ++ .../common/guava/ComplexSequenceTest.java | 1 + .../util}/common/guava/CombiningSequence.java | 7 +- .../java/util/common/guava/MergeWorkTask.java | 64 ++++----- .../common/guava/CombiningSequenceTest.java | 6 +- .../util/common/guava/MergeSequenceTest.java | 53 ++++++- .../util/common/guava/MergeWorkTaskTest.java | 50 +++---- .../query/ChainedExecutionQueryRunner.java | 98 +++++++------ .../druid/query/ResultMergeQueryRunner.java | 2 +- .../SegmentMetadataQueryQueryToolChest.java | 5 + .../io/druid/query/QueryRunnerTestHelper.java | 128 ++++++++++++++++- .../druid/client/CachingClusteredClient.java | 135 +++++++++++++++++- ...chingClusteredClientFunctionalityTest.java | 5 + .../client/CachingClusteredClientTest.java | 7 +- 14 files changed, 439 insertions(+), 131 deletions(-) rename {common/src/main/java/io/druid => java-util/src/main/java/io/druid/java/util}/common/guava/CombiningSequence.java (95%) rename {common/src/test/java/io/druid => java-util/src/test/java/io/druid/java/util}/common/guava/CombiningSequenceTest.java (96%) diff --git a/common/src/main/java/io/druid/common/guava/GuavaUtils.java b/common/src/main/java/io/druid/common/guava/GuavaUtils.java index 6ae0b830419f..646007da98a0 100644 --- a/common/src/main/java/io/druid/common/guava/GuavaUtils.java +++ b/common/src/main/java/io/druid/common/guava/GuavaUtils.java @@ -22,8 +22,12 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.primitives.Longs; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; import javax.annotation.Nullable; +import java.util.List; +import java.util.stream.Stream; /** */ @@ -62,4 +66,9 @@ public static > T getEnumIfPresent(final Class enumClass, f return null; } + + public static ListenableFuture> allFuturesAsList(Stream> futures) + { + return Futures.allAsList(futures::iterator); + } } diff --git a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java index 52208590b73c..112959498b9a 100644 --- a/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java +++ b/common/src/test/java/io/druid/common/guava/ComplexSequenceTest.java @@ -20,6 +20,7 @@ package io.druid.common.guava; import com.google.common.primitives.Ints; +import io.druid.java.util.common.guava.CombiningSequence; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; diff --git a/common/src/main/java/io/druid/common/guava/CombiningSequence.java b/java-util/src/main/java/io/druid/java/util/common/guava/CombiningSequence.java similarity index 95% rename from common/src/main/java/io/druid/common/guava/CombiningSequence.java rename to java-util/src/main/java/io/druid/java/util/common/guava/CombiningSequence.java index c3b5d995ce85..81e42a085fef 100644 --- a/common/src/main/java/io/druid/common/guava/CombiningSequence.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/CombiningSequence.java @@ -17,14 +17,9 @@ * under the License. */ -package io.druid.common.guava; +package io.druid.java.util.common.guava; import com.google.common.collect.Ordering; -import io.druid.java.util.common.guava.Accumulator; -import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.Yielders; -import io.druid.java.util.common.guava.YieldingAccumulator; import io.druid.java.util.common.guava.nary.BinaryFn; import java.io.IOException; diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index 7476612734c3..1d6f0d1d87fa 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -20,7 +20,6 @@ package io.druid.java.util.common.guava; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Ordering; import io.druid.java.util.common.Pair; import java.util.ArrayList; @@ -30,12 +29,14 @@ import java.util.List; import java.util.NoSuchElementException; import java.util.Spliterator; +import java.util.Spliterators; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinTask; -import java.util.function.Consumer; +import java.util.function.Function; import java.util.stream.Stream; +import java.util.stream.StreamSupport; public class MergeWorkTask extends ForkJoinTask> { @@ -51,7 +52,7 @@ public class MergeWorkTask extends ForkJoinTask> * Exceptions that happen during execution of the merge are passed through and bubbled up during the resulting sequence * iteration * - * @param ordering The ordering to pass into MergeSequence + * @param mergerFn The function that will merge a stream of sequences into a single sequence. If the baseSequences stream is parallel, this work will be done in the FJP, otherwise it will be called directly. * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work * @param fjp The ForkJoinPool to do the intermediate merges in. @@ -63,15 +64,19 @@ public class MergeWorkTask extends ForkJoinTask> * was encountered in an intermediate merge */ public static Sequence parallelMerge( - Ordering ordering, Stream> baseSequences, + Function>, Sequence> mergerFn, long batchSize, ForkJoinPool fjp ) { if (!baseSequences.isParallel()) { // Don't even try. +<<<<<<< HEAD return new MergeSequence<>(ordering, Sequences.fromStream(baseSequences)); +======= + return mergerFn.apply(baseSequences); +>>>>>>> 3753a98a3... Change streams to use fjp for merge work } @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); @@ -87,14 +92,14 @@ public static Sequence parallelMerge( final Spliterator> pop = spliteratorStack.pop(); if (pop.estimateSize() <= batchSize) { // Batch is small enough, yay! - tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + tasks.add(fjp.submit(new MergeWorkTask<>(mergerFn, pop))); continue; } final Spliterator> other = pop.trySplit(); if (other == null) { // splits are too big, but we can't split any more - tasks.add(fjp.submit(new MergeWorkTask<>(ordering, pop))); + tasks.add(fjp.submit(new MergeWorkTask<>(mergerFn, pop))); continue; } spliteratorStack.push(pop); @@ -114,15 +119,10 @@ public static Sequence parallelMerge( })); final long totalAdditions = tasks.size(); - return new MergeSequence<>( - ordering, - new BaseSequence<>( - new BaseSequence.IteratorMaker, Iterator>>() - { - @Override - public Iterator> make() - { - return new Iterator>() + return mergerFn.apply( + StreamSupport.stream( + Spliterators.spliterator( + new Iterator>() { long taken = 0L; @@ -133,7 +133,7 @@ public boolean hasNext() } @Override - public Sequence next() throws NoSuchElementException + public Sequence next() throws NoSuchElementException { if (taken >= totalAdditions) { throw new NoSuchElementException(); @@ -151,27 +151,26 @@ public Sequence next() throws NoSuchElementException throw new RuntimeException("Interrupted waiting for intermediate merge", e); } } - }; - } - - @Override - public void cleanup(Iterator> iterFromMake) - { - tasks.forEach(t -> t.cancel(true)); - } - } - ) + }, + totalAdditions, + Spliterator.NONNULL | Spliterator.SIZED + ), + false + ).onClose(() -> tasks.forEach(t -> t.cancel(true))) ); } - private final Ordering ordering; - private final Spliterator> baseSpliterator; + private final Spliterator> baseSpliterator; + private final Function>, Sequence> mergerFn; private Sequence result; @VisibleForTesting - MergeWorkTask(Ordering ordering, Spliterator> baseSpliterator) + MergeWorkTask( + Function>, Sequence> mergerFn, + Spliterator> baseSpliterator + ) { - this.ordering = ordering; + this.mergerFn = mergerFn; this.baseSpliterator = baseSpliterator; } @@ -190,13 +189,10 @@ protected void setRawResult(Sequence value) @Override protected boolean exec() { - final long estSize = baseSpliterator.estimateSize(); - final List> sequences = new ArrayList<>(estSize > 0 ? (int) estSize : 8); - baseSpliterator.forEachRemaining((Consumer>) sequences::add); // Force materialization "work" in this thread // For singleton lists it is not clear it is even worth the optimization of short circuiting the merge for the // extra code maintenance overhead - result = Sequences.simple(new MergeSequence<>(ordering, Sequences.simple(sequences)).toList()); + result = mergerFn.apply(StreamSupport.stream(baseSpliterator, false)); return true; } } diff --git a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/CombiningSequenceTest.java similarity index 96% rename from common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java rename to java-util/src/test/java/io/druid/java/util/common/guava/CombiningSequenceTest.java index 19486d990c5b..062d1fc8a3ae 100644 --- a/common/src/test/java/io/druid/common/guava/CombiningSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/CombiningSequenceTest.java @@ -17,7 +17,7 @@ * under the License. */ -package io.druid.common.guava; +package io.druid.java.util.common.guava; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; @@ -25,10 +25,6 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import io.druid.java.util.common.Pair; -import io.druid.java.util.common.guava.Sequence; -import io.druid.java.util.common.guava.Sequences; -import io.druid.java.util.common.guava.Yielder; -import io.druid.java.util.common.guava.YieldingAccumulator; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index 3b93b5567ff1..a982d82a2058 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -26,12 +26,30 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.function.Supplier; +import java.util.stream.Stream; /** */ public class MergeSequenceTest { + public static Supplier> naturalMergeSupplier( + Supplier>> stream + ) + { + return () -> naturalMerge(stream.get()); + } + + public static Sequence naturalMerge(Stream> stream) + { + return new MergeSequence<>( + Ordering.natural(), + Sequences.fromStream(stream) + ); + } + @Test public void testSanity() throws Exception { @@ -53,12 +71,16 @@ public void testSanity() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); @@ -83,12 +105,16 @@ public void testWorksWhenBeginningOutOfOrder() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -116,12 +142,16 @@ public void testMergeEmpties() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -149,12 +179,16 @@ public void testMergeEmpties1() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -183,12 +217,16 @@ public void testMergeEmpties2() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -212,12 +250,16 @@ public void testScrewsUpOnOutOfOrder() throws Exception } SequenceTestHelper.testAll( +<<<<<<< HEAD () -> new MergeSequence<>( Ordering.natural(), Sequences.fromStream( testSeqs.stream() ) ), +======= + naturalMergeSupplier(testSeqs::stream), +>>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -247,14 +289,15 @@ public void testHierarchicalMerge() throws Exception public void testMergeOne() throws Exception { final Sequence mergeOne = new MergeSequence<>( - Ordering.natural(), Sequences.>simple( - Lists.>newArrayList( - TestSequence.create(1) + Ordering.natural(), + Sequences.>simple( + Collections.singletonList( + TestSequence.create(1) + ) ) - ) ); - SequenceTestHelper.testAll(mergeOne, Arrays.asList(1)); + SequenceTestHelper.testAll(mergeOne, Collections.singletonList(1)); } } diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java index eb901df36ce6..170de2bd8bca 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeWorkTaskTest.java @@ -49,8 +49,8 @@ public void testNotParallelSequence() throws Exception final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), testSeqs.stream(), + s -> new MergeSequence<>(Ordering.natural(), Sequences.fromStream(s)), 999, ForkJoinPool.commonPool() ), expected); @@ -67,8 +67,8 @@ public void testOneBatchParallelSequence() throws Exception final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), testSeqs.stream().parallel(), + MergeSequenceTest::naturalMerge, 999, ForkJoinPool.commonPool() ), expected); @@ -85,8 +85,8 @@ public void testAllBatchParallelSequence() throws Exception final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), testSeqs.stream().parallel(), + MergeSequenceTest::naturalMerge, 1, ForkJoinPool.commonPool() ), expected); @@ -103,8 +103,8 @@ public void testSomeBatchParallelSequence() throws Exception final List expected = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9); SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), testSeqs.stream().parallel(), + MergeSequenceTest::naturalMerge, 2, ForkJoinPool.commonPool() ), expected); @@ -128,8 +128,8 @@ public void testFJPChoke() throws Exception false ); SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), testSeqs.stream().parallel(), + MergeSequenceTest::naturalMerge, 1, fjp ), expected); @@ -148,27 +148,27 @@ public void testBigMerge() throws Exception (t, e) -> exception.set(e), false ); - - // Take a big list of numbers, scatter them among a bunch of different buckets, then make sure the parallel merge - // returns the original list - - final List intList = IntStream.range(0, 10000).boxed().collect(Collectors.toList()); - final List> listList = new ArrayList<>(); - for (int i = 0; i < 500; i++) { - listList.add(new ArrayList<>()); + try (AutoCloseable closeable = fjp::shutdown) { + // Take a big list of numbers, scatter them among a bunch of different buckets, then make sure the parallel merge + // returns the original list + + final List intList = IntStream.range(0, 10000).boxed().collect(Collectors.toList()); + final List> listList = new ArrayList<>(); + for (int i = 0; i < 500; i++) { + listList.add(new ArrayList<>()); + } + final Random r = new Random(37489165L); + intList.forEach(i -> listList.get(r.nextInt(listList.size())).add(i)); + SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( + listList.stream( + ).map( + TestSequence::create + ).parallel(), + MergeSequenceTest::naturalMerge, + 10, + fjp + ), intList); } - final Random r = new Random(37489165L); - intList.forEach(i -> listList.get(r.nextInt(listList.size())).add(i)); - SequenceTestHelper.testAll(() -> MergeWorkTask.parallelMerge( - Ordering.natural(), - listList.stream( - ).map( - TestSequence::create - ).parallel(), - 10, - fjp - ), intList); - fjp.shutdown(); Assert.assertTrue(fjp.awaitTermination(5, TimeUnit.SECONDS)); Assert.assertNull(exception.get()); } diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index eaf7bc1900db..bfbbcd0b79b1 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -20,13 +20,11 @@ package io.druid.query; import com.google.common.base.Throwables; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -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.common.guava.GuavaUtils; import io.druid.java.util.common.ISE; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.BaseSequence; @@ -45,6 +43,8 @@ import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; /** * A QueryRunner that combines a list of other QueryRunners and executes them in parallel on an executor. @@ -63,7 +63,7 @@ public class ChainedExecutionQueryRunner implements QueryRunner { private static final Logger log = new Logger(ChainedExecutionQueryRunner.class); - private final Iterable> queryables; + private final Stream> queryables; private final ListeningExecutorService exec; private final QueryWatcher queryWatcher; @@ -73,7 +73,7 @@ public ChainedExecutionQueryRunner( QueryRunner... queryables ) { - this(exec, queryWatcher, Arrays.asList(queryables)); + this(exec, queryWatcher, Arrays.stream(queryables)); } public ChainedExecutionQueryRunner( @@ -81,69 +81,73 @@ public ChainedExecutionQueryRunner( QueryWatcher queryWatcher, Iterable> queryables ) + { + this(exec, queryWatcher, StreamSupport.stream(queryables.spliterator(), false)); + } + + public ChainedExecutionQueryRunner( + ExecutorService exec, + QueryWatcher queryWatcher, + Stream> queryables + ) { // listeningDecorator will leave PrioritizedExecutorService unchanged, // since it already implements ListeningExecutorService this.exec = MoreExecutors.listeningDecorator(exec); - this.queryables = Iterables.unmodifiableIterable(queryables); this.queryWatcher = queryWatcher; + this.queryables = queryables; } @Override public Sequence run(final QueryPlus queryPlus, final Map responseContext) { - Query query = queryPlus.getQuery(); + final Query query = queryPlus.getQuery(); final int priority = QueryContexts.getPriority(query); - final Ordering ordering = query.getResultOrdering(); + final Ordering ordering = query.getResultOrdering(); final QueryPlus threadSafeQueryPlus = queryPlus.withoutThreadUnsafeState(); - return new BaseSequence>( + return new BaseSequence<>( new BaseSequence.IteratorMaker>() { @Override public Iterator make() { // Make it a List<> to materialize all of the values (so that it will submit everything to the executor) - ListenableFuture>> futures = Futures.allAsList( - Lists.newArrayList( - Iterables.transform( - queryables, - input -> { - if (input == null) { - throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); + final ListenableFuture>> futures = GuavaUtils.allFuturesAsList( + queryables.peek( + queryRunner -> { + if (queryRunner == null) { + throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); + } + } + ).map( + queryRunner -> new AbstractPrioritizedCallable>(priority) + { + @Override + public Iterable call() + { + try { + Sequence result = queryRunner.run(threadSafeQueryPlus, responseContext); + if (result == null) { + throw new ISE("Got a null result! Segments are missing!"); + } + + List retVal = result.toList(); + if (retVal == null) { + throw new ISE("Got a null list of results! WTF?!"); } - return exec.submit( - new AbstractPrioritizedCallable>(priority) - { - @Override - public Iterable call() - { - try { - Sequence result = input.run(threadSafeQueryPlus, responseContext); - if (result == null) { - throw new ISE("Got a null result! Segments are missing!"); - } - - List retVal = result.toList(); - if (retVal == null) { - throw new ISE("Got a null list of results! WTF?!"); - } - - return retVal; - } - catch (QueryInterruptedException e) { - throw Throwables.propagate(e); - } - catch (Exception e) { - log.error(e, "Exception with one of the sequences!"); - throw Throwables.propagate(e); - } - } - } - ); + return retVal; + } + catch (QueryInterruptedException e) { + throw Throwables.propagate(e); + } + catch (Exception e) { + log.error(e, "Exception with one of the sequences!"); + throw Throwables.propagate(e); } - ) - ) + } + } + ).map(exec::submit) ); queryWatcher.registerQuery(query, futures); diff --git a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java index fd8882ae740d..545bfb1150ad 100644 --- a/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ResultMergeQueryRunner.java @@ -20,8 +20,8 @@ package io.druid.query; import com.google.common.collect.Ordering; -import io.druid.common.guava.CombiningSequence; import io.druid.guice.annotations.PublicApi; +import io.druid.java.util.common.guava.CombiningSequence; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.nary.BinaryFn; 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 34de6e49d6e8..71999c903d3f 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -31,7 +31,12 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; +<<<<<<< HEAD import io.druid.common.guava.CombiningSequence; +======= +import io.druid.java.util.common.guava.CombiningSequence; +import io.druid.data.input.impl.TimestampSpec; +>>>>>>> 3753a98a3... Change streams to use fjp for merge work import io.druid.java.util.common.JodaUtils; import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.granularity.Granularity; diff --git a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java index add0d071b2c8..842d7007a649 100644 --- a/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java +++ b/processing/src/test/java/io/druid/query/QueryRunnerTestHelper.java @@ -20,11 +20,13 @@ package io.druid.query; import com.google.common.base.Function; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.util.concurrent.MoreExecutors; +import io.druid.collections.StupidPool; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.granularity.Granularities; @@ -49,16 +51,40 @@ import io.druid.query.aggregation.post.FieldAccessPostAggregator; import io.druid.query.dimension.DefaultDimensionSpec; import io.druid.query.dimension.DimensionSpec; +import io.druid.query.groupby.GroupByQuery; +import io.druid.query.groupby.GroupByQueryConfig; +import io.druid.query.groupby.GroupByQueryRunnerTest; +import io.druid.query.groupby.strategy.GroupByStrategySelector; +import io.druid.query.metadata.SegmentMetadataQueryConfig; +import io.druid.query.metadata.SegmentMetadataQueryQueryToolChest; +import io.druid.query.metadata.SegmentMetadataQueryRunnerFactory; +import io.druid.query.metadata.metadata.SegmentMetadataQuery; +import io.druid.query.scan.ScanQuery; +import io.druid.query.scan.ScanQueryConfig; +import io.druid.query.scan.ScanQueryEngine; +import io.druid.query.scan.ScanQueryQueryToolChest; +import io.druid.query.scan.ScanQueryRunnerFactory; +import io.druid.query.select.SelectQuery; +import io.druid.query.select.SelectQueryConfig; +import io.druid.query.select.SelectQueryEngine; +import io.druid.query.select.SelectQueryQueryToolChest; +import io.druid.query.select.SelectQueryRunnerFactory; import io.druid.query.spec.MultipleIntervalSegmentSpec; import io.druid.query.spec.QuerySegmentSpec; import io.druid.query.spec.SpecificSegmentSpec; +import io.druid.query.timeseries.TimeseriesQuery; import io.druid.query.timeseries.TimeseriesQueryEngine; import io.druid.query.timeseries.TimeseriesQueryQueryToolChest; import io.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import io.druid.query.topn.TopNQuery; +import io.druid.query.topn.TopNQueryConfig; +import io.druid.query.topn.TopNQueryQueryToolChest; +import io.druid.query.topn.TopNQueryRunnerFactory; import io.druid.segment.IncrementalIndexSegment; import io.druid.segment.QueryableIndex; import io.druid.segment.QueryableIndexSegment; import io.druid.segment.Segment; +import io.druid.segment.TestHelper; import io.druid.segment.TestIndex; import io.druid.segment.incremental.IncrementalIndex; import io.druid.timeline.TimelineObjectHolder; @@ -67,6 +93,7 @@ import org.joda.time.Interval; import javax.annotation.Nullable; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -81,7 +108,8 @@ public class QueryRunnerTestHelper { - public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> {}; + public static final QueryWatcher NOOP_QUERYWATCHER = (query, future) -> { + }; public static final String segmentId = "testSegment"; public static final String dataSource = "testing"; @@ -276,6 +304,104 @@ public Object[] apply(@Nullable Object input) ); } + + public static final Map, QueryRunnerFactory> DEFAULT_CONGLOMERATE_MAP = ImmutableMap + ., QueryRunnerFactory>builder() + .put( + SegmentMetadataQuery.class, + new SegmentMetadataQueryRunnerFactory( + new SegmentMetadataQueryQueryToolChest( + new SegmentMetadataQueryConfig("P1W") + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + ScanQuery.class, + new ScanQueryRunnerFactory( + new ScanQueryQueryToolChest( + new ScanQueryConfig(), + new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) + ), + new ScanQueryEngine() + ) + ) + .put( + SelectQuery.class, + new SelectQueryRunnerFactory( + new SelectQueryQueryToolChest( + TestHelper.makeJsonMapper(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator(), + Suppliers.ofInstance( + new SelectQueryConfig(true) + ) + ), + new SelectQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TopNQuery.class, + new TopNQueryRunnerFactory( + new StupidPool<>( + "test-TopNQueryRunnerFactory-bufferPool", + () -> ByteBuffer.allocate(10 << 20) + ), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator() + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + GroupByQuery.class, + GroupByQueryRunnerTest.makeQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }, + new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return "test-processing-%s"; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 << 20; + } + + @Override + public int getNumMergeBuffers() + { + // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. + // Two buffers for the broker and one for the queryable + return 3; + } + } + ) + ) + .build(); + // simple cartesian iterable public static Iterable cartesian(final Iterable... iterables) { diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 7aee4749bef8..fb1dda5be21a 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -51,10 +51,13 @@ import io.druid.java.util.emitter.EmittingLogger; import io.druid.query.BySegmentResultValueClass; import io.druid.query.CacheStrategy; +import io.druid.query.FluentQueryRunnerBuilder; import io.druid.query.Query; import io.druid.query.QueryContexts; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerFactory; +import io.druid.query.QueryRunnerFactoryConglomerate; import io.druid.query.QuerySegmentWalker; import io.druid.query.QueryToolChest; import io.druid.query.QueryToolChestWarehouse; @@ -113,6 +116,7 @@ public class CachingClusteredClient implements QuerySegmentWalker new DruidServerConfig(), ServerType.HISTORICAL ); + private final QueryRunnerFactoryConglomerate conglomerate; private final QueryToolChestWarehouse warehouse; private final TimelineServerView serverView; private final Cache cache; @@ -123,6 +127,7 @@ public class CachingClusteredClient implements QuerySegmentWalker @Inject public CachingClusteredClient( + QueryRunnerFactoryConglomerate conglomerate, QueryToolChestWarehouse warehouse, TimelineServerView serverView, Cache cache, @@ -132,6 +137,7 @@ public CachingClusteredClient( @Processing ForkJoinPool mergeFjp ) { + this.conglomerate = conglomerate; this.warehouse = warehouse; this.serverView = serverView; this.cache = cache; @@ -163,6 +169,7 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { +<<<<<<< HEAD return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( queryPlus, responseContext, @@ -188,6 +195,12 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final } return timeline2; } +======= + return runAndMergeWithTimelineChange( + query, + // No change, but Function.identity() doesn't work here for some reason + stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup +>>>>>>> 3753a98a3... Change streams to use fjp for merge work ); } @@ -202,6 +215,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final ); } +<<<<<<< HEAD private Sequence runAndMergeWithTimelineChange( final QueryPlus queryPlus, final Map responseContext, @@ -214,19 +228,63 @@ private Sequence runAndMergeWithTimelineChange( responseContext, timelineConverter ); +======= + private QueryRunner runAndMergeWithTimelineChange( + final Query query, + final UnaryOperator> timelineConverter + ) + { +>>>>>>> 3753a98a3... Change streams to use fjp for merge work final OptionalLong mergeBatch = QueryContexts.getIntermediateMergeBatchThreshold(query); + if (mergeBatch.isPresent()) { - return MergeWorkTask.parallelMerge( - query.getResultOrdering(), - sequences.parallel(), - mergeBatch.getAsLong(), - mergeFjp - ); + final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(query); + final QueryToolChest> toolChest = queryRunnerFactory.getToolchest(); + return (queryPlus, responseContext) -> { + final Stream> sequences = run( + queryPlus, + responseContext, + timelineConverter + ); + return MergeWorkTask.parallelMerge( + sequences.parallel(), + sequenceStream -> + new FluentQueryRunnerBuilder<>(toolChest) + .create( + queryRunnerFactory.mergeRunners( + mergeFjp, + sequenceStream.map( + s -> (QueryRunner) (ignored0, ignored1) -> (Sequence) s + ).collect( + Collectors.toList() + ) + ) + ) + .mergeResults() + .run(queryPlus, responseContext), + mergeBatch.getAsLong(), + mergeFjp + ); + }; } else { +<<<<<<< HEAD return new MergeSequence<>( query.getResultOrdering(), Sequences.fromStream(sequences) ); +======= + return (queryPlus, responseContext) -> { + final Stream> sequences = run( + queryPlus, + responseContext, + timelineConverter + ); + return new MergeSequence<>( + query.getResultOrdering(), + Sequences.fromStream(sequences) + ); + }; +>>>>>>> 3753a98a3... Change streams to use fjp for merge work } } @@ -241,7 +299,35 @@ Stream> run( final UnaryOperator> timelineConverter ) { +<<<<<<< HEAD return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); +======= + return runAndMergeWithTimelineChange( + query, + timeline -> { + final VersionedIntervalTimeline timeline2 = + new VersionedIntervalTimeline<>(Ordering.natural()); + for (SegmentDescriptor spec : specs) { + final PartitionHolder entry = timeline.findEntry( + spec.getInterval(), + spec.getVersion() + ); + if (entry != null) { + final PartitionChunk chunk = entry.getChunk( + spec.getPartitionNumber()); + if (chunk != null) { + timeline2.add( + spec.getInterval(), + spec.getVersion(), + chunk + ); + } + } + } + return timeline2; + } + ); +>>>>>>> 3753a98a3... Change streams to use fjp for merge work } /** @@ -556,6 +642,43 @@ private CachePopulator getCachePopulator(String segmentId, Interval segmentInter */ private Sequence runOnServer(final DruidServer server, final List segmentsOfServer) { +<<<<<<< HEAD +======= + final List 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.fromStream( + segmentOrResult.stream( + ).map( + ServerMaybeSegmentMaybeCache::getCachedValue + ).filter( + Optional::isPresent + ).map( + Optional::get + ).map( + Collections::singletonList + ).map( + Sequences::simple + ) + )); + } + + final DruidServer server = segmentOrResult.get(0).getServer(); +>>>>>>> 3753a98a3... Change streams to use fjp for merge work final QueryRunner serverRunner = serverView.getQueryRunner(server); if (serverRunner == null) { diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java index 3a7f36cb3098..713cac0ba40e 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientFunctionalityTest.java @@ -33,10 +33,12 @@ import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.query.DataSource; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.Query; import io.druid.query.QueryPlus; import io.druid.query.QueryRunner; +import io.druid.query.QueryRunnerTestHelper; import io.druid.query.aggregation.AggregatorFactory; import io.druid.query.aggregation.CountAggregatorFactory; import io.druid.server.coordination.ServerType; @@ -229,6 +231,9 @@ protected CachingClusteredClient makeClient( ) { return new CachingClusteredClient( + new DefaultQueryRunnerFactoryConglomerate( + QueryRunnerTestHelper.DEFAULT_CONGLOMERATE_MAP + ), CachingClusteredClientTest.WAREHOUSE, new TimelineServerView() { diff --git a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java index ef8aed3a1415..cadc5d8e1ab4 100644 --- a/server/src/test/java/io/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/io/druid/client/CachingClusteredClientTest.java @@ -65,12 +65,14 @@ import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.FunctionalIterable; import io.druid.java.util.common.guava.MergeIterable; +import io.druid.java.util.common.guava.MergeSequence; import io.druid.java.util.common.guava.MergeWorkTask; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.guava.Sequences; import io.druid.java.util.common.guava.nary.TrinaryFn; import io.druid.query.BySegmentResultValueClass; import io.druid.query.DataSource; +import io.druid.query.DefaultQueryRunnerFactoryConglomerate; import io.druid.query.Druids; import io.druid.query.FinalizeResultsQueryRunner; import io.druid.query.MapQueryToolChestWarehouse; @@ -2658,6 +2660,9 @@ protected CachingClusteredClient makeClient( ) { return new CachingClusteredClient( + new DefaultQueryRunnerFactoryConglomerate( + QueryRunnerTestHelper.DEFAULT_CONGLOMERATE_MAP + ), WAREHOUSE, new TimelineServerView() { @@ -3209,12 +3214,12 @@ public void testSpliterator() final Sequence> parallelMergeResults = MergeWorkTask.parallelMerge( - query.getResultOrdering(), client.run( QueryPlus.wrap(query), context, stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup ).parallel(), + s -> new MergeSequence<>(query.getResultOrdering(), Sequences.fromStream(s)), 1, ForkJoinPool.commonPool() ); From d014b0f21a66f52128bf732aee1002c87b3853b3 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:01:49 -0700 Subject: [PATCH 34/58] Fix missed merge conflict --- .../java/io/druid/java/util/common/guava/MergeWorkTask.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index 1d6f0d1d87fa..ef811bfb97bf 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -72,11 +72,7 @@ public static Sequence parallelMerge( { if (!baseSequences.isParallel()) { // Don't even try. -<<<<<<< HEAD - return new MergeSequence<>(ordering, Sequences.fromStream(baseSequences)); -======= return mergerFn.apply(baseSequences); ->>>>>>> 3753a98a3... Change streams to use fjp for merge work } @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); From e8761603df51e6a4d606c040e860b434620e4ae4 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:07:59 -0700 Subject: [PATCH 35/58] Fix missed merge in test --- .../util/common/guava/MergeSequenceTest.java | 54 ------------------- 1 file changed, 54 deletions(-) diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index a982d82a2058..ea916bc5352d 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -71,16 +71,7 @@ public void testSanity() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); @@ -105,16 +96,7 @@ public void testWorksWhenBeginningOutOfOrder() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -142,16 +124,7 @@ public void testMergeEmpties() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -179,16 +152,7 @@ public void testMergeEmpties1() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -217,16 +181,7 @@ public void testMergeEmpties2() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } @@ -250,16 +205,7 @@ public void testScrewsUpOnOutOfOrder() throws Exception } SequenceTestHelper.testAll( -<<<<<<< HEAD - () -> new MergeSequence<>( - Ordering.natural(), - Sequences.fromStream( - testSeqs.stream() - ) - ), -======= naturalMergeSupplier(testSeqs::stream), ->>>>>>> 3753a98a3... Change streams to use fjp for merge work expected ); } From 8c9bc56268969f46828a536783927a9770dde84f Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:26:05 -0700 Subject: [PATCH 36/58] Use non-forbidden api --- .../src/main/java/io/druid/java/util/common/JodaUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java index a210d91f60f9..cc62fdc47619 100644 --- a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java @@ -152,7 +152,7 @@ public static DateTime maxDateTime(DateTime... times) */ public static long timeoutForDeadline(DateTime deadline) throws TimeoutException { - final DateTime now = DateTime.now(); + final DateTime now = DateTimes.nowUtc(); final long diff = deadline.getMillis() - now.getMillis(); if (now.isAfter(deadline) || diff == 0) { throw new TimeoutException(StringUtils.format("Deadline passed: [%s]", deadline)); From fa84ac7d51636c32b8ad16bdaeb8e7fae856cfcc Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:29:34 -0700 Subject: [PATCH 37/58] Fix merge conflict --- .../metadata/SegmentMetadataQueryQueryToolChest.java | 10 +++------- 1 file changed, 3 insertions(+), 7 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 71999c903d3f..ba9e462ffc2d 100644 --- a/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/io/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -31,15 +31,10 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -<<<<<<< HEAD -import io.druid.common.guava.CombiningSequence; -======= -import io.druid.java.util.common.guava.CombiningSequence; import io.druid.data.input.impl.TimestampSpec; ->>>>>>> 3753a98a3... Change streams to use fjp for merge work import io.druid.java.util.common.JodaUtils; -import io.druid.data.input.impl.TimestampSpec; import io.druid.java.util.common.granularity.Granularity; +import io.druid.java.util.common.guava.CombiningSequence; import io.druid.java.util.common.guava.Comparators; import io.druid.java.util.common.guava.MappedSequence; import io.druid.java.util.common.guava.Sequence; @@ -116,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( From 57dc8c7459a88022824612a53a27cbb34b2c8f77 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:46:07 -0700 Subject: [PATCH 38/58] Fix more missed merge conflicts --- .../druid/client/CachingClusteredClient.java | 159 +++--------------- 1 file changed, 26 insertions(+), 133 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index fb1dda5be21a..0a0687296809 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -167,74 +167,34 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data } @Override - public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) + public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { -<<<<<<< HEAD - return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( - queryPlus, - responseContext, - timeline -> { - final VersionedIntervalTimeline timeline2 = - new VersionedIntervalTimeline<>(Ordering.natural()); - for (SegmentDescriptor spec : specs) { - final PartitionHolder entry = timeline.findEntry( - spec.getInterval(), - spec.getVersion() - ); - if (entry != null) { - final PartitionChunk chunk = entry.getChunk( - spec.getPartitionNumber()); - if (chunk != null) { - timeline2.add( - spec.getInterval(), - spec.getVersion(), - chunk - ); - } - } - } - return timeline2; - } -======= return runAndMergeWithTimelineChange( query, // No change, but Function.identity() doesn't work here for some reason stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup ->>>>>>> 3753a98a3... Change streams to use fjp for merge work - ); - } - - @Override - public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) - { - return (queryPlus, responseContext) -> runAndMergeWithTimelineChange( - queryPlus, - responseContext, - // No change, but Function.identity() doesn't work here for some reason - serverSelectorTimelineLookup -> serverSelectorTimelineLookup ); } -<<<<<<< HEAD - private Sequence runAndMergeWithTimelineChange( + /** + * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different + * timeline, if desired. This is used by getQueryRunnerForSegments. + */ + @VisibleForTesting + Stream> run( final QueryPlus queryPlus, final Map responseContext, final UnaryOperator> timelineConverter ) { - final Query query = queryPlus.getQuery(); - final Stream> sequences = run( - queryPlus, - responseContext, - timelineConverter - ); -======= + return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); + } + private QueryRunner runAndMergeWithTimelineChange( final Query query, final UnaryOperator> timelineConverter ) { ->>>>>>> 3753a98a3... Change streams to use fjp for merge work final OptionalLong mergeBatch = QueryContexts.getIntermediateMergeBatchThreshold(query); if (mergeBatch.isPresent()) { @@ -267,12 +227,6 @@ private QueryRunner runAndMergeWithTimelineChange( ); }; } else { -<<<<<<< HEAD - return new MergeSequence<>( - query.getResultOrdering(), - Sequences.fromStream(sequences) - ); -======= return (queryPlus, responseContext) -> { final Stream> sequences = run( queryPlus, @@ -284,24 +238,12 @@ private QueryRunner runAndMergeWithTimelineChange( Sequences.fromStream(sequences) ); }; ->>>>>>> 3753a98a3... Change streams to use fjp for merge work } } - /** - * Run a query. The timelineConverter will be given the "master" timeline and can be used to return a different - * timeline, if desired. This is used by getQueryRunnerForSegments. - */ - @VisibleForTesting - Stream> run( - final QueryPlus queryPlus, - final Map responseContext, - final UnaryOperator> timelineConverter - ) + @Override + public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { -<<<<<<< HEAD - return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); -======= return runAndMergeWithTimelineChange( query, timeline -> { @@ -327,7 +269,6 @@ Stream> run( return timeline2; } ); ->>>>>>> 3753a98a3... Change streams to use fjp for merge work } /** @@ -425,7 +366,7 @@ Stream> run(final UnaryOperator runOnServer(final DruidServer server, final List segmentsOfServer) + private Sequence runOnServer(List> segmentOrResult) { -<<<<<<< HEAD -======= - final List segmentsOfServer = segmentOrResult.stream( - ).map( - ServerMaybeSegmentMaybeCache::getSegmentDescriptor - ).filter( - Optional::isPresent - ).map( - Optional::get - ).collect( - Collectors.toList() - ); + final List 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()) { @@ -678,7 +615,6 @@ private Sequence runOnServer(final DruidServer server, final List>>>>>> 3753a98a3... Change streams to use fjp for merge work final QueryRunner serverRunner = serverView.getQueryRunner(server); if (serverRunner == null) { @@ -699,49 +635,6 @@ private Sequence runOnServer(final DruidServer server, final List mergeCacheOrRunOnServer(List> segmentOrResult) - { - final List segmentsOfServer = segmentOrResult - .stream() - .map(ServerMaybeSegmentMaybeCache::getSegmentDescriptor) - // Later versions of Java have better Optional + Stream builtin methods - .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.fromStream( - segmentOrResult - .stream() - .map(ServerMaybeSegmentMaybeCache::getCachedValue) - .filter(Optional::isPresent) - .map(Optional::get) - .map(Collections::singletonList) - .map(Sequences::simple) - ) - ); - } else { - return runOnServer(segmentOrResult.get(0).getServer(), segmentsOfServer); - } - } - private ServerMaybeSegmentMaybeCache pickServer(SerializablePair> tuple) { final Optional maybeResult = tuple.getRhs(); From 700d5af64021a617ce6e003b47e042917ba2045a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:49:59 -0700 Subject: [PATCH 39/58] Don't use forbidden apis again --- .../java/io/druid/query/ChainedExecutionQueryRunner.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index bfbbcd0b79b1..b20ae4504da0 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import io.druid.common.guava.GuavaUtils; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.guava.BaseSequence; @@ -154,7 +155,7 @@ public Iterable call() try { final DateTime deadline = QueryContexts.hasTimeout(query) ? - new DateTime(DateTime.now().getMillis() + QueryContexts.getTimeout(query)) : + new DateTime(DateTimes.nowUtc().getMillis() + QueryContexts.getTimeout(query)) : new DateTime(JodaUtils.MAX_INSTANT); ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() { @@ -173,7 +174,7 @@ public boolean block() throws InterruptedException @Override public boolean isReleasable() { - return futures.isDone() || deadline.isBefore(DateTime.now()); + return futures.isDone() || deadline.isBefore(DateTimes.nowUtc()); } }); return new MergeIterable<>( From 4031bc0a997b6395a943b05a8ba24c5d512eb210 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Tue, 24 Jul 2018 20:52:46 -0700 Subject: [PATCH 40/58] More forbidden apis --- .../java/io/druid/query/ChainedExecutionQueryRunner.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index b20ae4504da0..7883199a4861 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -154,9 +154,10 @@ public Iterable call() queryWatcher.registerQuery(query, futures); try { - final DateTime deadline = QueryContexts.hasTimeout(query) ? - new DateTime(DateTimes.nowUtc().getMillis() + QueryContexts.getTimeout(query)) : - new DateTime(JodaUtils.MAX_INSTANT); + final DateTime deadline = + QueryContexts.hasTimeout(query) ? + DateTimes.utc(DateTimes.nowUtc().getMillis() + QueryContexts.getTimeout(query)) : + DateTimes.utc(JodaUtils.MAX_INSTANT); ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() { @Override From 88a0ac9bbff50fe640a3e49ab2bcf9f525a4150c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 30 Jul 2018 15:34:56 -0700 Subject: [PATCH 41/58] Add Execs.checkThreadNameFormat --- .../java/io/druid/concurrent/ExecsTest.java | 17 +++++++++++++++-- .../java/util/common/concurrent/Execs.java | 14 ++++++++++++++ .../io/druid/guice/ForkJoinPoolProvider.java | 3 +-- 3 files changed, 30 insertions(+), 4 deletions(-) diff --git a/common/src/test/java/io/druid/concurrent/ExecsTest.java b/common/src/test/java/io/druid/concurrent/ExecsTest.java index 8b479e130321..013c49132fb7 100644 --- a/common/src/test/java/io/druid/concurrent/ExecsTest.java +++ b/common/src/test/java/io/druid/concurrent/ExecsTest.java @@ -21,13 +21,12 @@ import com.google.common.base.Throwables; import com.google.common.util.concurrent.ThreadFactoryBuilder; - import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; - import org.junit.Assert; import org.junit.Test; +import java.util.IllegalFormatException; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -55,6 +54,20 @@ public void testBlockingExecutorServiceThreeCapacity() throws Exception runTest(3); } + @Test + public void testNameFormatGood() throws Exception + { + Execs.checkThreadNameFormat("good-%s"); + Execs.checkThreadNameFormat("good-%d"); + Execs.checkThreadNameFormat("whoops"); + } + + @Test(expected = IllegalFormatException.class) + public void testNameForamtBad() throws Exception + { + Execs.checkThreadNameFormat("%"); + } + private static void runTest(final int capacity) throws Exception { final int nTasks = (capacity + 1) * 3; diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index cdf9d35a2471..4e8ab746ef0e 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -165,4 +165,18 @@ public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool po t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.longValue())); return t; } + + private static final int dummyThradNumber = 17; + + /** + * Fail fast if the format can't take a single argument integer for a thread counter. + * + * Note that LACK of any argument in the format string still renders a valid name + * + * @param format The name format to check + */ + public static void checkThreadNameFormat(String format) + { + StringUtils.format(format, dummyThradNumber); + } } diff --git a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java index c6fa29b0bd5b..05598ce49e46 100644 --- a/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java +++ b/processing/src/main/java/io/druid/guice/ForkJoinPoolProvider.java @@ -19,7 +19,6 @@ package io.druid.guice; -import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.logger.Logger; @@ -35,7 +34,7 @@ public class ForkJoinPoolProvider implements Provider public ForkJoinPoolProvider(String nameFormat) { // Fail fast on bad name format - StringUtils.format(nameFormat, 3); + Execs.checkThreadNameFormat(nameFormat); this.nameFormat = nameFormat; } From 0130742520418dedbf6ddcc09845225da2eeae56 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 30 Jul 2018 15:38:06 -0700 Subject: [PATCH 42/58] Pretty deadline caluclating in ChainedExecutionQueryRunner --- .../io/druid/query/ChainedExecutionQueryRunner.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 7883199a4861..8a5452813ece 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -154,10 +154,12 @@ public Iterable call() queryWatcher.registerQuery(query, futures); try { - final DateTime deadline = - QueryContexts.hasTimeout(query) ? - DateTimes.utc(DateTimes.nowUtc().getMillis() + QueryContexts.getTimeout(query)) : - DateTimes.utc(JodaUtils.MAX_INSTANT); + final DateTime deadline; + if (QueryContexts.hasTimeout(query)) { + deadline = DateTimes.nowUtc().plusMillis((int) QueryContexts.getTimeout(query)); + } else { + deadline = DateTimes.utc(JodaUtils.MAX_INSTANT); + } ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() { @Override From 0a938a8c64808f38294a8a787cefbcda95c3a341 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 30 Jul 2018 15:40:19 -0700 Subject: [PATCH 43/58] Add javadoc --- .../src/main/java/io/druid/common/guava/GuavaUtils.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/common/src/main/java/io/druid/common/guava/GuavaUtils.java b/common/src/main/java/io/druid/common/guava/GuavaUtils.java index 646007da98a0..860d9a2bcb9b 100644 --- a/common/src/main/java/io/druid/common/guava/GuavaUtils.java +++ b/common/src/main/java/io/druid/common/guava/GuavaUtils.java @@ -67,6 +67,14 @@ public static > T getEnumIfPresent(final Class enumClass, f return null; } + /** + * Materialze the stream of futures into a single listenable future that will return the list of results. + * + * @param futures The futures to collect into a single Listenable future + * @param The return value for the futures + * + * @return A single ListenableFuture whose return value is a list of the completed values of the input stream. + */ public static ListenableFuture> allFuturesAsList(Stream> futures) { return Futures.allAsList(futures::iterator); From 34bddefcb42882de2969ea5210acf50acc70b0fd Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 30 Jul 2018 16:04:11 -0700 Subject: [PATCH 44/58] Refactor to split out functions some more --- .../druid/client/CachingClusteredClient.java | 159 ++++++++++-------- 1 file changed, 91 insertions(+), 68 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 0a0687296809..1b397d48ca5c 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -375,29 +375,43 @@ Stream> run(final UnaryOperator computeSegmentsToQuery(TimelineLookup 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 extractServerAndSegment(TimelineObjectHolder holder) { - return toolChest.filterSegments( - query, - query.getIntervals().stream().flatMap(i -> timeline.lookup(i).stream()).collect(Collectors.toList()) - ).stream().flatMap( - holder -> DimFilterUtils.filterShards( + return DimFilterUtils + .filterShards( query.getFilter(), holder.getObject(), partitionChunk -> partitionChunk.getObject().getSegment().getShardSpec(), Maps.newHashMap() - ).stream().map( - chunk -> { - ServerSelector server = chunk.getObject(); - final SegmentDescriptor segment = new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - chunk.getChunkNumber() - ); - return new ServerToSegment(server, segment); - } ) - ).distinct(); + .stream() + .map(chunk -> new ServerToSegment( + chunk.getObject(), + new SegmentDescriptor( + holder.getInterval(), + holder.getVersion(), + chunk.getChunkNumber() + ) + )); + } + + private Stream computeSegmentsToQuery(TimelineLookup 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 timeline) @@ -477,6 +491,27 @@ private String computeCurrentEtag(final Iterable segments, @Nul } } + private SerializablePair> lookupInCache( + SerializablePair key, + Map> cache + ) + { + final ServerToSegment segment = key.getLhs(); + final Cache.NamedKey segmentCacheKey = key.getRhs(); + final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); + final Optional cachedValue = Optional + .ofNullable(cache.get(segmentCacheKey)) + // Shouldn't happen in practice, but can screw up unit tests where cache state is mutated in crazy + // ways when the cache returns null instead of an optional. + .orElse(Optional.empty()); + if (!cachedValue.isPresent()) { + // if populating cache, add segment to list of segments to cache if it is not cached + final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); + addCachePopulator(segmentCacheKey, segmentIdentifier, segmentQueryInterval); + } + return new SerializablePair<>(segment, cachedValue); + } + /** * This materializes the input segment stream in order to let the BulkGet stuff in the cache system work * @@ -506,24 +541,9 @@ private Stream>> maybeFetchCa // A limitation of the cache system is that the cached values are returned without passing through the original // objects. This hash join is a way to get the ServerToSegment and Optional matched up again - return materializedKeyList.stream().map( - serializedPairSegmentAndKey -> { - final ServerToSegment segment = serializedPairSegmentAndKey.getLhs(); - final Cache.NamedKey segmentCacheKey = serializedPairSegmentAndKey.getRhs(); - final Interval segmentQueryInterval = segment.getSegmentDescriptor().getInterval(); - final Optional cachedValue = Optional - .ofNullable(cachedValues.get(segmentCacheKey)) - // Shouldn't happen in practice, but can screw up unit tests where cache state is mutated in crazy - // ways - .orElse(Optional.empty()); - if (!cachedValue.isPresent()) { - // if populating cache, add segment to list of segments to cache if it is not cached - final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); - addCachePopulator(segmentCacheKey, segmentIdentifier, segmentQueryInterval); - } - return new SerializablePair<>(segment, cachedValue); - } - ); + return materializedKeyList + .stream() + .map(serializedPairSegmentAndKey -> lookupInCache(serializedPairSegmentAndKey, cachedValues)); } private Stream> computePerSegmentCacheKeys( @@ -599,18 +619,13 @@ private Sequence runOnServer(List> segmentOrR // 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.fromStream( - segmentOrResult.stream( - ).map( - ServerMaybeSegmentMaybeCache::getCachedValue - ).filter( - Optional::isPresent - ).map( - Optional::get - ).map( - Collections::singletonList - ).map( - Sequences::simple - ) + segmentOrResult + .stream() + .map(ServerMaybeSegmentMaybeCache::getCachedValue) + .filter(Optional::isPresent) + .map(Optional::get) + .map(Collections::singletonList) + .map(Sequences::simple) )); } @@ -759,6 +774,34 @@ private Sequence getSimpleServerResults( return serverRunner.run(queryPlus.withQuerySegmentSpec(segmentsOfServerSpec), responseContext); } + private Sequence bySegmentWithCachePopulator( + Result> result, + Function cachePrep + ) + { + final BySegmentResultValueClass resultsOfSegment = result.getValue(); + final CachePopulator cachePopulator = getCachePopulator( + resultsOfSegment.getSegmentId(), + resultsOfSegment.getInterval() + ); + Sequence res = Sequences + .simple(resultsOfSegment.getResults()) + .map(r -> { + if (cachePopulator != null) { + // only compute cache data if populating cache + cachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cachePrep.apply(r))); + } + return r; + }) + .map( + toolChest.makePreComputeManipulatorFn(downstreamQuery, MetricManipulatorFns.deserializing())::apply + ); + if (cachePopulator != null) { + res = res.withEffect(cachePopulator::populate, MoreExecutors.sameThreadExecutor()); + } + return res; + } + private Sequence getAndCacheServerResults( final QueryRunner serverRunner, final MultipleSpecificSegmentSpec segmentsOfServerSpec @@ -773,27 +816,7 @@ private Sequence getAndCacheServerResults( ); final Function cacheFn = strategy.prepareForSegmentLevelCache()::apply; return resultsBySegments - .map(result -> { - final BySegmentResultValueClass resultsOfSegment = result.getValue(); - final CachePopulator cachePopulator = - getCachePopulator(resultsOfSegment.getSegmentId(), resultsOfSegment.getInterval()); - Sequence res = Sequences - .simple(resultsOfSegment.getResults()) - .map(r -> { - if (cachePopulator != null) { - // only compute cache data if populating cache - cachePopulator.cacheFutures.add(backgroundExecutorService.submit(() -> cacheFn.apply(r))); - } - return r; - }) - .map( - toolChest.makePreComputeManipulatorFn(downstreamQuery, MetricManipulatorFns.deserializing())::apply - ); - if (cachePopulator != null) { - res = res.withEffect(cachePopulator::populate, MoreExecutors.sameThreadExecutor()); - } - return res; - }) + .map(result -> bySegmentWithCachePopulator(result, cacheFn)) .flatMerge(Function.identity(), query.getResultOrdering()); } } From 4c10bda3f56889916da3a7c5e4219bc8a43b1384 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 30 Jul 2018 16:08:54 -0700 Subject: [PATCH 45/58] Use atomic long for state in fork join naming --- .../java/io/druid/java/util/common/concurrent/Execs.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index 4e8ab746ef0e..cbbecb96cf98 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -39,7 +39,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.LongAdder; +import java.util.concurrent.atomic.AtomicLong; /** */ @@ -152,7 +152,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) ); } - private static final LongAdder fjpWorkerThreadCount = new LongAdder(); + private static final AtomicLong fjpWorkerThreadCount = new AtomicLong(0L); public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool pool) { @@ -161,8 +161,8 @@ public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool po // No special handling in subclass }; t.setDaemon(true); - fjpWorkerThreadCount.increment(); - t.setName(StringUtils.nonStrictFormat(name, fjpWorkerThreadCount.longValue())); + final long threadNumber = fjpWorkerThreadCount.incrementAndGet(); + t.setName(StringUtils.nonStrictFormat(name, threadNumber)); return t; } From fea19d2fe66e21f4b4b47bc055984e223396a615 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 6 Aug 2018 13:40:58 -0700 Subject: [PATCH 46/58] UCASE constant --- .../java/io/druid/java/util/common/concurrent/Execs.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index cbbecb96cf98..be0475d72389 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -166,7 +166,7 @@ public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool po return t; } - private static final int dummyThradNumber = 17; + private static final int DUMMY_THREAD_NUMBER = 17; /** * Fail fast if the format can't take a single argument integer for a thread counter. @@ -174,9 +174,10 @@ public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool po * Note that LACK of any argument in the format string still renders a valid name * * @param format The name format to check + * @throws java.util.IllegalFormatException if the format passed in does is not able to take a single thread parameter */ public static void checkThreadNameFormat(String format) { - StringUtils.format(format, dummyThradNumber); + StringUtils.format(format, DUMMY_THREAD_NUMBER); } } From 14228d1c755c277bc87e2b539908f6123565794c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 8 Aug 2018 16:06:46 -0700 Subject: [PATCH 47/58] Address code comments --- .../druid/collections/SerializablePair.java | 18 +- .../io/druid/java/util/common/JodaUtils.java | 5 +- .../java/io/druid/java/util/common/Pair.java | 23 +++ .../util/common/guava/MergeSequenceTest.java | 4 +- .../druid/client/CachingClusteredClient.java | 170 ++++++++---------- .../java/io/druid/client/cache/Cache.java | 6 +- .../io/druid/client/cache/CaffeineCache.java | 6 +- .../io/druid/client/cache/HybridCache.java | 14 +- .../io/druid/client/cache/MemcachedCache.java | 35 ++-- .../druid/client/cache/HybridCacheTest.java | 22 +-- 10 files changed, 140 insertions(+), 163 deletions(-) diff --git a/common/src/main/java/io/druid/collections/SerializablePair.java b/common/src/main/java/io/druid/collections/SerializablePair.java index 6646a45962aa..1feacda4b0aa 100644 --- a/common/src/main/java/io/druid/collections/SerializablePair.java +++ b/common/src/main/java/io/druid/collections/SerializablePair.java @@ -23,10 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import io.druid.java.util.common.Pair; -import java.util.Map; -import java.util.stream.Collector; -import java.util.stream.Collectors; - public class SerializablePair extends Pair { @JsonCreator @@ -35,23 +31,17 @@ public SerializablePair(@JsonProperty("lhs") T1 lhs, @JsonProperty("rhs") T2 rhs super(lhs, rhs); } + @Override @JsonProperty public T1 getLhs() { - return lhs; + return super.getLhs(); } + @Override @JsonProperty public T2 getRhs() { - return rhs; - } - - public static Collector, ?, Map> mapCollector() - { - return Collectors.toMap( - SerializablePair::getLhs, - SerializablePair::getRhs - ); + return super.getRhs(); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java index c4bf16ab1103..699b959253b5 100644 --- a/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java +++ b/java-util/src/main/java/io/druid/java/util/common/JodaUtils.java @@ -153,10 +153,9 @@ public static DateTime maxDateTime(DateTime... times) public static long timeoutForDeadline(DateTime deadline) throws TimeoutException { final DateTime now = DateTimes.nowUtc(); - final long diff = deadline.getMillis() - now.getMillis(); - if (now.isAfter(deadline) || diff == 0) { + if (now.isAfter(deadline) || now.isEqual(deadline)) { throw new TimeoutException(StringUtils.format("Deadline passed: [%s]", deadline)); } - return diff; + return deadline.getMillis() - now.getMillis(); } } diff --git a/java-util/src/main/java/io/druid/java/util/common/Pair.java b/java-util/src/main/java/io/druid/java/util/common/Pair.java index 213d84a88e1f..594e07f27e1b 100644 --- a/java-util/src/main/java/io/druid/java/util/common/Pair.java +++ b/java-util/src/main/java/io/druid/java/util/common/Pair.java @@ -20,7 +20,10 @@ package io.druid.java.util.common; import javax.annotation.Nullable; +import java.util.Map; import java.util.Objects; +import java.util.stream.Collector; +import java.util.stream.Collectors; /** */ @@ -32,6 +35,14 @@ public static Pair of(@Nullable T1 lhs, @Nullable T2 rhs) return new Pair<>(lhs, rhs); } + public static Collector, ?, Map> mapCollector() + { + return Collectors.toMap( + Pair::getLhs, + Pair::getRhs + ); + } + @Nullable public final T1 lhs; @@ -47,6 +58,18 @@ public Pair( this.rhs = rhs; } + @Nullable + public T1 getLhs() + { + return lhs; + } + + @Nullable + public T2 getRhs() + { + return rhs; + } + @Override public boolean equals(Object o) { diff --git a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java index 3f5396b7f99a..f3a52932166e 100644 --- a/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java +++ b/java-util/src/test/java/io/druid/java/util/common/guava/MergeSequenceTest.java @@ -35,14 +35,14 @@ */ public class MergeSequenceTest { - public static Supplier> naturalMergeSupplier( + public static > Supplier> naturalMergeSupplier( Supplier>> stream ) { return () -> naturalMerge(stream.get()); } - public static Sequence naturalMerge(Stream> stream) + public static > Sequence naturalMerge(Stream> stream) { return new MergeSequence<>( Ordering.natural(), diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index ce6342465b6b..f7263dfc3d30 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -32,7 +32,6 @@ import io.druid.client.cache.CachePopulator; import io.druid.client.selector.QueryableDruidServer; import io.druid.client.selector.ServerSelector; -import io.druid.collections.SerializablePair; import io.druid.guice.annotations.Processing; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Intervals; @@ -82,6 +81,7 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; import java.util.Spliterators; @@ -166,7 +166,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final return runAndMergeWithTimelineChange( query, // No change, but Function.identity() doesn't work here for some reason - stringServerSelectorTimelineLookup -> stringServerSelectorTimelineLookup + identity -> identity ); } @@ -238,31 +238,21 @@ private QueryRunner runAndMergeWithTimelineChange( @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { - return runAndMergeWithTimelineChange( - query, - timeline -> { - final VersionedIntervalTimeline timeline2 = - new VersionedIntervalTimeline<>(Ordering.natural()); - for (SegmentDescriptor spec : specs) { - final PartitionHolder entry = timeline.findEntry( - spec.getInterval(), - spec.getVersion() - ); - if (entry != null) { - final PartitionChunk chunk = entry.getChunk( - spec.getPartitionNumber()); - if (chunk != null) { - timeline2.add( - spec.getInterval(), - spec.getVersion(), - chunk - ); - } - } + return runAndMergeWithTimelineChange(query, timeline -> { + final VersionedIntervalTimeline timeline2 = new VersionedIntervalTimeline<>( + Ordering.natural() + ); + for (SegmentDescriptor spec : specs) { + final PartitionHolder entry = timeline.findEntry(spec.getInterval(), spec.getVersion()); + if (entry != null) { + final PartitionChunk chunk = entry.getChunk(spec.getPartitionNumber()); + if (chunk != null) { + timeline2.add(spec.getInterval(), spec.getVersion(), chunk); } - return timeline2; } - ); + } + return timeline2; + }); } /** @@ -335,7 +325,7 @@ Stream> run(final UnaryOperator materializedSegments = segments.collect(Collectors.toList()); + final List materializedSegments = segments.collect(Collectors.toList()); segments = materializedSegments.stream(); @Nullable @@ -353,7 +343,7 @@ Stream> run(final UnaryOperator>> cacheResolvedResults = deserializeFromCache( + final Stream>> cacheResolvedResults = deserializeFromCache( maybeFetchCacheResults( queryCacheKey, segments @@ -388,11 +378,7 @@ private Stream extractServerAndSegment(TimelineObjectHolder new ServerToSegment( chunk.getObject(), - new SegmentDescriptor( - holder.getInterval(), - holder.getVersion(), - chunk.getChunkNumber() - ) + new SegmentDescriptor(holder.getInterval(), holder.getVersion(), chunk.getChunkNumber()) )); } @@ -485,8 +471,8 @@ private String computeCurrentEtag(final Iterable segments, @Nul } } - private SerializablePair> lookupInCache( - SerializablePair key, + private Pair> lookupInCache( + Pair key, Map> cache ) { @@ -503,7 +489,7 @@ private SerializablePair> lookupInCache( final String segmentIdentifier = segment.getServer().getSegment().getIdentifier(); addCachePopulatorKey(segmentCacheKey, segmentIdentifier, segmentQueryInterval); } - return new SerializablePair<>(segment, cachedValue); + return Pair.of(segment, cachedValue); } /** @@ -515,23 +501,23 @@ private SerializablePair> lookupInCache( * @return A stream of the server and segment combinations as well as an optional that is present * if a cached value was found */ - private Stream>> maybeFetchCacheResults( + private Stream>> maybeFetchCacheResults( final byte[] queryCacheKey, final Stream segments ) { if (queryCacheKey == null) { - return segments.map(s -> new SerializablePair<>(s, Optional.empty())); + return segments.map(s -> Pair.of(s, Optional.empty())); } // We materialize the stream here in order to have the bulk cache fetching work as expected - final List> materializedKeyList = computePerSegmentCacheKeys( + final List> materializedKeyList = computePerSegmentCacheKeys( segments, queryCacheKey ).collect(Collectors.toList()); // Do bulk fetch final Map> cachedValues = computeCachedValues(materializedKeyList.stream()) - .collect(SerializablePair.mapCollector()); + .collect(Pair.mapCollector()); // A limitation of the cache system is that the cached values are returned without passing through the original // objects. This hash join is a way to get the ServerToSegment and Optional matched up again @@ -540,30 +526,29 @@ private Stream>> maybeFetchCa .map(serializedPairSegmentAndKey -> lookupInCache(serializedPairSegmentAndKey, cachedValues)); } - private Stream> computePerSegmentCacheKeys( + private Stream> computePerSegmentCacheKeys( Stream segments, byte[] queryCacheKey ) { - return segments.map( - serverToSegment -> { + return segments + .map(serverToSegment -> { // cacheKeys map must preserve segment ordering, in order for shards to always be combined in the same order final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( serverToSegment.getServer().getSegment().getIdentifier(), serverToSegment.getSegmentDescriptor(), queryCacheKey ); - return new SerializablePair<>(serverToSegment, segmentCacheKey); - } - ); + return Pair.of(serverToSegment, segmentCacheKey); + }); } - private Stream>> computeCachedValues( - Stream> cacheKeys + private Stream>> computeCachedValues( + Stream> cacheKeys ) { if (useCache) { - return cache.getBulk(cacheKeys.limit(cacheConfig.getCacheBulkMergeLimit()).map(SerializablePair::getRhs)); + return cache.getBulk(cacheKeys.limit(cacheConfig.getCacheBulkMergeLimit()).map(Pair::getRhs)); } else { return Stream.empty(); } @@ -580,10 +565,7 @@ private void addCachePopulatorKey( Interval segmentQueryInterval ) { - cachePopulatorKeyMap.put( - cacheKey(segmentIdentifier, segmentQueryInterval), - segmentCacheKey - ); + cachePopulatorKeyMap.put(cacheKey(segmentIdentifier, segmentQueryInterval), segmentCacheKey); } @Nullable @@ -607,8 +589,7 @@ private Sequence runOnServer(List> segmentOrR final List segmentsOfServer = segmentOrResult .stream() .map(ServerMaybeSegmentMaybeCache::getSegmentDescriptor) - .filter(Optional::isPresent) - .map(Optional::get) + .filter(Objects::nonNull) .collect(Collectors.toList()); // We should only ever have cache or queries to run, not both. So if we have no segments, try caches @@ -621,8 +602,7 @@ private Sequence runOnServer(List> segmentOrR segmentOrResult .stream() .map(ServerMaybeSegmentMaybeCache::getCachedValue) - .filter(Optional::isPresent) - .map(Optional::get) + .filter(Objects::nonNull) .map(Collections::singletonList) .map(Sequences::simple) )); @@ -649,11 +629,11 @@ private Sequence runOnServer(List> segmentOrR return serverResults; } - private ServerMaybeSegmentMaybeCache pickServer(SerializablePair> tuple) + private ServerMaybeSegmentMaybeCache pickServer(Pair> tuple) { final Optional maybeResult = tuple.getRhs(); if (maybeResult.isPresent()) { - return new ServerMaybeSegmentMaybeCache(ALREADY_CACHED_SERVER, Optional.empty(), maybeResult); + return new ServerMaybeSegmentMaybeCache<>(ALREADY_CACHED_SERVER, null, maybeResult.get()); } final ServerToSegment serverToSegment = tuple.getLhs(); final QueryableDruidServer queryableDruidServer = serverToSegment.getServer().pick(); @@ -663,17 +643,13 @@ private ServerMaybeSegmentMaybeCache pickServer(SerializablePair( - ALREADY_CACHED_SERVER, - Optional.empty(), - Optional.empty() - ); + return new ServerMaybeSegmentMaybeCache<>(ALREADY_CACHED_SERVER, null, null); } final DruidServer server = queryableDruidServer.getServer(); - return new ServerMaybeSegmentMaybeCache( + return new ServerMaybeSegmentMaybeCache<>( server, - Optional.ofNullable(serverToSegment.getSegmentDescriptor()), - Optional.empty() + serverToSegment.getSegmentDescriptor(), + null ); } @@ -689,7 +665,7 @@ private ServerMaybeSegmentMaybeCache pickServer(SerializablePair>> groupCachedResultsByServer( - Stream>> cacheResolvedStream + Stream>> cacheResolvedStream ) { return cacheResolvedStream @@ -704,42 +680,36 @@ private Stream>> groupCachedResultsByServer .stream() .filter(l -> !l.isEmpty()) // Get rid of any alerted conditions missing queryableDruidServer - .filter(l -> l.get(0).getCachedValue().isPresent() || l.get(0).getSegmentDescriptor().isPresent()); + .filter(l -> l.get(0).getCachedValue() != null || l.get(0).getSegmentDescriptor() != null); } - private Stream>> deserializeFromCache( - final Stream>> cachedResults + private Stream>> deserializeFromCache( + final Stream>> cachedResults ) { if (strategy == null) { - return cachedResults.map(s -> new SerializablePair<>(s.getLhs(), Optional.empty())); + return cachedResults.map(s -> Pair.of(s.getLhs(), Optional.empty())); } final Function pullFromCacheFunction = strategy.pullFromSegmentLevelCache()::apply; final TypeReference cacheObjectClazz = strategy.getCacheObjectClazz(); return cachedResults.flatMap(cachedResultPair -> { if (!cachedResultPair.getRhs().isPresent()) { - return Stream.of(new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty())); + return Stream.of(Pair.of(cachedResultPair.getLhs(), Optional.empty())); } final byte[] cachedResult = cachedResultPair.getRhs().get(); try { if (cachedResult.length == 0) { - return Stream.of(new SerializablePair<>(cachedResultPair.getLhs(), Optional.empty())); + return Stream.of(Pair.of(cachedResultPair.getLhs(), Optional.empty())); } - // Query granularity in a segment may be higher fidelity than the segment as a file, so this might have multiple results - return StreamSupport.stream( - Spliterators.spliteratorUnknownSize( - objectMapper.readValues( - objectMapper.getFactory().createParser(cachedResult), - cacheObjectClazz - ), + // Query granularity in a segment may be higher fidelity than the segment as a file, + // so this might have multiple results + return StreamSupport + .stream(Spliterators.spliteratorUnknownSize( + objectMapper.readValues(objectMapper.getFactory().createParser(cachedResult), cacheObjectClazz), 0 - ), - false - ).map( - pullFromCacheFunction - ).map( - obj -> new SerializablePair<>(cachedResultPair.getLhs(), Optional.ofNullable(obj)) - ); + ), false) + .map(pullFromCacheFunction) + .map(obj -> Pair.of(cachedResultPair.getLhs(), Optional.ofNullable(obj))); } catch (IOException e) { throw new RuntimeException(e); @@ -757,11 +727,11 @@ private Sequence getBySegmentServerResults( .run(queryPlus.withQuerySegmentSpec(segmentsOfServerSpec), responseContext); // bySegment results need to be de-serialized, see DirectDruidClient.run() return (Sequence) resultsBySegments - .map(result -> result.map( - resultsOfSegment -> resultsOfSegment.mapResults( + .map(result -> result + .map(resultsOfSegment -> resultsOfSegment.mapResults( toolChest.makePreComputeManipulatorFn(query, MetricManipulatorFns.deserializing())::apply - ) - )); + )) + ); } @SuppressWarnings("unchecked") @@ -816,28 +786,30 @@ private Sequence getAndCacheServerResults( private static class ServerMaybeSegmentMaybeCache { private final DruidServer server; - private final Optional segmentDescriptor; - private final Optional cachedValue; + private final SegmentDescriptor segmentDescriptor; + private final T cachedValue; public DruidServer getServer() { return server; } - public Optional getSegmentDescriptor() + @Nullable + public SegmentDescriptor getSegmentDescriptor() { return segmentDescriptor; } - public Optional getCachedValue() + @Nullable + public T getCachedValue() { return cachedValue; } private ServerMaybeSegmentMaybeCache( DruidServer server, - Optional segmentDescriptor, - Optional cachedValue + @Nullable SegmentDescriptor segmentDescriptor, + @Nullable T cachedValue ) { this.server = server; @@ -855,12 +827,12 @@ private ServerToSegment(ServerSelector server, SegmentDescriptor segment) ServerSelector getServer() { - return lhs; + return super.getLhs(); } SegmentDescriptor getSegmentDescriptor() { - return rhs; + return super.getRhs(); } } } diff --git a/server/src/main/java/io/druid/client/cache/Cache.java b/server/src/main/java/io/druid/client/cache/Cache.java index 9ef7bdf2d50e..4993712c5c9c 100644 --- a/server/src/main/java/io/druid/client/cache/Cache.java +++ b/server/src/main/java/io/druid/client/cache/Cache.java @@ -20,7 +20,7 @@ package io.druid.client.cache; import com.google.common.base.Preconditions; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.emitter.service.ServiceEmitter; @@ -56,9 +56,9 @@ public interface Cache * * @return */ - default Stream>> getBulk(Stream keys) + default Stream>> getBulk(Stream keys) { - return keys.map(key -> new SerializablePair<>(key, Optional.ofNullable(get(key)))); + return keys.map(key -> new Pair<>(key, Optional.ofNullable(get(key)))); } void close(String namespace); diff --git a/server/src/main/java/io/druid/client/cache/CaffeineCache.java b/server/src/main/java/io/druid/client/cache/CaffeineCache.java index 95093c302a09..9fec3feebfa1 100644 --- a/server/src/main/java/io/druid/client/cache/CaffeineCache.java +++ b/server/src/main/java/io/druid/client/cache/CaffeineCache.java @@ -25,7 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.emitter.service.ServiceEmitter; import io.druid.java.util.emitter.service.ServiceMetricEvent; @@ -109,10 +109,10 @@ public Map getBulk(Iterable keys) } @Override - public Stream>> getBulk(Stream keys) + public Stream>> getBulk(Stream keys) { return keys.map( - k -> new SerializablePair<>( + k -> Pair.of( k, Optional.ofNullable( cache.getIfPresent(k) diff --git a/server/src/main/java/io/druid/client/cache/HybridCache.java b/server/src/main/java/io/druid/client/cache/HybridCache.java index e3b4f646069d..2d43cf0b588d 100644 --- a/server/src/main/java/io/druid/client/cache/HybridCache.java +++ b/server/src/main/java/io/druid/client/cache/HybridCache.java @@ -21,7 +21,7 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; -import io.druid.collections.SerializablePair; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.emitter.service.ServiceEmitter; @@ -130,30 +130,30 @@ private Map getBulkL2(Iterable keys) } @Override - public Stream>> getBulk(Stream keys) + public Stream>> getBulk(Stream keys) { if (!config.getUseL2()) { return level1.getBulk(keys); } - final List>> materializedL1Results = level1 + final List>> materializedL1Results = level1 .getBulk(keys) .collect(Collectors.toList()); - final List>> materializedL2Results = level2 + final List>> materializedL2Results = level2 .getBulk( materializedL1Results.stream( ).filter( s -> !s.getRhs().isPresent() ).map( - SerializablePair::getLhs + Pair::getLhs ) ).collect(Collectors.toList()); // The l2 list should only have "missing" ones from l1. So we loop through and look for the missing L1 results // and replace with whatever l2 found int l2Pos = 0; for (int i = 0; i < materializedL1Results.size(); i++) { - final SerializablePair> me = materializedL1Results.get(i); + final Pair> me = materializedL1Results.get(i); if (!me.getRhs().isPresent()) { - final SerializablePair> other = materializedL2Results.get(l2Pos++); + final Pair> other = materializedL2Results.get(l2Pos++); if (!me.getLhs().equals(other.getLhs())) { // sanity check for something very broken break; diff --git a/server/src/main/java/io/druid/client/cache/MemcachedCache.java b/server/src/main/java/io/druid/client/cache/MemcachedCache.java index 817e91610275..1f52bb1910bf 100644 --- a/server/src/main/java/io/druid/client/cache/MemcachedCache.java +++ b/server/src/main/java/io/druid/client/cache/MemcachedCache.java @@ -29,8 +29,8 @@ import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import io.druid.collections.ResourceHolder; -import io.druid.collections.SerializablePair; import io.druid.collections.StupidResourceHolder; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import io.druid.java.util.common.logger.Logger; import io.druid.java.util.emitter.service.ServiceEmitter; @@ -572,31 +572,26 @@ Map getCacheMap(Collection keys) } @Override - public Stream>> getBulk(Stream keys) + public Stream>> getBulk(Stream keys) { - final List> materializedKeys = keys.map( - k -> new SerializablePair<>(k, computeKeyHash(memcachedPrefix, k)) + final List> materializedKeys = keys.map( + k -> Pair.of(k, computeKeyHash(memcachedPrefix, k)) ).collect( Collectors.toList() ); final Map some = getCacheMap( - materializedKeys.stream( - ).map( - SerializablePair::getRhs - ).collect( - Collectors.toList() - ) + materializedKeys + .stream() + .map(Pair::getRhs) + .collect(Collectors.toList()) ); return materializedKeys.stream().map(k -> { final NamedKey key = k.getLhs(); final String cacheKey = k.getRhs(); - return new SerializablePair<>( + return Pair.of( key, - Optional.ofNullable( - some.get(cacheKey) - ).map( - val -> deserializeValue(key, (byte[]) val) - ) + Optional.ofNullable(some.get(cacheKey)) + .map(val -> deserializeValue(key, (byte[]) val)) ); }); } @@ -604,11 +599,9 @@ public Stream>> getBulk(Stream getBulk(Iterable keys) { - return getBulk( - StreamSupport.stream(keys.spliterator(), false) - ).filter(s -> s.getRhs().isPresent()).collect( - Collectors.toMap(SerializablePair::getLhs, s -> s.getRhs().get()) - ); + return getBulk(StreamSupport.stream(keys.spliterator(), false)) + .filter(s -> s.getRhs().isPresent()) + .collect(Collectors.toMap(Pair::getLhs, s -> s.getRhs().get())); } @Override diff --git a/server/src/test/java/io/druid/client/cache/HybridCacheTest.java b/server/src/test/java/io/druid/client/cache/HybridCacheTest.java index ede9d13ee4c9..588b8f5ca7f1 100644 --- a/server/src/test/java/io/druid/client/cache/HybridCacheTest.java +++ b/server/src/test/java/io/druid/client/cache/HybridCacheTest.java @@ -25,11 +25,11 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.name.Names; -import io.druid.collections.SerializablePair; import io.druid.guice.CacheModule; import io.druid.guice.GuiceInjectors; import io.druid.guice.annotations.Global; import io.druid.initialization.Initialization; +import io.druid.java.util.common.Pair; import io.druid.java.util.common.StringUtils; import org.junit.Assert; import org.junit.Test; @@ -137,13 +137,13 @@ public void testSanity() // test streaming bulk get with l1 and l2 { final List keys = ImmutableList.of(key1, key2, key3); - final List>> res = cache + final List>> res = cache .getBulk(keys.stream()).collect(Collectors.toList()); Assert.assertNotNull(res); Assert.assertEquals(Arrays.asList( - new SerializablePair<>(key1, Optional.of(value1)), - new SerializablePair<>(key2, Optional.of(value2)), - new SerializablePair<>(key3, Optional.of(value3)) + Pair.of(key1, Optional.of(value1)), + Pair.of(key2, Optional.of(value2)), + Pair.of(key3, Optional.of(value3)) ), res); hits += 3; @@ -193,12 +193,12 @@ public void testSanity() { final List keys = ImmutableList.of(key3, key4); - final List>> res = cache + final List>> res = cache .getBulk(keys.stream()).collect(Collectors.toList()); Assert.assertNotNull(res); Assert.assertEquals(Arrays.asList( - new SerializablePair<>(key3, Optional.of(value3)), - new SerializablePair<>(key4, Optional.empty()) + Pair.of(key3, Optional.of(value3)), + Pair.of(key4, Optional.empty()) ), res); Assert.assertEquals(++misses, cache.getStats().getNumMisses()); Assert.assertEquals(++hits, cache.getStats().getNumHits()); @@ -206,12 +206,12 @@ public void testSanity() { final List keys = ImmutableList.of(key1, key4); - final List>> res = cache + final List>> res = cache .getBulk(keys.stream()).collect(Collectors.toList()); Assert.assertNotNull(res); Assert.assertEquals(Arrays.asList( - new SerializablePair<>(key1, Optional.of(value1)), - new SerializablePair<>(key4, Optional.empty()) + Pair.of(key1, Optional.of(value1)), + Pair.of(key4, Optional.empty()) ), res); Assert.assertEquals(++misses, cache.getStats().getNumMisses()); Assert.assertEquals(++hits, cache.getStats().getNumHits()); From 3aba8b75a2b988c65fbe0ac2e39b44c6f8f40a4c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Mon, 27 Aug 2018 11:01:10 -0700 Subject: [PATCH 48/58] Add tests for deadline --- .../io/druid/common/utils/JodaUtilsTest.java | 33 +++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java b/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java index 138954d21d00..339b231d7ca8 100644 --- a/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java +++ b/common/src/test/java/io/druid/common/utils/JodaUtilsTest.java @@ -19,6 +19,7 @@ package io.druid.common.utils; +import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.Intervals; import io.druid.java.util.common.JodaUtils; import org.joda.time.Duration; @@ -30,6 +31,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeoutException; /** */ @@ -144,4 +146,35 @@ public void testMinMaxPeriod() Assert.assertEquals(Long.MAX_VALUE, period.getMinutes()); } + @Test(expected = TimeoutException.class) + public void testDeadlinePassed() throws TimeoutException + { + JodaUtils.timeoutForDeadline(DateTimes.nowUtc().minus(1)); + } + + @Test(expected = TimeoutException.class) + public void testDeadlineMinInstant() throws TimeoutException + { + JodaUtils.timeoutForDeadline(DateTimes.utc(JodaUtils.MIN_INSTANT)); + } + + @Test + public void testDeadlineMaxInstant() throws TimeoutException + { + final long ms = JodaUtils.timeoutForDeadline(DateTimes.utc(JodaUtils.MAX_INSTANT)); + Assert.assertTrue(ms > 0); + } + + @Test(expected = TimeoutException.class) + public void testDeadlineLongMin() throws TimeoutException + { + JodaUtils.timeoutForDeadline(DateTimes.utc(Long.MIN_VALUE)); + } + + @Test + public void testDeadlineLongMax() throws TimeoutException + { + final long ms = JodaUtils.timeoutForDeadline(DateTimes.utc(Long.MAX_VALUE)); + Assert.assertTrue(ms > 0); + } } From 498c909e92f0a03ae33c749d90da86774fa61052 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 10:20:47 -0700 Subject: [PATCH 49/58] Address code comments --- .../java/io/druid/concurrent/ExecsTest.java | 4 +- .../java/util/common/concurrent/Execs.java | 94 ++++++++++++++++++- .../java/util/common/guava/MergeWorkTask.java | 14 +-- .../query/ChainedExecutionQueryRunner.java | 40 +++----- .../druid/client/CachingClusteredClient.java | 13 ++- .../io/druid/client/DirectDruidClient.java | 25 +---- 6 files changed, 121 insertions(+), 69 deletions(-) diff --git a/common/src/test/java/io/druid/concurrent/ExecsTest.java b/common/src/test/java/io/druid/concurrent/ExecsTest.java index 013c49132fb7..4988de6a61de 100644 --- a/common/src/test/java/io/druid/concurrent/ExecsTest.java +++ b/common/src/test/java/io/druid/concurrent/ExecsTest.java @@ -55,7 +55,7 @@ public void testBlockingExecutorServiceThreeCapacity() throws Exception } @Test - public void testNameFormatGood() throws Exception + public void testNameFormatGood() { Execs.checkThreadNameFormat("good-%s"); Execs.checkThreadNameFormat("good-%d"); @@ -63,7 +63,7 @@ public void testNameFormatGood() throws Exception } @Test(expected = IllegalFormatException.class) - public void testNameForamtBad() throws Exception + public void testNameForamtBad() { Execs.checkThreadNameFormat("%"); } diff --git a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java index be0475d72389..35761af187ba 100644 --- a/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java +++ b/java-util/src/main/java/io/druid/java/util/common/concurrent/Execs.java @@ -22,16 +22,21 @@ import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import io.druid.java.util.common.DateTimes; +import io.druid.java.util.common.JodaUtils; import io.druid.java.util.common.StringUtils; +import org.joda.time.DateTime; import javax.annotation.Nullable; import javax.validation.constraints.NotNull; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ForkJoinPool; import java.util.concurrent.ForkJoinWorkerThread; +import java.util.concurrent.Future; import java.util.concurrent.RejectedExecutionException; import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.ScheduledExecutorService; @@ -39,6 +44,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; /** @@ -156,10 +162,7 @@ public void rejectedExecution(Runnable r, ThreadPoolExecutor executor) public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool pool) { - final ForkJoinWorkerThread t = new ForkJoinWorkerThread(pool) - { - // No special handling in subclass - }; + final ForkJoinWorkerThread t = ForkJoinPool.defaultForkJoinWorkerThreadFactory.newThread(pool); t.setDaemon(true); final long threadNumber = fjpWorkerThreadCount.incrementAndGet(); t.setName(StringUtils.nonStrictFormat(name, threadNumber)); @@ -174,10 +177,93 @@ public static ForkJoinWorkerThread makeWorkerThread(String name, ForkJoinPool po * Note that LACK of any argument in the format string still renders a valid name * * @param format The name format to check + * * @throws java.util.IllegalFormatException if the format passed in does is not able to take a single thread parameter */ public static void checkThreadNameFormat(String format) { StringUtils.format(format, DUMMY_THREAD_NUMBER); } + + /** + * Get the result for the future (without timeout), but do so in a way safe for running in a ForkJoinPool + * + * @param future The future to block on completion + * @param The type of the return value + * + * @return The result of the future if successfully completed, or one of the exceptions if not + * + * @throws InterruptedException If the call to future.get() was interrupted + * @throws ExecutionException If the future completed with an exception + */ + public static T futureManagedBlockGet(final Future future) + throws InterruptedException, ExecutionException + { + ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() + { + @Override + public boolean block() throws InterruptedException + { + try { + future.get(); + } + catch (ExecutionException e) { + // Ignore, will be caught when get is called below + } + return true; + } + + @Override + public boolean isReleasable() + { + return future.isDone(); + } + }); + return future.get(); + } + + /** + * Attempt to get the result of the future before the deadline, but do so in a way safe to run in a ForkJoinPool. + * The deadline is best effort. It is possible the future completes, but the deadline is exceeded before the result + * can be returned. In such a scenario a TimeoutException will be thrown. + * + * The caller is responsible for handling the state of the Future in the case of an exception being thrown. + * Specifically, if an InterruptedException or a TimeoutException is thrown, there is no attempt in this method + * to change the behavior of the future. The caller should handle the potentially still active future as they see fit. + * + * @param future The future to await completion + * @param deadline Best effort deadline for the completion of the future. + * @param The future's yielded type + * + * @return The yield of the future or else a thrown exception + * + * @throws InterruptedException If the call to future.get is interrupted + * @throws TimeoutException If the deadline is exceeded + * @throws ExecutionException If the future completed with an exception + */ + public static T futureManagedBlockGet(final Future future, final DateTime deadline) + throws InterruptedException, TimeoutException, ExecutionException + { + ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() + { + @Override + public boolean block() throws InterruptedException + { + try { + future.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS); + } + catch (ExecutionException | TimeoutException e) { + // Will get caught later + } + return true; + } + + @Override + public boolean isReleasable() + { + return future.isDone() || deadline.isBefore(DateTimes.nowUtc()); + } + }); + return future.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS); + } } diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index ef811bfb97bf..f9c9ae912c81 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -43,16 +43,18 @@ public class MergeWorkTask extends ForkJoinTask> /** * Take a stream of sequences, split them as possible, and do intermediate merges. If the input stream is not - * a parallel stream, do a traditional merge. The stream attempts to use groups of {@code batchSize} to do its work, but this - * goal is on a best effort basis. Input streams that cannot be split or are not sized or not subsized might not be - * elligable for this parallelization. The intermediate merges are done in the passed in ForkJoinPool, but the final - * merge is still done when the returned sequence accumulated. The intermediate merges are yielded in the order - * in which they are ready. + * a parallel stream, do a traditional merge. The stream attempts to use groups of {@code batchSize} to do its work, + * but this goal is on a best effort basis. Input streams that cannot be split or are not sized or not subsized + * might not be elligable for this parallelization. The intermediate merges are done in the passed in ForkJoinPool, + * but the final merge is still done when the returned sequence accumulated. The intermediate merges are yielded + * in the order in which they are ready. * * Exceptions that happen during execution of the merge are passed through and bubbled up during the resulting sequence * iteration * - * @param mergerFn The function that will merge a stream of sequences into a single sequence. If the baseSequences stream is parallel, this work will be done in the FJP, otherwise it will be called directly. + * @param mergerFn The function that will merge a stream of sequences into a single sequence. If the + * baseSequences stream is parallel, this work will be done in the FJP, otherwise it + * will be called directly. * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work * @param fjp The ForkJoinPool to do the intermediate merges in. diff --git a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java index 8a5452813ece..cf3291c1498c 100644 --- a/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/io/druid/query/ChainedExecutionQueryRunner.java @@ -27,12 +27,11 @@ import io.druid.common.guava.GuavaUtils; import io.druid.java.util.common.DateTimes; import io.druid.java.util.common.ISE; -import io.druid.java.util.common.JodaUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.MergeIterable; import io.druid.java.util.common.guava.Sequence; import io.druid.java.util.common.logger.Logger; -import org.joda.time.DateTime; import java.util.Arrays; import java.util.Iterator; @@ -41,8 +40,6 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ForkJoinPool; -import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Stream; import java.util.stream.StreamSupport; @@ -114,11 +111,13 @@ public Iterator make() { // Make it a List<> to materialize all of the values (so that it will submit everything to the executor) final ListenableFuture>> futures = GuavaUtils.allFuturesAsList( - queryables.peek( + queryables.map( + // Don't use peek here: https://github.com/apache/incubator-druid/pull/5913#discussion_r213472699 queryRunner -> { if (queryRunner == null) { throw new ISE("Null queryRunner! Looks to be some segment unmapping action happening"); } + return queryRunner; } ).map( queryRunner -> new AbstractPrioritizedCallable>(priority) @@ -154,35 +153,18 @@ public Iterable call() queryWatcher.registerQuery(query, futures); try { - final DateTime deadline; + final List> result; if (QueryContexts.hasTimeout(query)) { - deadline = DateTimes.nowUtc().plusMillis((int) QueryContexts.getTimeout(query)); + result = Execs.futureManagedBlockGet( + futures, + DateTimes.nowUtc().plusMillis((int) QueryContexts.getTimeout(query)) + ); } else { - deadline = DateTimes.utc(JodaUtils.MAX_INSTANT); + result = Execs.futureManagedBlockGet(futures); } - ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() - { - @Override - public boolean block() throws InterruptedException - { - try { - futures.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS); - } - catch (ExecutionException | TimeoutException e) { - // Will get caught later - } - return true; - } - - @Override - public boolean isReleasable() - { - return futures.isDone() || deadline.isBefore(DateTimes.nowUtc()); - } - }); return new MergeIterable<>( ordering.nullsFirst(), - futures.get(JodaUtils.timeoutForDeadline(deadline), TimeUnit.MILLISECONDS) + result ).iterator(); } catch (InterruptedException e) { diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index f7263dfc3d30..4248b685ca2d 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -324,7 +324,7 @@ Stream> run(final UnaryOperator materializedSegments = segments.collect(Collectors.toList()); segments = materializedSegments.stream(); @@ -704,10 +704,13 @@ private Stream>> deserializeFromCache( // Query granularity in a segment may be higher fidelity than the segment as a file, // so this might have multiple results return StreamSupport - .stream(Spliterators.spliteratorUnknownSize( - objectMapper.readValues(objectMapper.getFactory().createParser(cachedResult), cacheObjectClazz), - 0 - ), false) + .stream( + Spliterators.spliteratorUnknownSize( + objectMapper.readValues(objectMapper.getFactory().createParser(cachedResult), cacheObjectClazz), + 0 + ), + false + ) .map(pullFromCacheFunction) .map(obj -> Pair.of(cachedResultPair.getLhs(), Optional.ofNullable(obj))); } diff --git a/server/src/main/java/io/druid/client/DirectDruidClient.java b/server/src/main/java/io/druid/client/DirectDruidClient.java index 93edd4c85eeb..71ffc370fcc0 100644 --- a/server/src/main/java/io/druid/client/DirectDruidClient.java +++ b/server/src/main/java/io/druid/client/DirectDruidClient.java @@ -36,6 +36,7 @@ import io.druid.java.util.common.Pair; import io.druid.java.util.common.RE; import io.druid.java.util.common.StringUtils; +import io.druid.java.util.common.concurrent.Execs; import io.druid.java.util.common.guava.BaseSequence; import io.druid.java.util.common.guava.CloseQuietly; import io.druid.java.util.common.guava.Sequence; @@ -84,7 +85,6 @@ import java.util.concurrent.CancellationException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.Future; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeUnit; @@ -591,28 +591,7 @@ private void init() { if (jp == null) { try { - // Safety for if we are in a FJP - ForkJoinPool.managedBlock(new ForkJoinPool.ManagedBlocker() - { - @Override - public boolean block() throws InterruptedException - { - try { - future.get(); - } - catch (ExecutionException e) { - // Ignore, will be caught when get is called below - } - return true; - } - - @Override - public boolean isReleasable() - { - return future.isDone(); - } - }); - InputStream is = future.get(); + final InputStream is = Execs.futureManagedBlockGet(future); if (is == null) { throw new QueryInterruptedException( new ResourceLimitExceededException( From 4028a22f4a98048dfbecbb99a99a422f61d4b287 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 10:36:20 -0700 Subject: [PATCH 50/58] Add more examples --- .../io/druid/guice/LifecycleForkJoinPool.java | 2 +- .../druid/guice/ForkJoinPoolProviderTest.java | 61 +++++++++++++++++++ 2 files changed, 62 insertions(+), 1 deletion(-) create mode 100644 processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java diff --git a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java index 960a340f1816..79dab1b8ddc5 100644 --- a/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java +++ b/processing/src/main/java/io/druid/guice/LifecycleForkJoinPool.java @@ -45,7 +45,7 @@ public void stop() LOG.info("Shutting down ForkJoinPool [%s]", this); shutdown(); try { - // Should this be configurable? + // This should be configurable https://github.com/apache/incubator-druid/issues/6264 if (!awaitTermination(1, TimeUnit.MINUTES)) { LOG.warn("Failed to complete all tasks in FJP [%s]", this); } diff --git a/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java b/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java new file mode 100644 index 000000000000..0380f042d285 --- /dev/null +++ b/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.guice; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ForkJoinPool; + +public class ForkJoinPoolProviderTest +{ + private static final String GOOD_NAME_FORMAT = "test-fjp-%d"; + private static final ForkJoinPoolProvider FORK_JOIN_POOL_PROVIDER = new ForkJoinPoolProvider(GOOD_NAME_FORMAT); + + @Test + public void testThreadThrowsException() throws InterruptedException + { + final ForkJoinPool fjp = FORK_JOIN_POOL_PROVIDER.get(); + final RuntimeException re = new RuntimeException("test exception"); + try { + fjp.submit(() -> { + throw re; + }).get(); + } + catch (ExecutionException e) { + if (!re.equals(e.getCause().getCause())) { + throw new RuntimeException("Unexpected exception", e); + } + return; + } + Assert.fail("Should have thrown exception"); + } + + @Test + public void testThreadSwallowsException() + { + final ForkJoinPool fjp = FORK_JOIN_POOL_PROVIDER.get(); + final RuntimeException re = new RuntimeException("test exception"); + fjp.execute(() -> { + throw re; + }); + } +} \ No newline at end of file From 66f6af114644eb06039cd91560e1d06b0c3482fd Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 10:36:30 -0700 Subject: [PATCH 51/58] newline --- .../src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java b/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java index 0380f042d285..503cdcba3d8f 100644 --- a/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java +++ b/processing/src/test/java/io/druid/guice/ForkJoinPoolProviderTest.java @@ -58,4 +58,4 @@ public void testThreadSwallowsException() throw re; }); } -} \ No newline at end of file +} From b6ed828c1661018a88b807ace9b8c487d409ff3a Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 10:57:55 -0700 Subject: [PATCH 52/58] Super awesome mega array preallocation --- .../java/util/common/guava/MergeWorkTask.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index f9c9ae912c81..075927139f3a 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -22,10 +22,10 @@ import com.google.common.annotations.VisibleForTesting; import io.druid.java.util.common.Pair; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Deque; import java.util.Iterator; -import java.util.LinkedList; import java.util.List; import java.util.NoSuchElementException; import java.util.Spliterator; @@ -53,8 +53,8 @@ public class MergeWorkTask extends ForkJoinTask> * iteration * * @param mergerFn The function that will merge a stream of sequences into a single sequence. If the - * baseSequences stream is parallel, this work will be done in the FJP, otherwise it - * will be called directly. + * baseSequences stream is parallel, this work will be done in the FJP, otherwise it + * will be called directly. * @param baseSequences The sequences that need merged * @param batchSize The input stream should be split down to this number if possible. This sets the target number of segments per merge thread work * @param fjp The ForkJoinPool to do the intermediate merges in. @@ -76,12 +76,20 @@ public static Sequence parallelMerge( // Don't even try. return mergerFn.apply(baseSequences); } + if (batchSize < 1) { + throw new IllegalArgumentException("Batch size must be greater than 0"); + } @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); // Accumulate a list of forked off tasks final List>> tasks = new ArrayList<>(); - final Deque>> spliteratorStack = new LinkedList<>(); + final long totalResults = baseSpliterator.estimateSize(); + long dequeueInitialCapacity = totalResults / batchSize + 1; + if (dequeueInitialCapacity < 16) { + dequeueInitialCapacity = 16; + } + final Deque>> spliteratorStack = new ArrayDeque<>((int) dequeueInitialCapacity); // Push the base spliterator onto the stack, keep splitting until we can't or splits are small spliteratorStack.push(baseSpliterator); From ecd465cc7c82d63b0db3c58916c31fd7a6c470b4 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 12:37:00 -0700 Subject: [PATCH 53/58] Add java docs explaining fork/join mechanism --- .../java/util/common/guava/MergeWorkTask.java | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) diff --git a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java index 075927139f3a..09b6271681c3 100644 --- a/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java +++ b/java-util/src/main/java/io/druid/java/util/common/guava/MergeWorkTask.java @@ -79,6 +79,34 @@ public static Sequence parallelMerge( if (batchSize < 1) { throw new IllegalArgumentException("Batch size must be greater than 0"); } + + // At first glance this looks like an alternative implementation for a RecursiveTask because it does the following: + // 1. Divides the input work up into batches + // 2. Joins the results in a merging operation + // + // While these are true, there are some differences in this implementation and a raw RecursiveTask that are worth + // calling out. First, the results are fed into a BlockingQueue so that the final merge can accumulate as soon as + // the first intermediate result is available. This design constraint makes a RecursiveTask rather odd since the + // intended use case would have intermediate merges chain up to the top merge, rather than a single top merge + // accumulating the total results. This does not preclude a RecursiveAction that can feed the results into a + // blocking queue. + // + // But in such an implementation the total needed queue size is not known until all the recursive actions are + // forked off similar to the implementation here. The difference being the implementation below has a dedicated + // action submitted to the fjp for joining the result and feeding it into the result stream. Since this dedicated + // feeder work is submitted after all the tasks are launched, the total queue size needed is known ahead of time, + // and the blocking queue can be pre-allocated with the correct capacity to ensure submission to the queue never + // blocks. + // + // In addition, there exists an ability in this implementation to cancel all the forked tasks if the stream is + // closed (like on the case of query cancellation). + // + // Since there is a desire to + // 1. Ensure the intermediate results do not block when being fed into the final merge queue + // 2. Have the ability to cancel outstanding work tasks if the resulting Sequence is cancelled + // this implementation deviates from a straight up RecursiveTask or RecursiveAction implementation to attempt to + // provide an easy to follow and reason about workflow. + @SuppressWarnings("unchecked") // Wildcard erasure is fine here final Spliterator> baseSpliterator = (Spliterator>) baseSequences.spliterator(); @@ -87,6 +115,7 @@ public static Sequence parallelMerge( final long totalResults = baseSpliterator.estimateSize(); long dequeueInitialCapacity = totalResults / batchSize + 1; if (dequeueInitialCapacity < 16) { + // 16 is the default element count size in ArrayDeque at the time of this writing. dequeueInitialCapacity = 16; } final Deque>> spliteratorStack = new ArrayDeque<>((int) dequeueInitialCapacity); @@ -114,6 +143,8 @@ public static Sequence parallelMerge( // We guarantee enough space to put all the results so that the FJP doesn't block waiting for results to come in final BlockingQueue, Throwable>> readyForFinalMerge = new ArrayBlockingQueue<>(tasks.size()); + // Submit a simple feeder into the final merge queue. Since readyForFinalMerge is sized to the number of tasks, + // the readyForFinalMerge.add call should never block. tasks.forEach(task -> fjp.submit(() -> { try { readyForFinalMerge.add(Pair.of(task.join(), null)); @@ -139,7 +170,7 @@ public boolean hasNext() } @Override - public Sequence next() throws NoSuchElementException + public Sequence next() { if (taken >= totalAdditions) { throw new NoSuchElementException(); From 1398913c093535496a7806d5e32d0840a4a56655 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 12:44:46 -0700 Subject: [PATCH 54/58] Move some things to single line --- .../java/io/druid/client/CachingClusteredClient.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 4248b685ca2d..5c4f1eb7d081 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -222,15 +222,8 @@ private QueryRunner runAndMergeWithTimelineChange( }; } else { return (queryPlus, responseContext) -> { - final Stream> sequences = run( - queryPlus, - responseContext, - timelineConverter - ); - return new MergeSequence<>( - query.getResultOrdering(), - Sequences.fromStream(sequences) - ); + final Stream> sequences = run(queryPlus, responseContext, timelineConverter); + return new MergeSequence<>(query.getResultOrdering(), Sequences.fromStream(sequences)); }; } } From 0aca2973ff49dc04b211a44aacdf08cf5155f387 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 12:49:05 -0700 Subject: [PATCH 55/58] Code formatting --- .../java/io/druid/client/CachingClusteredClient.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 5c4f1eb7d081..51b1533319fc 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -337,10 +337,7 @@ Stream> run(final UnaryOperator>> cacheResolvedResults = deserializeFromCache( - maybeFetchCacheResults( - queryCacheKey, - segments - ) + maybeFetchCacheResults(queryCacheKey, segments) ); return groupCachedResultsByServer(cacheResolvedResults) .map(this::runOnServer) @@ -639,11 +636,7 @@ private ServerMaybeSegmentMaybeCache pickServer(Pair(ALREADY_CACHED_SERVER, null, null); } final DruidServer server = queryableDruidServer.getServer(); - return new ServerMaybeSegmentMaybeCache<>( - server, - serverToSegment.getSegmentDescriptor(), - null - ); + return new ServerMaybeSegmentMaybeCache<>(server, serverToSegment.getSegmentDescriptor(), null); } /** From 857f5484a5757f773ac4f6be86d5f78ab976747c Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Wed, 29 Aug 2018 12:51:53 -0700 Subject: [PATCH 56/58] Fix code formatting --- .../main/java/io/druid/client/CachingClusteredClient.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/main/java/io/druid/client/CachingClusteredClient.java b/server/src/main/java/io/druid/client/CachingClusteredClient.java index 51b1533319fc..9c5d137ebe4b 100644 --- a/server/src/main/java/io/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/io/druid/client/CachingClusteredClient.java @@ -195,11 +195,7 @@ private QueryRunner runAndMergeWithTimelineChange( final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(query); final QueryToolChest> toolChest = queryRunnerFactory.getToolchest(); return (queryPlus, responseContext) -> { - final Stream> sequences = run( - queryPlus, - responseContext, - timelineConverter - ); + final Stream> sequences = run(queryPlus, responseContext, timelineConverter); return MergeWorkTask.parallelMerge( sequences.parallel(), sequenceStream -> From 972304310fdf03423d05f1130e63e091ca5e0d38 Mon Sep 17 00:00:00 2001 From: Charles Allen Date: Thu, 4 Oct 2018 14:10:44 -0700 Subject: [PATCH 57/58] Get rid of divide by zero --- .../java/org/apache/druid/client/CachingClusteredClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index a233badb80a4..4ba3e7c4c819 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -345,7 +345,7 @@ Stream> run(final UnaryOperator Date: Thu, 4 Oct 2018 16:49:32 -0700 Subject: [PATCH 58/58] Start addressing comments --- .../org/apache/druid/query/QueryRunner.java | 5 + .../druid/client/CachingClusteredClient.java | 98 ++++++++++--------- 2 files changed, 55 insertions(+), 48 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/QueryRunner.java b/processing/src/main/java/org/apache/druid/query/QueryRunner.java index a7d62d4514bd..d2b6b53452d9 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/QueryRunner.java @@ -31,4 +31,9 @@ public interface QueryRunner * Runs the given query and returns results in a time-ordered sequence. */ Sequence run(QueryPlus queryPlus, Map responseContext); + + @SuppressWarnings("unchecked") + static QueryRunner of(Sequence s) { + return (ignored0, ignored1) -> (Sequence) s; + } } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 4ba3e7c4c819..6b87e356f405 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -168,11 +168,7 @@ public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, Data @Override public QueryRunner getQueryRunnerForIntervals(final Query query, final Iterable intervals) { - return runAndMergeWithTimelineChange( - query, - // No change, but Function.identity() doesn't work here for some reason - identity -> identity - ); + return runAndMergeWithTimelineChange(query, UnaryOperator.identity()); } /** @@ -186,7 +182,7 @@ Stream> run( final UnaryOperator> timelineConverter ) { - return new SpecificQueryRunnable<>(queryPlus, responseContext).run(timelineConverter); + return new SpecificQueryRunnable<>(queryPlus, responseContext).runByServer(timelineConverter); } private QueryRunner runAndMergeWithTimelineChange( @@ -203,16 +199,12 @@ private QueryRunner runAndMergeWithTimelineChange( final Stream> sequences = run(queryPlus, responseContext, timelineConverter); return MergeWorkTask.parallelMerge( sequences.parallel(), - sequenceStream -> + (Stream> sequenceStream) -> new FluentQueryRunnerBuilder<>(toolChest) .create( queryRunnerFactory.mergeRunners( mergeFjp, - sequenceStream.map( - s -> (QueryRunner) (ignored0, ignored1) -> (Sequence) s - ).collect( - Collectors.toList() - ) + sequenceStream.map(QueryRunner::of).collect(Collectors.toList()) ) ) .mergeResults() @@ -302,7 +294,15 @@ private Map makeDownstreamQueryContext() return Collections.unmodifiableMap(contextBuilder); } - Stream> run(final UnaryOperator> timelineConverter) + /** + * This is the main workflow for the query setup. The sequences are created but not accumulated here. + * + * @param timelineConverter Any manipulations to the timeline that need done + * + * @return A stream of the sequences. Each sequence is either a server result or the total cache result. A + * spliterator on the returned stream should be sized and subsized. + */ + Stream> runByServer(final UnaryOperator> timelineConverter) { @Nullable TimelineLookup timeline = serverView.getTimeline(query.getDataSource()); @@ -354,45 +354,12 @@ Stream> run(final UnaryOperator extractServerAndSegment(TimelineObjectHolder holder) - { - 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()) - )); - } - - private Stream computeSegmentsToQuery(TimelineLookup 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 timeline) { final List uncoveredIntervals = new ArrayList<>(uncoveredIntervalsLimit); @@ -434,6 +401,41 @@ private void computeUncoveredIntervals(TimelineLookup ti } } + /** + * Create a stream of the partition chunks which are relevant to 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 extractServerAndSegment(TimelineObjectHolder holder) + { + 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()) + )); + } + + private Stream computeSegmentsToQuery(TimelineLookup timeline) + { + return toolChest + .filterSegments( + query, + query.getIntervals().stream().flatMap(i -> timeline.lookup(i).stream()).collect(Collectors.toList()) + ) + .stream() + .flatMap(this::extractServerAndSegment) + .distinct(); + } + @Nullable private byte[] computeQueryCacheKey() {