From 6f51584916ffb23d471ea4e67e485b2140f1d6c1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 02:32:30 -0700 Subject: [PATCH 1/8] add CachingClusteredClient benchmark, refactor some stuff --- .../CachingClusteredClientBenchmark.java | 569 ++++++++++++++++++ .../CombiningFunction.java} | 9 +- .../druid/collections/CombiningIterable.java | 22 +- .../druid/collections/CombiningIterator.java | 7 +- .../collections/OrderedMergeIterator.java | 139 ----- .../collections/OrderedMergeSequence.java | 212 ------- .../druid/common/guava/CombiningSequence.java | 12 +- .../java/util/common/guava/BaseSequence.java | 6 +- .../guava/nary/SortedMergeIterator.java | 105 ---- .../collections/CombiningIteratorTest.java | 15 +- .../collections/OrderedMergeIteratorTest.java | 190 ------ .../collections/OrderedMergeSequenceTest.java | 350 ----------- .../common/guava/ComplexSequenceTest.java | 23 +- .../guava/nary/SortedMergeIteratorTest.java | 55 -- .../druid/indexer/DeterminePartitionsJob.java | 38 +- .../org/apache/druid/query/QueryRunner.java | 9 + .../apache/druid/query/QueryToolChest.java | 22 + .../druid/query/ResultMergeQueryRunner.java | 23 +- .../groupby/GroupByQueryQueryToolChest.java | 15 + .../epinephelinae/GroupByBinaryFnV2.java | 7 +- .../groupby/strategy/GroupByStrategy.java | 23 + .../groupby/strategy/GroupByStrategyV2.java | 33 +- .../SegmentMetadataQueryQueryToolChest.java | 13 +- .../druid/query/search/SearchBinaryFn.java | 5 +- .../search/SearchQueryQueryToolChest.java | 43 +- .../druid/query/select/SelectBinaryFn.java | 5 +- .../select/SelectQueryQueryToolChest.java | 46 +- .../query/timeseries/TimeseriesBinaryFn.java | 5 +- .../TimeseriesQueryQueryToolChest.java | 47 +- .../apache/druid/query/topn/TopNBinaryFn.java | 4 +- .../druid/query/topn/TopNQueryBuilder.java | 6 + .../query/topn/TopNQueryQueryToolChest.java | 53 +- .../collections/CombiningIterableTest.java | 36 +- .../apache/druid/client/BrokerServerView.java | 2 +- .../druid/client/DirectDruidClient.java | 3 +- .../client/selector/QueryableDruidServer.java | 21 +- .../selector/ServerSelectorStrategy.java | 2 +- ...va => WeightedServerSelectorStrategy.java} | 4 +- .../druid/client/DirectDruidClientTest.java | 22 +- .../selector/TierSelectorStrategyTest.java | 6 +- 40 files changed, 881 insertions(+), 1326 deletions(-) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java rename core/src/main/java/org/apache/druid/{java/util/common/guava/nary/BinaryFn.java => collections/CombiningFunction.java} (84%) delete mode 100644 core/src/main/java/org/apache/druid/collections/OrderedMergeIterator.java delete mode 100644 core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java delete mode 100644 core/src/main/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIterator.java delete mode 100644 core/src/test/java/org/apache/druid/collections/OrderedMergeIteratorTest.java delete mode 100644 core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java delete mode 100644 core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java rename server/src/main/java/org/apache/druid/client/selector/{ConnectionCountServerSelectorStrategy.java => WeightedServerSelectorStrategy.java} (91%) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java new file mode 100644 index 000000000000..7b83b48adc01 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -0,0 +1,569 @@ +/* + * 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 org.apache.druid.benchmark.query; + +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; +import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; +import org.apache.druid.benchmark.datagen.BenchmarkSchemas; +import org.apache.druid.benchmark.datagen.SegmentGenerator; +import org.apache.druid.client.CachingClusteredClient; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.TimelineServerView; +import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.client.cache.CachePopulatorStats; +import org.apache.druid.client.cache.ForegroundCachePopulator; +import org.apache.druid.client.cache.MapCache; +import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; +import org.apache.druid.client.selector.QueryableDruidServer; +import org.apache.druid.client.selector.RandomServerSelectorStrategy; +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.DefaultBlockingPool; +import org.apache.druid.collections.NonBlockingPool; +import org.apache.druid.collections.StupidPool; +import org.apache.druid.data.input.Row; +import org.apache.druid.guice.http.DruidHttpClientConfig; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.BySegmentQueryRunner; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; +import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.Druids; +import org.apache.druid.query.FinalizeResultsQueryRunner; +import org.apache.druid.query.FluentQueryRunnerBuilder; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryRunnerFactory; +import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryToolChestWarehouse; +import org.apache.druid.query.Result; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryEngine; +import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; +import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; +import org.apache.druid.query.groupby.GroupByQueryRunnerTest; +import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; +import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesQueryEngine; +import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; +import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.query.topn.TopNQueryConfig; +import org.apache.druid.query.topn.TopNQueryQueryToolChest; +import org.apache.druid.query.topn.TopNQueryRunnerFactory; +import org.apache.druid.query.topn.TopNResultValue; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.apache.druid.timeline.TimelineLookup; +import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.LinearShardSpec; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.concurrent.Executor; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1, jvmArgsAppend = "-XX:+UseG1GC") +@Warmup(iterations = 15) +@Measurement(iterations = 30) +public class CachingClusteredClientBenchmark +{ + @Param({"8"}) + private int numServers; + + @Param({"4", "2", "1"}) + private int numProcessingThreads; + + @Param({"75000"}) + private int rowsPerSegment; + + @Param({"all"}) + private String queryGranularity; + + private static final Logger log = new Logger(CachingClusteredClientBenchmark.class); + private static final String DATA_SOURCE = "ds"; + public static final ObjectMapper JSON_MAPPER; + + private QueryToolChestWarehouse toolChestWarehouse; + private QueryRunnerFactoryConglomerate conglomerate; + private CachingClusteredClient cachingClusteredClient; + private ExecutorService processingPool; + + private Query query; + + private final Closer closer = Closer.create(); + + private final BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + private final QuerySegmentSpec basicSchemaIntervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + + static { + JSON_MAPPER = new DefaultObjectMapper(); + JSON_MAPPER.setInjectableValues( + new Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), JSON_MAPPER) + .addValue(DataSegment.PruneLoadSpecHolder.class, DataSegment.PruneLoadSpecHolder.DEFAULT) + ); + } + + @Setup(Level.Trial) + public void setup() + { + final String schemaName = "basic"; + + BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + + Map queryableIndexes = new HashMap<>(numServers); + + for (int i = 0; i < numServers; i++) { + + final DataSegment dataSegment = DataSegment.builder() + .dataSource(DATA_SOURCE) + .interval(schemaInfo.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(i)) + .build(); + final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); + log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); + final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment); + queryableIndexes.put(dataSegment, index); + } + + final DruidProcessingConfig processingConfig = new DruidProcessingConfig() + { + @Override + public String getFormatString() + { + return null; + } + + @Override + public int intermediateComputeSizeBytes() + { + return 10 * 1024 * 1024; + } + + @Override + public int getNumMergeBuffers() + { + // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. + // Two buffers for the broker and one for the queryable + return 3; + } + + @Override + public int getNumThreads() + { + return numProcessingThreads; + } + }; + + conglomerate = new DefaultQueryRunnerFactoryConglomerate( + ImmutableMap., QueryRunnerFactory>builder() + .put( + TimeseriesQuery.class, + new TimeseriesQueryRunnerFactory( + new TimeseriesQueryQueryToolChest( + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() + ), + new TimeseriesQueryEngine(), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + TopNQuery.class, + new TopNQueryRunnerFactory( + new StupidPool<>( + "TopNQueryRunnerFactory-bufferPool", + () -> ByteBuffer.allocate(10 * 1024 * 1024) + ), + new TopNQueryQueryToolChest( + new TopNQueryConfig(), + QueryRunnerTestHelper.noopIntervalChunkingQueryRunnerDecorator() + ), + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ) + .put( + GroupByQuery.class, + makeGroupByQueryRunnerFactory( + GroupByQueryRunnerTest.DEFAULT_MAPPER, + new GroupByQueryConfig() + { + @Override + public String getDefaultStrategy() + { + return GroupByStrategySelector.STRATEGY_V2; + } + }, + processingConfig + ) + ) + .build() + ); + + toolChestWarehouse = new QueryToolChestWarehouse() + { + @Override + public > QueryToolChest getToolChest(final QueryType query) + { + return conglomerate.findFactory(query).getToolchest(); + } + }; + + SimpleServerView serverView = new SimpleServerView(); + int serverSuffx = 1; + for (Entry entry : queryableIndexes.entrySet()) { + serverView.addServer( + createServer(serverSuffx++), + entry.getKey(), + entry.getValue() + ); + } + + processingPool = Execs.multiThreaded(processingConfig.getNumThreads(), "caching-clustered-client-benchmark"); + cachingClusteredClient = new CachingClusteredClient( + toolChestWarehouse, + serverView, + MapCache.create(0), + JSON_MAPPER, + new ForegroundCachePopulator(JSON_MAPPER, new CachePopulatorStats(), 0), + new CacheConfig(), + new DruidHttpClientConfig() + ); + } + + private static GroupByQueryRunnerFactory makeGroupByQueryRunnerFactory( + final ObjectMapper mapper, + final GroupByQueryConfig config, + final DruidProcessingConfig processingConfig + ) + { + final Supplier configSupplier = Suppliers.ofInstance(config); + final Supplier bufferSupplier = + () -> ByteBuffer.allocateDirect(processingConfig.intermediateComputeSizeBytes()); + + final NonBlockingPool bufferPool = new StupidPool<>( + "GroupByQueryEngine-bufferPool", + bufferSupplier + ); + final BlockingPool mergeBufferPool = new DefaultBlockingPool<>( + bufferSupplier, + processingConfig.getNumMergeBuffers() + ); + final GroupByStrategySelector strategySelector = new GroupByStrategySelector( + configSupplier, + new GroupByStrategyV1( + configSupplier, + new GroupByQueryEngine(configSupplier, bufferPool), + QueryRunnerTestHelper.NOOP_QUERYWATCHER, + bufferPool + ), + new GroupByStrategyV2( + processingConfig, + configSupplier, + bufferPool, + mergeBufferPool, + mapper, + QueryRunnerTestHelper.NOOP_QUERYWATCHER + ) + ); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + strategySelector, + QueryRunnerTestHelper.sameThreadIntervalChunkingQueryRunnerDecorator() + ); + return new GroupByQueryRunnerFactory( + strategySelector, + toolChest + ); + } + + @TearDown(Level.Trial) + public void tearDown() throws IOException + { + closer.close(); + processingPool.shutdown(); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void timeseriesQuery(Blackhole blackhole) + { + query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(basicSchemaIntervalSpec) + .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .granularity(Granularity.fromString(queryGranularity)) + .build(); + + final List> results = runQuery(); + + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void topNQuery(Blackhole blackhole) + { + query = new TopNQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(basicSchemaIntervalSpec) + .dimension(new DefaultDimensionSpec("dimUniform", null)) + .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .granularity(Granularity.fromString(queryGranularity)) + .metric("sumLongSequential") + .threshold(20480) + .build(); + + final List> results = runQuery(); + + for (Result result : results) { + blackhole.consume(result); + } + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void groupByQuery(Blackhole blackhole) + { + query = GroupByQuery + .builder() + .setDataSource(DATA_SOURCE) + .setQuerySegmentSpec(basicSchemaIntervalSpec) + .setDimensions( + new DefaultDimensionSpec("dimUniform", null), + new DefaultDimensionSpec("dimZipf", null) + ) + .setAggregatorSpecs(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) + .setGranularity(Granularity.fromString(queryGranularity)) + .build(); + + final List results = runQuery(); + + for (Row result : results) { + blackhole.consume(result); + } + } + + private List runQuery() + { + //noinspection unchecked + QueryRunner theRunner = new FluentQueryRunnerBuilder<>(toolChestWarehouse.getToolChest(query)) + .create(cachingClusteredClient.getQueryRunnerForIntervals(query, query.getIntervals())) + .applyPreMergeDecoration() + .mergeResults() + .applyPostMergeDecoration(); + + //noinspection unchecked + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>()); + + return queryResult.toList(); + } + + private class SimpleServerView implements TimelineServerView + { + private final TierSelectorStrategy tierSelectorStrategy = new HighestPriorityTierSelectorStrategy( + new RandomServerSelectorStrategy() + ); + // server -> queryRunner + private final Map servers = new HashMap<>(); + // segmentId -> serverSelector + private final Map selectors = new HashMap<>(); + // dataSource -> version -> serverSelector + private final Map> timelines = new HashMap<>(); + + void addServer(DruidServer server, DataSegment dataSegment, QueryableIndex queryableIndex) + { + servers.put( + server, + new SingleSegmentDruidServer( + server, + new SimpleQueryRunner( + conglomerate, + dataSegment.getId(), + queryableIndex + ) + ) + ); + addSegmentToServer(server, dataSegment); + } + + void addSegmentToServer(DruidServer server, DataSegment segment) + { + final ServerSelector selector = selectors.computeIfAbsent( + segment.getId().toString(), + k -> new ServerSelector(segment, tierSelectorStrategy) + ); + selector.addServerAndUpdateSegment(servers.get(server), segment); + timelines.computeIfAbsent(segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural())) + .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); + } + + @Nullable + @Override + public TimelineLookup getTimeline(DataSource dataSource) + { + final String table = Iterables.getOnlyElement(dataSource.getNames()); + return timelines.get(table); + } + + @Override + public List getDruidServers() + { + return Collections.emptyList(); + } + + @Override + public QueryRunner getQueryRunner(DruidServer server) + { + final SingleSegmentDruidServer queryableDruidServer = Preconditions.checkNotNull(servers.get(server), "server"); + return (QueryRunner) queryableDruidServer.getQueryRunner(); + } + + @Override + public void registerTimelineCallback(Executor exec, TimelineCallback callback) + { + // do nothing + } + + @Override + public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback callback) + { + // do nothing + } + + @Override + public void registerSegmentCallback(Executor exec, SegmentCallback callback) + { + // do nothing + } + } + + private class SimpleQueryRunner implements QueryRunner + { + private final QueryRunnerFactoryConglomerate conglomerate; + private final QueryableIndexSegment segment; + + public SimpleQueryRunner(QueryRunnerFactoryConglomerate conglomerate, SegmentId segmentId, QueryableIndex queryableIndex) + { + this.conglomerate = conglomerate; + this.segment = new QueryableIndexSegment(queryableIndex, segmentId); + } + + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + final QueryRunnerFactory factory = conglomerate.findFactory(queryPlus.getQuery()); + //noinspection unchecked + return factory.getToolchest().preMergeQueryDecoration( + new FinalizeResultsQueryRunner<>( + new BySegmentQueryRunner<>( + segment.getId(), + segment.getDataInterval().getStart(), + factory.createRunner(segment) + ), + factory.getToolchest() + ) + ).run(queryPlus, responseContext); + } + } + + private class SingleSegmentDruidServer extends QueryableDruidServer + { + SingleSegmentDruidServer(DruidServer server, SimpleQueryRunner runner) + { + super(server, runner); + } + } + + private static DruidServer createServer(int nameSuiffix) + { + return new DruidServer( + "server_" + nameSuiffix, + "127.0.0." + nameSuiffix, + null, + 10240L, + ServerType.HISTORICAL, + "default", + 0 + ); + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/nary/BinaryFn.java b/core/src/main/java/org/apache/druid/collections/CombiningFunction.java similarity index 84% rename from core/src/main/java/org/apache/druid/java/util/common/guava/nary/BinaryFn.java rename to core/src/main/java/org/apache/druid/collections/CombiningFunction.java index a89125a948a8..93d1390e2312 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/nary/BinaryFn.java +++ b/core/src/main/java/org/apache/druid/collections/CombiningFunction.java @@ -17,11 +17,10 @@ * under the License. */ -package org.apache.druid.java.util.common.guava.nary; +package org.apache.druid.collections; -/** - */ -public interface BinaryFn +@FunctionalInterface +public interface CombiningFunction { - OutType apply(Type1 arg1, Type2 arg2); + T apply(T arg1, T arg2); } diff --git a/core/src/main/java/org/apache/druid/collections/CombiningIterable.java b/core/src/main/java/org/apache/druid/collections/CombiningIterable.java index 08c169d27956..69eb48162117 100644 --- a/core/src/main/java/org/apache/druid/collections/CombiningIterable.java +++ b/core/src/main/java/org/apache/druid/collections/CombiningIterable.java @@ -20,7 +20,6 @@ package org.apache.druid.collections; import org.apache.druid.java.util.common.guava.MergeIterable; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import java.util.Comparator; import java.util.Iterator; @@ -51,16 +50,11 @@ public static CombiningIterable createSplatted( return create( new MergeIterable(comparator, (Iterable>) in), comparator, - new BinaryFn() - { - @Override - public InType apply(InType arg1, InType arg2) - { - if (arg1 == null) { - return arg2; - } - return arg1; + (arg1, arg2) -> { + if (arg1 == null) { + return arg2; } + return arg1; } ); } @@ -68,20 +62,20 @@ public InType apply(InType arg1, InType arg2) public static CombiningIterable create( Iterable it, Comparator comparator, - BinaryFn fn + CombiningFunction fn ) { - return new CombiningIterable(it, comparator, fn); + return new CombiningIterable<>(it, comparator, fn); } private final Iterable it; private final Comparator comparator; - private final BinaryFn fn; + private final CombiningFunction fn; public CombiningIterable( Iterable it, Comparator comparator, - BinaryFn fn + CombiningFunction fn ) { this.it = it; diff --git a/core/src/main/java/org/apache/druid/collections/CombiningIterator.java b/core/src/main/java/org/apache/druid/collections/CombiningIterator.java index 8ae59907f812..32ec4d45f6a9 100644 --- a/core/src/main/java/org/apache/druid/collections/CombiningIterator.java +++ b/core/src/main/java/org/apache/druid/collections/CombiningIterator.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import java.util.Comparator; import java.util.Iterator; @@ -34,7 +33,7 @@ public class CombiningIterator implements Iterator public static CombiningIterator create( Iterator it, Comparator comparator, - BinaryFn fn + CombiningFunction fn ) { return new CombiningIterator(it, comparator, fn); @@ -42,12 +41,12 @@ public static CombiningIterator create( private final PeekingIterator it; private final Comparator comparator; - private final BinaryFn fn; + private final CombiningFunction fn; public CombiningIterator( Iterator it, Comparator comparator, - BinaryFn fn + CombiningFunction fn ) { this.it = Iterators.peekingIterator(it); diff --git a/core/src/main/java/org/apache/druid/collections/OrderedMergeIterator.java b/core/src/main/java/org/apache/druid/collections/OrderedMergeIterator.java deleted file mode 100644 index bb2cea426e58..000000000000 --- a/core/src/main/java/org/apache/druid/collections/OrderedMergeIterator.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * 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 org.apache.druid.collections; - -import com.google.common.base.Function; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterators; -import com.google.common.collect.PeekingIterator; -import org.apache.druid.java.util.common.guava.FunctionalIterator; - -import java.util.Comparator; -import java.util.Iterator; -import java.util.NoSuchElementException; -import java.util.PriorityQueue; - -/** - * An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming - * that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators. - * - * Normally a merge operation could operate with the actual input iterators in any order as long as the actual values - * in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators - * be provided in the order of the first element of each iterator. - * - * If this doesn't make sense, check out OrderedMergeIteratorTest.testScrewsUpOnOutOfOrderBeginningOfList() - * - * It places this extra restriction on the input data in order to implement an optimization that allows it to - * remain as lazy as possible in the face of a common case where the iterators are just appended one after the other. - */ -public class OrderedMergeIterator implements Iterator -{ - private final PriorityQueue> firstElementComparedPQueue; - - private PeekingIterator> iterOfIterators; - private final Comparator comparator; - - public OrderedMergeIterator( - final Comparator comparator, - Iterator> iterators - ) - { - this.comparator = comparator; - firstElementComparedPQueue = new PriorityQueue>( - 16, - new Comparator>() - { - @Override - public int compare(PeekingIterator lhs, PeekingIterator rhs) - { - return comparator.compare(lhs.peek(), rhs.peek()); - } - } - ); - - iterOfIterators = Iterators.peekingIterator( - FunctionalIterator.create(iterators) - .filter( - new Predicate>() - { - @Override - public boolean apply(Iterator input) - { - return input.hasNext(); - } - } - ) - .transform( - new Function, PeekingIterator>() - { - @Override - public PeekingIterator apply(Iterator input) - { - return Iterators.peekingIterator(input); - } - } - ) - ); - } - - @Override - public boolean hasNext() - { - return !firstElementComparedPQueue.isEmpty() || iterOfIterators.hasNext(); - } - - @Override - public T next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - final PeekingIterator littleIt; - if (!iterOfIterators.hasNext()) { - littleIt = firstElementComparedPQueue.remove(); - } else if (firstElementComparedPQueue.isEmpty()) { - littleIt = iterOfIterators.next(); - } else { - T pQueueValue = firstElementComparedPQueue.peek().peek(); - T iterItersValue = iterOfIterators.peek().peek(); - - if (comparator.compare(pQueueValue, iterItersValue) <= 0) { - littleIt = firstElementComparedPQueue.remove(); - } else { - littleIt = iterOfIterators.next(); - } - } - - T retVal = littleIt.next(); - - if (littleIt.hasNext()) { - firstElementComparedPQueue.add(littleIt); - } - - return retVal; - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } -} diff --git a/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java b/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java deleted file mode 100644 index 7c3f7aa4b39a..000000000000 --- a/core/src/main/java/org/apache/druid/collections/OrderedMergeSequence.java +++ /dev/null @@ -1,212 +0,0 @@ -/* - * 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 org.apache.druid.collections; - -import com.google.common.base.Function; -import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.guava.Accumulator; -import org.apache.druid.java.util.common.guava.CloseQuietly; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; -import org.apache.druid.java.util.common.guava.YieldingAccumulator; -import org.apache.druid.java.util.common.guava.YieldingAccumulators; -import org.apache.druid.java.util.common.io.Closer; - -import java.io.IOException; -import java.util.PriorityQueue; - -/** - * An OrderedMergeIterator is an iterator that merges together multiple sorted iterators. It is written assuming - * that the input Iterators are provided in order. That is, it places an extra restriction in the input iterators. - * - * Normally a merge operation could operate with the actual input iterators in any order as long as the actual values - * in the iterators are sorted. This requires that not only the individual values be sorted, but that the iterators - * be provided in the order of the first element of each iterator. - * - * If this doesn't make sense, check out OrderedMergeSequenceTest.testScrewsUpOnOutOfOrderBeginningOfList() - * - * It places this extra restriction on the input data in order to implement an optimization that allows it to - * remain as lazy as possible in the face of a common case where the iterators are just appended one after the other. - */ -public class OrderedMergeSequence implements Sequence -{ - private final Ordering ordering; - private final Sequence> sequences; - - public OrderedMergeSequence( - final Ordering ordering, - Sequence> sequences - ) - { - this.ordering = ordering; - this.sequences = sequences; - } - - @Override - public OutType accumulate(OutType initValue, Accumulator accumulator) - { - Yielder yielder = null; - try { - yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); - return yielder.get(); - } - finally { - CloseQuietly.close(yielder); - } - } - - @Override - public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) - { - PriorityQueue> pQueue = new PriorityQueue>( - 32, - ordering.onResultOf( - new Function, T>() - { - @Override - public T apply(Yielder input) - { - return input.get(); - } - } - ) - ); - - Yielder> oldDudeAtCrosswalk = sequences.toYielder( - null, - new YieldingAccumulator, Sequence>() - { - @Override - public Yielder accumulate(Yielder accumulated, Sequence in) - { - final Yielder retVal = in.toYielder( - null, - new YieldingAccumulator() - { - @Override - public T accumulate(T accumulated, T in) - { - yield(); - return in; - } - } - ); - - if (retVal.isDone()) { - try { - retVal.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - return null; - } else { - yield(); - } - - return retVal; - } - } - ); - - return makeYielder(pQueue, oldDudeAtCrosswalk, initValue, accumulator); - } - - private Yielder makeYielder( - final PriorityQueue> pQueue, - Yielder> oldDudeAtCrosswalk, - OutType initVal, - final YieldingAccumulator accumulator - ) - { - OutType retVal = initVal; - while (!accumulator.yielded() && (!pQueue.isEmpty() || !oldDudeAtCrosswalk.isDone())) { - Yielder yielder; - if (oldDudeAtCrosswalk.isDone()) { - yielder = pQueue.remove(); - } else if (pQueue.isEmpty()) { - yielder = oldDudeAtCrosswalk.get(); - oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } else { - Yielder queueYielder = pQueue.peek(); - Yielder iterYielder = oldDudeAtCrosswalk.get(); - - if (ordering.compare(queueYielder.get(), iterYielder.get()) <= 0) { - yielder = pQueue.remove(); - } else { - yielder = oldDudeAtCrosswalk.get(); - oldDudeAtCrosswalk = oldDudeAtCrosswalk.next(null); - } - } - - retVal = accumulator.accumulate(retVal, yielder.get()); - yielder = yielder.next(null); - if (yielder.isDone()) { - try { - yielder.close(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } else { - pQueue.add(yielder); - } - } - - if (!accumulator.yielded()) { - return Yielders.done(retVal, oldDudeAtCrosswalk); - } - - final OutType yieldVal = retVal; - final Yielder> finalOldDudeAtCrosswalk = oldDudeAtCrosswalk; - return new Yielder() - { - @Override - public OutType get() - { - return yieldVal; - } - - @Override - public Yielder next(OutType initValue) - { - accumulator.reset(); - return makeYielder(pQueue, finalOldDudeAtCrosswalk, initValue, accumulator); - } - - @Override - public boolean isDone() - { - return false; - } - - @Override - public void close() throws IOException - { - Closer closer = Closer.create(); - while (!pQueue.isEmpty()) { - closer.register(pQueue.remove()); - } - closer.close(); - } - }; - } -} diff --git a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java index 33dd06421dcb..f3eba7edaf32 100644 --- a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java +++ b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java @@ -20,12 +20,12 @@ package org.apache.druid.common.guava; import com.google.common.collect.Ordering; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.guava.YieldingAccumulator; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import java.io.IOException; @@ -36,7 +36,7 @@ public class CombiningSequence implements Sequence public static CombiningSequence create( Sequence baseSequence, Ordering ordering, - BinaryFn mergeFn + CombiningFunction mergeFn ) { return new CombiningSequence<>(baseSequence, ordering, mergeFn); @@ -44,12 +44,12 @@ public static CombiningSequence create( private final Sequence baseSequence; private final Ordering ordering; - private final BinaryFn mergeFn; + private final CombiningFunction mergeFn; private CombiningSequence( Sequence baseSequence, Ordering ordering, - BinaryFn mergeFn + CombiningFunction mergeFn ) { this.baseSequence = baseSequence; @@ -148,7 +148,7 @@ public void close() throws IOException private static class CombiningYieldingAccumulator extends YieldingAccumulator { private final Ordering ordering; - private final BinaryFn mergeFn; + private final CombiningFunction mergeFn; private final YieldingAccumulator accumulator; private OutType retVal; @@ -157,7 +157,7 @@ private static class CombiningYieldingAccumulator extends YieldingAc CombiningYieldingAccumulator( Ordering ordering, - BinaryFn mergeFn, + CombiningFunction mergeFn, YieldingAccumulator accumulator ) { diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java index a11d742342f0..f1184c49adcc 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java @@ -40,9 +40,11 @@ public BaseSequence( public OutType accumulate(OutType initValue, final Accumulator fn) { IterType iterator = maker.make(); + OutType accumulated = initValue; + try { while (iterator.hasNext()) { - initValue = fn.accumulate(initValue, iterator.next()); + accumulated = fn.accumulate(accumulated, iterator.next()); } } catch (Throwable t) { @@ -55,7 +57,7 @@ public OutType accumulate(OutType initValue, final Accumulator implements Iterator -{ - public static SortedMergeIterator create( - Iterator lhs, - Iterator rhs, - Comparator comparator, - BinaryFn fn - ) - { - return new SortedMergeIterator<>(lhs, rhs, comparator, fn); - } - - private final PeekingIterator lhs; - private final PeekingIterator rhs; - private final Comparator comparator; - private final BinaryFn fn; - - public SortedMergeIterator( - Iterator lhs, - Iterator rhs, - Comparator comparator, - BinaryFn fn - ) - { - this.lhs = Iterators.peekingIterator(lhs); - this.rhs = Iterators.peekingIterator(rhs); - this.comparator = comparator; - this.fn = fn; - } - - @Override - public boolean hasNext() - { - return lhs.hasNext() || rhs.hasNext(); - } - - @Override - public OutType next() - { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - if (!lhs.hasNext()) { - return fn.apply(null, rhs.next()); - } - if (!rhs.hasNext()) { - return fn.apply(lhs.next(), null); - } - - int compared = comparator.compare(lhs.peek(), rhs.peek()); - - if (compared < 0) { - return fn.apply(lhs.next(), null); - } - if (compared == 0) { - return fn.apply(lhs.next(), rhs.next()); - } - - return fn.apply(null, rhs.next()); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } -} diff --git a/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java b/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java index 985c156ae791..57a2290b605c 100644 --- a/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java +++ b/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.collections; import com.google.common.collect.PeekingIterator; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; @@ -34,7 +33,7 @@ public class CombiningIteratorTest { private CombiningIterator testingIterator; private Comparator comparator; - private BinaryFn binaryFn; + private CombiningFunction combiningFunction; private PeekingIterator peekIterator; @Before @@ -42,8 +41,8 @@ public void setUp() { peekIterator = EasyMock.createMock(PeekingIterator.class); comparator = EasyMock.createMock(Comparator.class); - binaryFn = EasyMock.createMock(BinaryFn.class); - testingIterator = CombiningIterator.create(peekIterator, comparator, binaryFn); + combiningFunction = EasyMock.createMock(CombiningFunction.class); + testingIterator = CombiningIterator.create(peekIterator, comparator, combiningFunction); } @After @@ -84,19 +83,19 @@ public void testNext() String defaultString = "S1"; String resString = "S2"; EasyMock.expect(peekIterator.next()).andReturn(defaultString); - EasyMock.expect(binaryFn.apply(EasyMock.eq(defaultString), EasyMock.isNull())) + EasyMock.expect(combiningFunction.apply(EasyMock.eq(defaultString), EasyMock.isNull())) .andReturn(resString); EasyMock.expect(peekIterator.next()).andReturn(defaultString); EasyMock.expect(comparator.compare(EasyMock.eq(resString), EasyMock.eq(defaultString))) .andReturn(0); EasyMock.expect(peekIterator.next()).andReturn(defaultString); - EasyMock.expect(binaryFn.apply(EasyMock.eq(resString), EasyMock.eq(defaultString))) + EasyMock.expect(combiningFunction.apply(EasyMock.eq(resString), EasyMock.eq(defaultString))) .andReturn(resString); EasyMock.expect(comparator.compare(EasyMock.eq(resString), EasyMock.eq(defaultString))) .andReturn(1); EasyMock.replay(peekIterator); - EasyMock.replay(binaryFn); + EasyMock.replay(combiningFunction); EasyMock.replay(comparator); String actual = testingIterator.next(); @@ -104,7 +103,7 @@ public void testNext() EasyMock.verify(peekIterator); EasyMock.verify(comparator); - EasyMock.verify(binaryFn); + EasyMock.verify(combiningFunction); } @Test(expected = NoSuchElementException.class) diff --git a/core/src/test/java/org/apache/druid/collections/OrderedMergeIteratorTest.java b/core/src/test/java/org/apache/druid/collections/OrderedMergeIteratorTest.java deleted file mode 100644 index e58b83b4e1a5..000000000000 --- a/core/src/test/java/org/apache/druid/collections/OrderedMergeIteratorTest.java +++ /dev/null @@ -1,190 +0,0 @@ -/* - * 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 org.apache.druid.collections; - -import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.NoSuchElementException; - -/** - */ -public class OrderedMergeIteratorTest -{ - @Test - public void testSanity() - { - final ArrayList> iterators = new ArrayList<>(); - iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator()); - iterators.add(Arrays.asList(2, 8).iterator()); - iterators.add(Arrays.asList(4, 6, 8).iterator()); - - OrderedMergeIterator iter = new OrderedMergeIterator( - Ordering.natural(), - iterators.iterator() - ); - - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9), Lists.newArrayList(iter)); - } - - @Test - public void testScrewsUpOnOutOfOrderBeginningOfList() - { - final ArrayList> iterators = new ArrayList<>(); - iterators.add(Arrays.asList(1, 3, 5, 7, 9).iterator()); - iterators.add(Arrays.asList(4, 6).iterator()); - iterators.add(Arrays.asList(2, 8).iterator()); - - OrderedMergeIterator iter = new OrderedMergeIterator( - Ordering.natural(), - iterators.iterator() - ); - - Assert.assertEquals(Arrays.asList(1, 3, 4, 2, 5, 6, 7, 8, 9), Lists.newArrayList(iter)); - } - - @Test - public void testScrewsUpOnOutOfOrderInList() - { - final ArrayList> iterators = new ArrayList<>(); - iterators.add(Arrays.asList(1, 3, 5, 4, 7, 9).iterator()); - iterators.add(Arrays.asList(2, 8).iterator()); - iterators.add(Arrays.asList(4, 6).iterator()); - - OrderedMergeIterator iter = new OrderedMergeIterator( - Ordering.natural(), - iterators.iterator() - ); - - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9), Lists.newArrayList(iter)); - } - - @Test - public void testLaziness() - { - final boolean[] done = new boolean[]{false, false}; - - final ArrayList> iterators = new ArrayList<>(); - iterators.add( - new IteratorShell(Arrays.asList(1, 2, 3).iterator()) - { - @Override - public boolean hasNext() - { - boolean retVal = super.hasNext(); - if (!retVal) { - done[0] = true; - } - return retVal; - } - } - ); - iterators.add( - new IteratorShell(Arrays.asList(4, 5, 6).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - boolean retVal = super.hasNext(); - if (!retVal) { - done[1] = true; - } - return retVal; - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - ++count; - return super.next(); - } - } - ); - - iterators.add( - new IteratorShell(Arrays.asList(7, 8, 9).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - return super.hasNext(); - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - ++count; - return super.next(); - } - } - ); - - OrderedMergeIterator iter = new OrderedMergeIterator( - Ordering.natural(), - iterators.iterator() - ); - - Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9), Lists.newArrayList(iter)); - } - - @Test(expected = NoSuchElementException.class) - public void testNoElementInNext() - { - OrderedMergeIterator iter = new OrderedMergeIterator<>( - Ordering.natural(), - Collections.emptyIterator() - ); - iter.next(); - } - - @Test(expected = UnsupportedOperationException.class) - public void testRemove() - { - OrderedMergeIterator iter = new OrderedMergeIterator<>( - Ordering.natural(), - Collections.emptyIterator() - ); - iter.remove(); - } -} diff --git a/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java b/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java deleted file mode 100644 index 741ac7d6e2cb..000000000000 --- a/core/src/test/java/org/apache/druid/collections/OrderedMergeSequenceTest.java +++ /dev/null @@ -1,350 +0,0 @@ -/* - * 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 org.apache.druid.collections; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.guava.BaseSequence; -import org.apache.druid.java.util.common.guava.MergeSequence; -import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.SequenceTestHelper; -import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.TestSequence; -import org.junit.Assert; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; - -/** - */ -public class OrderedMergeSequenceTest -{ - @Test - public void testSanity() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 7, 9), - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testMergeEmptySequence() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(ImmutableList.of()), - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testMergeEmptySequenceAtEnd() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(2, 8), - TestSequence.create(4, 6, 8), - TestSequence.create(ImmutableList.of()) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - - @Test - public void testMergeEmptySequenceMiddle() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(2, 8), - TestSequence.create(ImmutableList.of()), - TestSequence.create(4, 6, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(2, 4, 6, 8, 8)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testScrewsUpOnOutOfOrderBeginningOfList() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 7, 9), - TestSequence.create(4, 6, 8), - TestSequence.create(2, 8) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 3, 4, 2, 5, 6, 7, 8, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testScrewsUpOnOutOfOrderInList() throws Exception - { - final ArrayList> testSequences = Lists.newArrayList( - TestSequence.create(1, 3, 5, 4, 7, 9), - TestSequence.create(2, 8), - TestSequence.create(4, 6) - ); - - OrderedMergeSequence seq = makeMergedSequence(Ordering.natural(), testSequences); - - SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9)); - - for (TestSequence sequence : testSequences) { - Assert.assertTrue(sequence.isClosed()); - } - } - - @Test - public void testLazinessAccumulation() - { - final ArrayList> sequences = makeSyncedSequences(); - OrderedMergeSequence seq = new OrderedMergeSequence( - Ordering.natural(), Sequences.simple(sequences) - ); - SequenceTestHelper.testAccumulation("", seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)); - } - - @Test - public void testLazinessYielder() throws Exception - { - final ArrayList> sequences = makeSyncedSequences(); - OrderedMergeSequence seq = new OrderedMergeSequence( - Ordering.natural(), Sequences.simple(sequences) - ); - SequenceTestHelper.testYield("", seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9)); - } - - private ArrayList> makeSyncedSequences() - { - final boolean[] done = new boolean[]{false, false}; - - final ArrayList> sequences = new ArrayList<>(); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return Arrays.asList(1, 2, 3).iterator(); - } - - @Override - public void cleanup(Iterator iterFromMake) - { - done[0] = true; - } - } - ) - ); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return new IteratorShell(Arrays.asList(4, 5, 6).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - return super.hasNext(); - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("First iterator not complete", done[0]); - } - ++count; - return super.next(); - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - done[1] = true; - } - } - ) - ); - sequences.add( - new BaseSequence>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() - { - return new IteratorShell(Arrays.asList(7, 8, 9).iterator()) - { - int count = 0; - - @Override - public boolean hasNext() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - return super.hasNext(); - } - - @Override - public Integer next() - { - if (count >= 1) { - Assert.assertTrue("Second iterator not complete", done[1]); - } - Assert.assertTrue("First iterator not complete", done[0]); - ++count; - return super.next(); - } - }; - } - - @Override - public void cleanup(Iterator iterFromMake) - { - } - } - ) - ); - return sequences; - } - - private OrderedMergeSequence makeMergedSequence( - Ordering ordering, - List> seqs - ) - { - return new OrderedMergeSequence( - ordering, - Sequences.simple((List>) (List) seqs) - ); - } - - private MergeSequence makeUnorderedMergedSequence( - Ordering ordering, - List> seqs - ) - { - return new MergeSequence(ordering, Sequences.simple(seqs)); - } - - @Test - public void testHierarchicalMerge() throws Exception - { - final Sequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - - final Sequence seq2 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - final OrderedMergeSequence finalMerged = new OrderedMergeSequence( - Ordering.natural(), - Sequences.simple( - Lists.newArrayList(seq1, seq2) - ) - ); - - SequenceTestHelper.testAll(finalMerged, Arrays.asList(1, 1)); - } - - @Test - public void testMergeMerge() throws Exception - { - final Sequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - final OrderedMergeSequence finalMerged = new OrderedMergeSequence( - Ordering.natural(), - Sequences.simple( - Collections.singletonList(seq1) - ) - ); - - SequenceTestHelper.testAll(finalMerged, Collections.singletonList(1)); - } - - @Test - public void testOne() throws Exception - { - final MergeSequence seq1 = makeUnorderedMergedSequence( - Ordering.natural(), - Collections.singletonList(TestSequence.create(1)) - ); - - SequenceTestHelper.testAll(seq1, Collections.singletonList(1)); - } -} diff --git a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java index 74879b339885..835afce0604c 100644 --- a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java @@ -20,12 +20,12 @@ package org.apache.druid.common.guava; import com.google.common.primitives.Ints; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.YieldingAccumulator; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.junit.Assert; import org.junit.Test; @@ -88,20 +88,15 @@ private Sequence concat(Sequence... sequences) return Sequences.concat(Arrays.asList(sequences)); } - private final BinaryFn plus = new BinaryFn() - { - @Override - public Integer apply(Integer arg1, Integer arg2) - { - if (arg1 == null) { - return arg2; - } - - if (arg2 == null) { - return arg1; - } + private final CombiningFunction plus = (arg1, arg2) -> { + if (arg1 == null) { + return arg2; + } - return arg1 + arg2; + if (arg2 == null) { + return arg1; } + + return arg1 + arg2; }; } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java deleted file mode 100644 index 97953dab2c6c..000000000000 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/nary/SortedMergeIteratorTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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 org.apache.druid.java.util.common.guava.nary; - -import com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; - -import java.util.Arrays; -import java.util.Comparator; - -/** - */ -public class SortedMergeIteratorTest -{ - @Test - public void testSanity() - { - SortedMergeIterator iter = SortedMergeIterator.create( - Arrays.asList(1, 4, 5, 7, 9).iterator(), - Arrays.asList(1, 2, 3, 6, 7, 8, 9, 10, 11).iterator(), - Comparator.naturalOrder(), - new BinaryFn() - { - @Override - public Integer apply(Integer arg1, Integer arg2) - { - return arg1 == null ? arg2 : arg2 == null ? arg1 : arg1 + arg2; - } - } - ); - - Assert.assertEquals( - Arrays.asList(2, 2, 3, 4, 5, 6, 14, 8, 18, 10, 11), - Lists.newArrayList(iter) - ); - } -} diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index 115a926c2d22..57f593af22c5 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -20,7 +20,6 @@ package org.apache.druid.indexer; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.base.Function; import com.google.common.base.Joiner; import com.google.common.base.Optional; import com.google.common.base.Splitter; @@ -39,7 +38,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; @@ -73,7 +71,6 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; @@ -573,36 +570,17 @@ private Iterable combineRows(Iterable input) return new CombiningIterable<>( Iterables.transform( input, - new Function() - { - @Override - public DimValueCount apply(Text input) - { - return DimValueCount.fromText(input); - } - } + input1 -> DimValueCount.fromText(input1) ), - new Comparator() - { - @Override - public int compare(DimValueCount o1, DimValueCount o2) - { - return ComparisonChain.start().compare(o1.dim, o2.dim).compare(o1.value, o2.value).result(); + (o1, o2) -> ComparisonChain.start().compare(o1.dim, o2.dim).compare(o1.value, o2.value).result(), + (arg1, arg2) -> { + if (arg2 == null) { + return arg1; } - }, - new BinaryFn() - { - @Override - public DimValueCount apply(DimValueCount arg1, DimValueCount arg2) - { - if (arg2 == null) { - return arg1; - } - // Respect "poisoning" (negative values mean we can't use this dimension) - final long newNumRows = (arg1.numRows >= 0 && arg2.numRows >= 0 ? arg1.numRows + arg2.numRows : -1); - return new DimValueCount(arg1.dim, arg1.value, newNumRows); - } + // Respect "poisoning" (negative values mean we can't use this dimension) + final long newNumRows = (arg1.numRows >= 0 && arg2.numRows >= 0 ? arg1.numRows + arg2.numRows : -1); + return new DimValueCount(arg1.dim, arg1.value, newNumRows); } ); } 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..53b38a237c7c 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,13 @@ public interface QueryRunner * Runs the given query and returns results in a time-ordered sequence. */ Sequence run(QueryPlus queryPlus, Map responseContext); + + /** + * Get 'weight' for a QueryRunner, e.g. the number of open connections, as some measure of how heavily utilized a + * query runner is to aid in server selection to decide where to run queries. + */ + default int getWeight() + { + return 0; + } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index 81f59e43f2e5..b50a7f1cfbf0 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -23,7 +23,10 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.type.TypeFactory; import com.google.common.base.Function; +import com.google.common.collect.Ordering; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.guice.annotations.ExtensionPoint; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.timeline.LogicalSegment; @@ -83,6 +86,25 @@ public final JavaType getBySegmentResultType() */ public abstract QueryRunner mergeResults(QueryRunner runner); + /** + * Creates a merge function that is used to merge intermediate aggregates from historicals in broker. This merge + * function is used in {@link ResultMergeQueryRunner} and can be used in additional future merge + * implementations + */ + public CombiningFunction createMergeFn(Query query) + { + throw new UOE("%s doesn't support merge function", query.getClass().getName()); + } + + /** + * Creates an ordering comparator that is used to order results. This ordering function is used in + * {@link ResultMergeQueryRunner} + */ + public Ordering createOrderingFn(Query query) + { + throw new UOE("%s doesn't support ordering function", query.getClass().getName()); + } + /** * Creates a {@link QueryMetrics} object that is used to generate metrics for this specific query type. This exists * to allow for query-specific dimensions and metrics. That is, the ToolChest is expected to set some diff --git a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java index 444aadffb6c2..efbdf0cc5190 100644 --- a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java @@ -20,33 +20,40 @@ package org.apache.druid.query; import com.google.common.collect.Ordering; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import java.util.Map; +import java.util.function.Function; /** */ @PublicApi -public abstract class ResultMergeQueryRunner extends BySegmentSkippingQueryRunner +public class ResultMergeQueryRunner extends BySegmentSkippingQueryRunner { + private final Function, Ordering> orderingFnGenerator; + private final Function, CombiningFunction> mergeFnGenerator; + public ResultMergeQueryRunner( - QueryRunner baseRunner + QueryRunner baseRunner, + Function, Ordering> orderingFnGenerator, + Function, CombiningFunction> mergeFnGenerator ) { super(baseRunner); + this.orderingFnGenerator = orderingFnGenerator; + this.mergeFnGenerator = mergeFnGenerator; } @Override public Sequence doRun(QueryRunner baseRunner, QueryPlus queryPlus, Map context) { Query query = queryPlus.getQuery(); - return CombiningSequence.create(baseRunner.run(queryPlus, context), makeOrdering(query), createMergeFn(query)); + return CombiningSequence.create( + baseRunner.run(queryPlus, context), + orderingFnGenerator.apply(query), + mergeFnGenerator.apply(query)); } - - protected abstract Ordering makeOrdering(Query query); - - protected abstract BinaryFn createMergeFn(Query query); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index e7d1e27efb43..ccec22e87996 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -30,7 +30,9 @@ 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.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.ISE; @@ -41,6 +43,7 @@ import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; @@ -132,6 +135,18 @@ public Sequence run(QueryPlus queryPlus, Map responseC }; } + @Override + public CombiningFunction createMergeFn(Query query) + { + return strategySelector.strategize((GroupByQuery) query).createMergeFn(query); + } + + @Override + public Ordering createOrderingFn(Query query) + { + return strategySelector.strategize((GroupByQuery) query).createOrderingFn(query); + } + private Sequence initAndMergeGroupByResults( final GroupByQuery query, QueryRunner runner, diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java index 983ffc727144..e0e0d18a243a 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java @@ -20,10 +20,10 @@ package org.apache.druid.query.groupby.epinephelinae; import com.google.common.collect.Maps; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.granularity.AllGranularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; @@ -31,7 +31,7 @@ import java.util.Map; -public class GroupByBinaryFnV2 implements BinaryFn +public class GroupByBinaryFnV2 implements CombiningFunction { private final GroupByQuery query; @@ -50,8 +50,7 @@ public Row apply(final Row arg1, final Row arg2) } final Map newMap = Maps.newHashMapWithExpectedSize( - query.getDimensions().size() - + query.getAggregatorSpecs().size() + query.getDimensions().size() + query.getAggregatorSpecs().size() ); // Add dimensions diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index eec4d6f6d6d7..5198a9043f46 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -19,10 +19,14 @@ package org.apache.druid.query.groupby.strategy; +import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.Row; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.groupby.GroupByQuery; @@ -30,6 +34,7 @@ import org.apache.druid.query.groupby.resource.GroupByQueryResource; import org.apache.druid.segment.StorageAdapter; +import javax.annotation.Nullable; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -69,6 +74,24 @@ QueryRunner createIntervalChunkingRunner( Sequence mergeResults(QueryRunner baseRunner, GroupByQuery query, Map responseContext); + /** + * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)}. + */ + @Nullable + default CombiningFunction createMergeFn(Query query) + { + throw new UOE("%s doesn't support merge function", this.getClass().getName()); + } + + /** + * See {@link org.apache.druid.query.QueryToolChest#createOrderingFn(Query)}. + */ + @Nullable + default Ordering createOrderingFn(Query queryParam) + { + throw new UOE("%s doesn't support merge function", this.getClass().getName()); + } + Sequence applyPostProcessing(Sequence results, GroupByQuery query); Sequence processSubqueryResult( diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index cd6988392e6c..2f64f76209f9 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -30,6 +30,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; import org.apache.druid.collections.BlockingPool; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.data.input.MapBasedRow; @@ -43,7 +44,6 @@ import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.DataSource; import org.apache.druid.query.DruidProcessingConfig; import org.apache.druid.query.InsufficientResourcesException; @@ -213,6 +213,18 @@ public QueryRunner createIntervalChunkingRunner( return runner; } + @Override + public Ordering createOrderingFn(Query queryParam) + { + return ((GroupByQuery) queryParam).getRowOrdering(true); + } + + @Override + public CombiningFunction createMergeFn(Query queryParam) + { + return new GroupByBinaryFnV2((GroupByQuery) queryParam); + } + @Override public Sequence mergeResults( final QueryRunner baseRunner, @@ -222,20 +234,11 @@ public Sequence mergeResults( { // Merge streams using ResultMergeQueryRunner, then apply postaggregators, then apply limit (which may // involve materialization) - final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner(baseRunner) - { - @Override - protected Ordering makeOrdering(Query queryParam) - { - return ((GroupByQuery) queryParam).getRowOrdering(true); - } - - @Override - protected BinaryFn createMergeFn(Query queryParam) - { - return new GroupByBinaryFnV2((GroupByQuery) queryParam); - } - }; + final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( + baseRunner, + this::createOrderingFn, + this::createMergeFn + ); // Fudge timestamp, maybe. final DateTime fudgeTimestamp = getUniversalTimestamp(query); diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index 2813000ab760..cab0ce336328 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -30,6 +30,7 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.JodaUtils; @@ -37,7 +38,6 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.MappedSequence; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.BySegmentSkippingQueryRunner; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; @@ -137,16 +137,9 @@ private Ordering makeOrdering(SegmentMetadataQuery query) return query.getResultOrdering(); // No two elements should be equal, so it should never merge } - private BinaryFn createMergeFn(final SegmentMetadataQuery inQ) + private CombiningFunction createMergeFn(final SegmentMetadataQuery inQ) { - return new BinaryFn() - { - @Override - public SegmentAnalysis apply(SegmentAnalysis arg1, SegmentAnalysis arg2) - { - return mergeAnalyses(arg1, arg2, inQ.isLenientAggregatorMerge()); - } - }; + return (arg1, arg2) -> mergeAnalyses(arg1, arg2, inQ.isLenientAggregatorMerge()); } }; } diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java b/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java index 31e954d7af9e..f11e98c6e7e4 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java @@ -21,9 +21,9 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.Result; import org.joda.time.DateTime; @@ -32,8 +32,7 @@ /** */ -public class SearchBinaryFn - implements BinaryFn, Result, Result> +public class SearchBinaryFn implements CombiningFunction> { private final SearchSortSpec searchSortSpec; private final Granularity gran; diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index 8f35b2541bd8..af5c9ef736cf 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -29,12 +29,12 @@ import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.google.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; @@ -99,26 +99,29 @@ public QueryRunner> mergeResults( QueryRunner> runner ) { - return new ResultMergeQueryRunner>(runner) - { - @Override - protected Ordering> makeOrdering(Query> query) - { - return ResultGranularTimestampComparator.create( - ((SearchQuery) query).getGranularity(), - query.isDescending() - ); - } + return new ResultMergeQueryRunner<>( + runner, + this::createOrderingFn, + this::createMergeFn + ); + } - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - SearchQuery query = (SearchQuery) input; - return new SearchBinaryFn(query.getSort(), query.getGranularity(), query.getLimit()); - } - }; + @Override + public CombiningFunction> createMergeFn( + Query> query + ) + { + final SearchQuery searchQuery = (SearchQuery) query; + return new SearchBinaryFn(searchQuery.getSort(), searchQuery.getGranularity(), searchQuery.getLimit()); + } + + @Override + public Ordering> createOrderingFn(Query> query) + { + return ResultGranularTimestampComparator.create( + query.getGranularity(), + query.isDescending() + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java b/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java index 534cd42f0398..b11b44b5da3c 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java @@ -20,9 +20,9 @@ package org.apache.druid.query.select; import com.google.common.collect.Sets; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.Result; import org.joda.time.DateTime; @@ -31,8 +31,7 @@ /** */ -public class SelectBinaryFn - implements BinaryFn, Result, Result> +public class SelectBinaryFn implements CombiningFunction> { private final Granularity gran; private final PagingSpec pagingSpec; diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index 9a5fb5a79a97..fc3d8050aebc 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -28,11 +28,11 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Sequence; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; @@ -104,29 +104,29 @@ public QueryRunner> mergeResults( QueryRunner> queryRunner ) { - return new ResultMergeQueryRunner>(queryRunner) - { - @Override - protected Ordering> makeOrdering(Query> query) - { - return ResultGranularTimestampComparator.create( - ((SelectQuery) query).getGranularity(), query.isDescending() - ); - } + return new ResultMergeQueryRunner<>(queryRunner, this::createOrderingFn, this::createMergeFn); + } - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - SelectQuery query = (SelectQuery) input; - return new SelectBinaryFn( - query.getGranularity(), - query.getPagingSpec(), - query.isDescending() - ); - } - }; + @Override + public CombiningFunction> createMergeFn( + Query> query + ) + { + final SelectQuery selectQuery = (SelectQuery) query; + return new SelectBinaryFn( + selectQuery.getGranularity(), + selectQuery.getPagingSpec(), + selectQuery.isDescending() + ); + } + + @Override + public Ordering> createOrderingFn(Query> query) + { + return ResultGranularTimestampComparator.create( + query.getGranularity(), + query.isDescending() + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java index 9aafdaba7a14..bda96807dfcf 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java @@ -19,9 +19,9 @@ package org.apache.druid.query.timeseries; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -31,8 +31,7 @@ /** */ -public class TimeseriesBinaryFn - implements BinaryFn, Result, Result> +public class TimeseriesBinaryFn implements CombiningFunction> { private final Granularity gran; private final List aggregations; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 686ed6824ef7..d706377b8ceb 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -29,13 +29,13 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; @@ -99,7 +99,10 @@ public QueryRunner> mergeResults( ) { final QueryRunner> resultMergeQueryRunner = new ResultMergeQueryRunner>( - queryRunner) + queryRunner, + this::createOrderingFn, + this::createMergeFn + ) { @Override public Sequence> doRun( @@ -120,26 +123,6 @@ public Sequence> doRun( } return result; } - - @Override - protected Ordering> makeOrdering(Query> query) - { - return ResultGranularTimestampComparator.create( - ((TimeseriesQuery) query).getGranularity(), query.isDescending() - ); - } - - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - TimeseriesQuery query = (TimeseriesQuery) input; - return new TimeseriesBinaryFn( - query.getGranularity(), - query.getAggregatorSpecs() - ); - } }; return (queryPlus, responseContext) -> { @@ -211,6 +194,26 @@ protected BinaryFn, Result, }; } + @Override + public CombiningFunction> createMergeFn( + Query> query + ) + { + TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; + return new TimeseriesBinaryFn( + timeseriesQuery.getGranularity(), + timeseriesQuery.getAggregatorSpecs() + ); + } + + @Override + public Ordering> createOrderingFn(Query> query) + { + return ResultGranularTimestampComparator.create( + query.getGranularity(), query.isDescending() + ); + } + private Result getNullTimeseriesResultValue(TimeseriesQuery query) { List aggregatorSpecs = query.getAggregatorSpecs(); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java b/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java index 99acecbb33cb..ea29bff51150 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java @@ -19,9 +19,9 @@ package org.apache.druid.query.topn; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; @@ -36,7 +36,7 @@ /** */ -public class TopNBinaryFn implements BinaryFn, Result, Result> +public class TopNBinaryFn implements CombiningFunction> { private final DimensionSpec dimSpec; private final Granularity gran; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java index e140a20453e9..93667b06cc09 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java @@ -253,6 +253,12 @@ public TopNQueryBuilder aggregators(List a) return this; } + public TopNQueryBuilder aggregators(AggregatorFactory... aggs) + { + aggregatorSpecs = Arrays.asList(aggs); + return this; + } + public TopNQueryBuilder postAggregators(Collection p) { postAggregatorSpecs = new ArrayList<>(p); // defensive copy diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 53af0908c6d8..fc55bc94caf1 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -27,11 +27,11 @@ import com.google.common.collect.Maps; import com.google.common.collect.Ordering; import com.google.inject.Inject; +import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.BySegmentResultValue; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; @@ -113,32 +113,31 @@ public QueryRunner> mergeResults( QueryRunner> runner ) { - return new ResultMergeQueryRunner>(runner) - { - @Override - protected Ordering> makeOrdering(Query> query) - { - return ResultGranularTimestampComparator.create( - ((TopNQuery) query).getGranularity(), query.isDescending() - ); - } + return new ResultMergeQueryRunner<>(runner, this::createOrderingFn, this::createMergeFn); + } - @Override - protected BinaryFn, Result, Result> createMergeFn( - Query> input - ) - { - TopNQuery query = (TopNQuery) input; - return new TopNBinaryFn( - query.getGranularity(), - query.getDimensionSpec(), - query.getTopNMetricSpec(), - query.getThreshold(), - query.getAggregatorSpecs(), - query.getPostAggregatorSpecs() - ); - } - }; + @Override + public CombiningFunction> createMergeFn( + Query> query + ) + { + TopNQuery topNQuery = (TopNQuery) query; + return new TopNBinaryFn( + topNQuery.getGranularity(), + topNQuery.getDimensionSpec(), + topNQuery.getTopNMetricSpec(), + topNQuery.getThreshold(), + topNQuery.getAggregatorSpecs(), + topNQuery.getPostAggregatorSpecs() + ); + } + + @Override + public Ordering> createOrderingFn(Query> query) + { + return ResultGranularTimestampComparator.create( + ((TopNQuery) query).getGranularity(), query.isDescending() + ); } @Override @@ -582,7 +581,7 @@ public Result apply(Result input) BySegmentResultValue> value = (BySegmentResultValue>) input .getValue(); - return new Result( + return new Result<>( input.getTimestamp(), new BySegmentTopNResultValue( Lists.transform( diff --git a/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java b/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java index 83ac6d657e5f..497ddec08dd2 100644 --- a/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java +++ b/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java @@ -20,14 +20,12 @@ package org.apache.druid.collections; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.guava.nary.BinaryFn; import org.apache.druid.query.Result; import org.junit.Assert; import org.junit.Test; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -49,32 +47,20 @@ public void testMerge() Iterable> resultsAfter = CombiningIterable.create( resultsBefore, - new Comparator>() - { - @Override - public int compare(Result r1, Result r2) - { - return r1.getTimestamp().compareTo(r2.getTimestamp()); + (r1, r2) -> r1.getTimestamp().compareTo(r2.getTimestamp()), + (arg1, arg2) -> { + if (arg1 == null) { + return arg2; } - }, - new BinaryFn, Result, Result>() - { - @Override - public Result apply(final Result arg1, final Result arg2) - { - if (arg1 == null) { - return arg2; - } - if (arg2 == null) { - return arg1; - } - - return new Result( - arg1.getTimestamp(), - ((Long) arg1.getValue()).longValue() + ((Long) arg2.getValue()).longValue() - ); + if (arg2 == null) { + return arg1; } + + return new Result<>( + arg1.getTimestamp(), + ((Long) arg1.getValue()).longValue() + ((Long) arg2.getValue()).longValue() + ); } ); diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 7df034a0b6e9..fd784c802048 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -326,7 +326,7 @@ public QueryRunner getQueryRunner(DruidServer server) log.error("WTF?! No QueryableDruidServer found for %s", server.getName()); return null; } - return queryableDruidServer.getClient(); + return queryableDruidServer.getQueryRunner(); } } diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 0db333533ad1..89d2b3abeb6f 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -135,7 +135,8 @@ public DirectDruidClient( this.openConnections = new AtomicInteger(); } - public int getNumOpenConnections() + @Override + public int getWeight() { return openConnections.get(); } diff --git a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java index b98f1518b147..9da2a04ec5b1 100644 --- a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java @@ -19,20 +19,20 @@ package org.apache.druid.client.selector; -import org.apache.druid.client.DirectDruidClient; import org.apache.druid.client.DruidServer; +import org.apache.druid.query.QueryRunner; /** */ -public class QueryableDruidServer +public class QueryableDruidServer { private final DruidServer server; - private final DirectDruidClient client; + private final T queryRunner; - public QueryableDruidServer(DruidServer server, DirectDruidClient client) + public QueryableDruidServer(DruidServer server, T queryRunner) { this.server = server; - this.client = client; + this.queryRunner = queryRunner; } public DruidServer getServer() @@ -40,9 +40,14 @@ public DruidServer getServer() return server; } - public DirectDruidClient getClient() + public T getQueryRunner() { - return client; + return queryRunner; + } + + public int getWeight() + { + return queryRunner.getWeight(); } @Override @@ -50,7 +55,7 @@ public String toString() { return "QueryableDruidServer{" + "server=" + server + - ", client=" + client + + ", queryRunner=" + queryRunner + '}'; } } diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java index 79110d39a4ae..80b68f511e97 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java @@ -29,7 +29,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RandomServerSelectorStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "random", value = RandomServerSelectorStrategy.class), - @JsonSubTypes.Type(name = "connectionCount", value = ConnectionCountServerSelectorStrategy.class) + @JsonSubTypes.Type(name = "connectionCount", value = WeightedServerSelectorStrategy.class) }) public interface ServerSelectorStrategy { diff --git a/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java b/server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java similarity index 91% rename from server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java rename to server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java index fc401c424e3c..338888aaa002 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java +++ b/server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java @@ -28,10 +28,10 @@ import java.util.List; import java.util.Set; -public class ConnectionCountServerSelectorStrategy implements ServerSelectorStrategy +public class WeightedServerSelectorStrategy implements ServerSelectorStrategy { private static final Comparator COMPARATOR = - Comparator.comparingInt(s -> s.getClient().getNumOpenConnections()); + Comparator.comparingInt(QueryableDruidServer::getWeight); @Override public QueryableDruidServer pick(Set servers, DataSegment segment) diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java index 65a8cd62c44f..6ad302425113 100644 --- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java @@ -23,10 +23,10 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; -import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy; import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.client.selector.WeightedServerSelectorStrategy; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -129,7 +129,7 @@ public void testRun() throws Exception 0, 0L ), - new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( @@ -168,20 +168,20 @@ public void testRun() throws Exception Assert.assertTrue(capturedRequest.hasCaptured()); Assert.assertEquals(url, capturedRequest.getValue().getUrl()); Assert.assertEquals(HttpMethod.POST, capturedRequest.getValue().getMethod()); - Assert.assertEquals(1, client1.getNumOpenConnections()); + Assert.assertEquals(1, client1.getWeight()); // simulate read timeout client1.run(QueryPlus.wrap(query), defaultContext); - Assert.assertEquals(2, client1.getNumOpenConnections()); + Assert.assertEquals(2, client1.getWeight()); futureException.setException(new ReadTimeoutException()); - Assert.assertEquals(1, client1.getNumOpenConnections()); + Assert.assertEquals(1, client1.getWeight()); // subsequent connections should work client1.run(QueryPlus.wrap(query), defaultContext); client1.run(QueryPlus.wrap(query), defaultContext); client1.run(QueryPlus.wrap(query), defaultContext); - Assert.assertTrue(client1.getNumOpenConnections() == 4); + Assert.assertTrue(client1.getWeight() == 4); // produce result for first connection futureResult.set( @@ -192,12 +192,12 @@ public void testRun() throws Exception List results = s1.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals(DateTimes.of("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); - Assert.assertEquals(3, client1.getNumOpenConnections()); + Assert.assertEquals(3, client1.getWeight()); client2.run(QueryPlus.wrap(query), defaultContext); client2.run(QueryPlus.wrap(query), defaultContext); - Assert.assertTrue(client2.getNumOpenConnections() == 2); + Assert.assertTrue(client2.getWeight() == 2); Assert.assertTrue(serverSelector.pick() == queryableDruidServer2); @@ -247,7 +247,7 @@ public void testCancel() 0, 0L ), - new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( @@ -271,7 +271,7 @@ public void testCancel() cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); Sequence results = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod()); - Assert.assertEquals(0, client1.getNumOpenConnections()); + Assert.assertEquals(0, client1.getWeight()); QueryInterruptedException exception = null; @@ -318,7 +318,7 @@ public void testQueryInterruptionExceptionLogMessage() ); final ServerSelector serverSelector = new ServerSelector( dataSegment, - new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( diff --git a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java index ce7fe8aac86a..6bef7aab1baf 100644 --- a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java +++ b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java @@ -54,7 +54,7 @@ public void testHighestPriorityTierSelectorStrategy() ); testTierSelectorStrategy( - new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()), + new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()), highPriority, lowPriority ); } @@ -73,7 +73,7 @@ public void testLowestPriorityTierSelectorStrategy() ); testTierSelectorStrategy( - new LowestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()), + new LowestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()), lowPriority, highPriority ); } @@ -97,7 +97,7 @@ public void testCustomPriorityTierSelectorStrategy() testTierSelectorStrategy( new CustomTierSelectorStrategy( - new ConnectionCountServerSelectorStrategy(), + new WeightedServerSelectorStrategy(), new CustomTierSelectorStrategyConfig() { @Override From 73e8e5163d68a5daeb9b1845a144860ca0570c70 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 16:04:36 -0700 Subject: [PATCH 2/8] revert WeightedServerSelectorStrategy to ConnectionCountServerSelectorStrategy and remove getWeight since felt artificial, default mergeResults in toolchest implementation for topn, search, select --- .../org/apache/druid/query/QueryRunner.java | 9 ---- .../apache/druid/query/QueryToolChest.java | 18 +++++--- .../search/SearchQueryQueryToolChest.java | 13 ------ .../select/SelectQueryQueryToolChest.java | 9 ---- .../query/topn/TopNQueryQueryToolChest.java | 9 ---- .../apache/druid/client/BrokerServerView.java | 42 +++++++------------ .../druid/client/DirectDruidClient.java | 3 +- ...onnectionCountServerSelectorStrategy.java} | 5 ++- .../selector/ServerSelectorStrategy.java | 2 +- .../druid/client/DirectDruidClientTest.java | 22 +++++----- .../selector/TierSelectorStrategyTest.java | 6 +-- 11 files changed, 47 insertions(+), 91 deletions(-) rename server/src/main/java/org/apache/druid/client/selector/{WeightedServerSelectorStrategy.java => ConnectionCountServerSelectorStrategy.java} (87%) 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 53b38a237c7c..a7d62d4514bd 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/QueryRunner.java @@ -31,13 +31,4 @@ public interface QueryRunner * Runs the given query and returns results in a time-ordered sequence. */ Sequence run(QueryPlus queryPlus, Map responseContext); - - /** - * Get 'weight' for a QueryRunner, e.g. the number of open connections, as some measure of how heavily utilized a - * query runner is to aid in server selection to decide where to run queries. - */ - default int getWeight() - { - return 0; - } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index b50a7f1cfbf0..c2a9ea37a202 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -80,16 +80,24 @@ public final JavaType getBySegmentResultType() * ResultType objects in time order (ascending or descending). This method should return a new QueryRunner that * potentially merges the stream of ordered ResultType objects. * + * A default implementation constructs a {@link ResultMergeQueryRunner} which creates a + * {@link org.apache.druid.common.guava.CombiningSequence} using the supplied {@link QueryRunner} with + * {@link QueryToolChest#createOrderingFn(Query)} and {@link QueryToolChest#createMergeFn(Query)}} supplied by this + * toolchest. + * * @param runner A QueryRunner that provides a series of ResultType objects in time order (ascending or descending) * * @return a QueryRunner that potentially merges the stream of ordered ResultType objects */ - public abstract QueryRunner mergeResults(QueryRunner runner); + public QueryRunner mergeResults(QueryRunner runner) + { + return new ResultMergeQueryRunner<>(runner, this::createOrderingFn, this::createMergeFn); + } /** * Creates a merge function that is used to merge intermediate aggregates from historicals in broker. This merge - * function is used in {@link ResultMergeQueryRunner} and can be used in additional future merge - * implementations + * function is used in the default {@link ResultMergeQueryRunner} provided by + * {@link QueryToolChest#mergeResults(QueryRunner)} and can be used in additional future merge implementations */ public CombiningFunction createMergeFn(Query query) { @@ -97,8 +105,8 @@ public CombiningFunction createMergeFn(Query query) } /** - * Creates an ordering comparator that is used to order results. This ordering function is used in - * {@link ResultMergeQueryRunner} + * Creates an ordering comparator that is used to order results. This ordering function is used in the defaul + * {@link ResultMergeQueryRunner} provided by {@link QueryToolChest#mergeResults(QueryRunner)} */ public Ordering createOrderingFn(Query query) { diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index af5c9ef736cf..d635efbf4d65 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -44,7 +44,6 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.Result; import org.apache.druid.query.ResultGranularTimestampComparator; -import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; @@ -94,18 +93,6 @@ public SearchQueryQueryToolChest( this.queryMetricsFactory = queryMetricsFactory; } - @Override - public QueryRunner> mergeResults( - QueryRunner> runner - ) - { - return new ResultMergeQueryRunner<>( - runner, - this::createOrderingFn, - this::createMergeFn - ); - } - @Override public CombiningFunction> createMergeFn( Query> query diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index fc3d8050aebc..732fb9787d03 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -41,7 +41,6 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.Result; import org.apache.druid.query.ResultGranularTimestampComparator; -import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.DimFilter; @@ -99,14 +98,6 @@ public SelectQueryQueryToolChest( this.queryMetricsFactory = queryMetricsFactory; } - @Override - public QueryRunner> mergeResults( - QueryRunner> queryRunner - ) - { - return new ResultMergeQueryRunner<>(queryRunner, this::createOrderingFn, this::createMergeFn); - } - @Override public CombiningFunction> createMergeFn( Query> query diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index fc55bc94caf1..0307a8adb729 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -42,7 +42,6 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.Result; import org.apache.druid.query.ResultGranularTimestampComparator; -import org.apache.druid.query.ResultMergeQueryRunner; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; import org.apache.druid.query.aggregation.MetricManipulationFn; @@ -108,14 +107,6 @@ private static List prunePostAggregators(TopNQuery query) ); } - @Override - public QueryRunner> mergeResults( - QueryRunner> runner - ) - { - return new ResultMergeQueryRunner<>(runner, this::createOrderingFn, this::createMergeFn); - } - @Override public CombiningFunction> createMergeFn( Query> query diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index fd784c802048..2745be95ee96 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -108,25 +108,18 @@ public BrokerServerView( this.selectors = new HashMap<>(); this.timelines = new HashMap<>(); - this.segmentFilter = new Predicate>() - { - @Override - public boolean apply( - Pair input - ) - { - if (segmentWatcherConfig.getWatchedTiers() != null - && !segmentWatcherConfig.getWatchedTiers().contains(input.lhs.getTier())) { - return false; - } - - if (segmentWatcherConfig.getWatchedDataSources() != null - && !segmentWatcherConfig.getWatchedDataSources().contains(input.rhs.getDataSource())) { - return false; - } + this.segmentFilter = metadataAndSegment -> { + if (segmentWatcherConfig.getWatchedTiers() != null + && !segmentWatcherConfig.getWatchedTiers().contains(metadataAndSegment.lhs.getTier())) { + return false; + } - return true; + if (segmentWatcherConfig.getWatchedDataSources() != null + && !segmentWatcherConfig.getWatchedDataSources().contains(metadataAndSegment.rhs.getDataSource())) { + return false; } + + return true; }; ExecutorService exec = Execs.singleThreaded("BrokerServerView-%s"); baseView.registerSegmentCallback( @@ -160,14 +153,9 @@ public CallbackAction segmentViewInitialized() baseView.registerServerRemovedCallback( exec, - new ServerRemovedCallback() - { - @Override - public ServerView.CallbackAction serverRemoved(DruidServer server) - { - removeServer(server); - return ServerView.CallbackAction.CONTINUE; - } + server -> { + removeServer(server); + return CallbackAction.CONTINUE; } ); } @@ -195,10 +183,10 @@ public void awaitInitialization() throws InterruptedException private QueryableDruidServer addServer(DruidServer server) { - QueryableDruidServer retVal = new QueryableDruidServer(server, makeDirectClient(server)); + QueryableDruidServer retVal = new QueryableDruidServer<>(server, makeDirectClient(server)); QueryableDruidServer exists = clients.put(server.getName(), retVal); if (exists != null) { - log.warn("QueryRunner for server[%s] already existed!? Well it's getting replaced", server); + log.warn("QueryRunner for server[%s] already exists!? Well it's getting replaced", server); } return retVal; diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 89d2b3abeb6f..0db333533ad1 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -135,8 +135,7 @@ public DirectDruidClient( this.openConnections = new AtomicInteger(); } - @Override - public int getWeight() + public int getNumOpenConnections() { return openConnections.get(); } diff --git a/server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java b/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java similarity index 87% rename from server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java rename to server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java index 338888aaa002..542d608e41c8 100644 --- a/server/src/main/java/org/apache/druid/client/selector/WeightedServerSelectorStrategy.java +++ b/server/src/main/java/org/apache/druid/client/selector/ConnectionCountServerSelectorStrategy.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; +import org.apache.druid.client.DirectDruidClient; import org.apache.druid.timeline.DataSegment; import java.util.Collections; @@ -28,10 +29,10 @@ import java.util.List; import java.util.Set; -public class WeightedServerSelectorStrategy implements ServerSelectorStrategy +public class ConnectionCountServerSelectorStrategy implements ServerSelectorStrategy { private static final Comparator COMPARATOR = - Comparator.comparingInt(QueryableDruidServer::getWeight); + Comparator.comparingInt(s -> ((DirectDruidClient) s.getQueryRunner()).getNumOpenConnections()); @Override public QueryableDruidServer pick(Set servers, DataSegment segment) diff --git a/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java b/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java index 80b68f511e97..79110d39a4ae 100644 --- a/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java +++ b/server/src/main/java/org/apache/druid/client/selector/ServerSelectorStrategy.java @@ -29,7 +29,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RandomServerSelectorStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "random", value = RandomServerSelectorStrategy.class), - @JsonSubTypes.Type(name = "connectionCount", value = WeightedServerSelectorStrategy.class) + @JsonSubTypes.Type(name = "connectionCount", value = ConnectionCountServerSelectorStrategy.class) }) public interface ServerSelectorStrategy { diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java index 6ad302425113..60c4a4c5f2ab 100644 --- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java @@ -26,7 +26,7 @@ import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.client.selector.WeightedServerSelectorStrategy; +import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -129,7 +129,7 @@ public void testRun() throws Exception 0, 0L ), - new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( @@ -168,20 +168,20 @@ public void testRun() throws Exception Assert.assertTrue(capturedRequest.hasCaptured()); Assert.assertEquals(url, capturedRequest.getValue().getUrl()); Assert.assertEquals(HttpMethod.POST, capturedRequest.getValue().getMethod()); - Assert.assertEquals(1, client1.getWeight()); + Assert.assertEquals(1, client1.getNumOpenConnections()); // simulate read timeout client1.run(QueryPlus.wrap(query), defaultContext); - Assert.assertEquals(2, client1.getWeight()); + Assert.assertEquals(2, client1.getNumOpenConnections()); futureException.setException(new ReadTimeoutException()); - Assert.assertEquals(1, client1.getWeight()); + Assert.assertEquals(1, client1.getNumOpenConnections()); // subsequent connections should work client1.run(QueryPlus.wrap(query), defaultContext); client1.run(QueryPlus.wrap(query), defaultContext); client1.run(QueryPlus.wrap(query), defaultContext); - Assert.assertTrue(client1.getWeight() == 4); + Assert.assertTrue(client1.getNumOpenConnections() == 4); // produce result for first connection futureResult.set( @@ -192,12 +192,12 @@ public void testRun() throws Exception List results = s1.toList(); Assert.assertEquals(1, results.size()); Assert.assertEquals(DateTimes.of("2014-01-01T01:02:03Z"), results.get(0).getTimestamp()); - Assert.assertEquals(3, client1.getWeight()); + Assert.assertEquals(3, client1.getNumOpenConnections()); client2.run(QueryPlus.wrap(query), defaultContext); client2.run(QueryPlus.wrap(query), defaultContext); - Assert.assertTrue(client2.getWeight() == 2); + Assert.assertTrue(client2.getNumOpenConnections() == 2); Assert.assertTrue(serverSelector.pick() == queryableDruidServer2); @@ -247,7 +247,7 @@ public void testCancel() 0, 0L ), - new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( @@ -271,7 +271,7 @@ public void testCancel() cancellationFuture.set(new StatusResponseHolder(HttpResponseStatus.OK, new StringBuilder("cancelled"))); Sequence results = client1.run(QueryPlus.wrap(query), defaultContext); Assert.assertEquals(HttpMethod.DELETE, capturedRequest.getValue().getMethod()); - Assert.assertEquals(0, client1.getWeight()); + Assert.assertEquals(0, client1.getNumOpenConnections()); QueryInterruptedException exception = null; @@ -318,7 +318,7 @@ public void testQueryInterruptionExceptionLogMessage() ); final ServerSelector serverSelector = new ServerSelector( dataSegment, - new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()) + new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()) ); DirectDruidClient client1 = new DirectDruidClient( diff --git a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java index 6bef7aab1baf..ce7fe8aac86a 100644 --- a/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java +++ b/server/src/test/java/org/apache/druid/client/selector/TierSelectorStrategyTest.java @@ -54,7 +54,7 @@ public void testHighestPriorityTierSelectorStrategy() ); testTierSelectorStrategy( - new HighestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()), + new HighestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()), highPriority, lowPriority ); } @@ -73,7 +73,7 @@ public void testLowestPriorityTierSelectorStrategy() ); testTierSelectorStrategy( - new LowestPriorityTierSelectorStrategy(new WeightedServerSelectorStrategy()), + new LowestPriorityTierSelectorStrategy(new ConnectionCountServerSelectorStrategy()), lowPriority, highPriority ); } @@ -97,7 +97,7 @@ public void testCustomPriorityTierSelectorStrategy() testTierSelectorStrategy( new CustomTierSelectorStrategy( - new WeightedServerSelectorStrategy(), + new ConnectionCountServerSelectorStrategy(), new CustomTierSelectorStrategyConfig() { @Override From c9be4b0969882f556b908211720a6da802600d98 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 16:12:05 -0700 Subject: [PATCH 3/8] adjust javadoc --- .../groupby/GroupByQueryQueryToolChest.java | 21 +++++++------------ .../groupby/strategy/GroupByStrategy.java | 6 ++++-- 2 files changed, 12 insertions(+), 15 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index ccec22e87996..bc613dfa77fc 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -117,21 +117,16 @@ public GroupByQueryQueryToolChest( @Override public QueryRunner mergeResults(final QueryRunner runner) { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) - { - if (QueryContexts.isBySegment(queryPlus.getQuery())) { - return runner.run(queryPlus, responseContext); - } - - final GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); - if (strategySelector.strategize(groupByQuery).doMergeResults(groupByQuery)) { - return initAndMergeGroupByResults(groupByQuery, runner, responseContext); - } + return (queryPlus, responseContext) -> { + if (QueryContexts.isBySegment(queryPlus.getQuery())) { return runner.run(queryPlus, responseContext); } + + final GroupByQuery groupByQuery = (GroupByQuery) queryPlus.getQuery(); + if (strategySelector.strategize(groupByQuery).doMergeResults(groupByQuery)) { + return initAndMergeGroupByResults(groupByQuery, runner, responseContext); + } + return runner.run(queryPlus, responseContext); }; } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index 5198a9043f46..417a5adc64f2 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -75,7 +75,8 @@ QueryRunner createIntervalChunkingRunner( Sequence mergeResults(QueryRunner baseRunner, GroupByQuery query, Map responseContext); /** - * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)}. + * See {@link org.apache.druid.query.QueryToolChest#createMergeFn(Query)} for details, allows + * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy */ @Nullable default CombiningFunction createMergeFn(Query query) @@ -84,7 +85,8 @@ default CombiningFunction createMergeFn(Query query) } /** - * See {@link org.apache.druid.query.QueryToolChest#createOrderingFn(Query)}. + * See {@link org.apache.druid.query.QueryToolChest#createOrderingFn(Query)}, allows + * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy */ @Nullable default Ordering createOrderingFn(Query queryParam) From c5de93c6a7c299598305b308d1847ab3a290526a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 16:17:00 -0700 Subject: [PATCH 4/8] adjustments --- .../java/org/apache/druid/indexer/DeterminePartitionsJob.java | 2 +- .../java/org/apache/druid/client/DirectDruidClientTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index 57f593af22c5..669366c58458 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -570,7 +570,7 @@ private Iterable combineRows(Iterable input) return new CombiningIterable<>( Iterables.transform( input, - input1 -> DimValueCount.fromText(input1) + DimValueCount::fromText ), (o1, o2) -> ComparisonChain.start().compare(o1.dim, o2.dim).compare(o1.value, o2.value).result(), (arg1, arg2) -> { diff --git a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java index 60c4a4c5f2ab..65a8cd62c44f 100644 --- a/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java +++ b/server/src/test/java/org/apache/druid/client/DirectDruidClientTest.java @@ -23,10 +23,10 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.SettableFuture; +import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy; import org.apache.druid.client.selector.HighestPriorityTierSelectorStrategy; import org.apache.druid.client.selector.QueryableDruidServer; import org.apache.druid.client.selector.ServerSelector; -import org.apache.druid.client.selector.ConnectionCountServerSelectorStrategy; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; From 2afa991010525a93315aa94645ce20eded42d207 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 16:59:33 -0700 Subject: [PATCH 5/8] oops --- .../apache/druid/client/selector/QueryableDruidServer.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java index 9da2a04ec5b1..3d94986c8f15 100644 --- a/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/selector/QueryableDruidServer.java @@ -45,11 +45,6 @@ public T getQueryRunner() return queryRunner; } - public int getWeight() - { - return queryRunner.getWeight(); - } - @Override public String toString() { From 1270c7248514428a773fa8eb68f2e928fdbea9cc Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 16 Jul 2019 23:27:18 -0700 Subject: [PATCH 6/8] use it --- .../DistinctCountTopNQueryTest.java | 6 +- .../DatasourceOptimizerTest.java | 12 +- .../MaterializedViewQueryTest.java | 3 +- .../ApproximateHistogramTopNQueryTest.java | 8 +- .../FixedBucketsHistogramTopNQueryTest.java | 8 +- .../variance/VarianceTopNQueryTest.java | 4 +- .../druid/query/topn/TopNQueryBuilder.java | 6 + .../druid/query/DefaultQueryMetricsTest.java | 3 +- .../druid/query/MultiValuedDimensionTest.java | 6 +- .../topn/DefaultTopNQueryMetricsTest.java | 3 +- .../topn/TopNMetricSpecOptimizationsTest.java | 83 ++-- .../query/topn/TopNQueryRunnerBenchmark.java | 3 +- .../druid/query/topn/TopNQueryRunnerTest.java | 362 ++++++++---------- .../druid/query/topn/TopNQueryTest.java | 5 +- .../druid/query/topn/TopNUnionQueryTest.java | 10 +- .../org/apache/druid/segment/AppendTest.java | 4 +- .../druid/segment/SchemalessTestFullTest.java | 4 +- .../segment/SchemalessTestSimpleTest.java | 4 +- .../IncrementalIndexStorageAdapterTest.java | 2 +- .../virtual/DummyStringVirtualColumnTest.java | 4 +- .../client/CachingClusteredClientTest.java | 2 +- ...criptTieredBrokerSelectorStrategyTest.java | 24 +- .../druid/sql/calcite/CalciteQueryTest.java | 44 +-- 23 files changed, 257 insertions(+), 353 deletions(-) diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java index f7b12ce21c09..9d3d32689f41 100644 --- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java +++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java @@ -122,10 +122,8 @@ public void testTopNWithDistinctCountAgg() throws Exception .metric("UV") .threshold(10) .aggregators( - Lists.newArrayList( - QueryRunnerTestHelper.rowsCount, - new DistinctCountAggregatorFactory("UV", visitor_id, null) - ) + QueryRunnerTestHelper.rowsCount, + new DistinctCountAggregatorFactory("UV", visitor_id, null) ) .build(); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java index e73f55584458..ee136c25ca2d 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/DatasourceOptimizerTest.java @@ -208,9 +208,7 @@ public void testOptimize() throws InterruptedException .metric("cost") .threshold(4) .intervals("2011-04-01/2011-04-06") - .aggregators( - Collections.singletonList(new LongSumAggregatorFactory("cost", "cost")) - ) + .aggregators(new LongSumAggregatorFactory("cost", "cost")) .build(); List expectedQueryAfterOptimizing = Lists.newArrayList( @@ -221,9 +219,7 @@ public void testOptimize() throws InterruptedException .metric("cost") .threshold(4) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-01/2011-04-04")))) - .aggregators( - Collections.singletonList(new LongSumAggregatorFactory("cost", "cost")) - ) + .aggregators(new LongSumAggregatorFactory("cost", "cost")) .build(), new TopNQueryBuilder() .dataSource("base") @@ -232,9 +228,7 @@ public void testOptimize() throws InterruptedException .metric("cost") .threshold(4) .intervals(new MultipleIntervalSegmentSpec(Collections.singletonList(Intervals.of("2011-04-04/2011-04-06")))) - .aggregators( - Collections.singletonList(new LongSumAggregatorFactory("cost", "cost")) - ) + .aggregators(new LongSumAggregatorFactory("cost", "cost")) .build() ); Assert.assertEquals(expectedQueryAfterOptimizing, optimizer.optimize(userQuery)); diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java index 6a46a341cf9b..9936d2abab14 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.Collections; public class MaterializedViewQueryTest { @@ -80,7 +79,7 @@ public void testQuerySerialization() throws IOException ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); MaterializedViewQuery query = new MaterializedViewQuery(topNQuery, optimizer); String json = jsonMapper.writeValueAsString(query); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java index 5399a16e61a8..23c843f7eabc 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/ApproximateHistogramTopNQueryTest.java @@ -142,11 +142,9 @@ public void testTopNWithApproximateHistogramAgg() ) ) .postAggregators( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, - new QuantilePostAggregator("quantile", "apphisto", 0.5f) - ) + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + new QuantilePostAggregator("quantile", "apphisto", 0.5f) ) .build(); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java index ea9afba69eed..3ed0a6ee7974 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/FixedBucketsHistogramTopNQueryTest.java @@ -142,11 +142,9 @@ public void testTopNWithFixedHistogramAgg() ) ) .postAggregators( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, - new QuantilePostAggregator("quantile", "histo", 0.5f) - ) + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + new QuantilePostAggregator("quantile", "histo", 0.5f) ) .build(); diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java index 7b5406bc4b3e..3a93a65a2c6f 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/VarianceTopNQueryTest.java @@ -85,11 +85,11 @@ public void testFullOnTopNOverUniques() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java index 93667b06cc09..45227b93b739 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryBuilder.java @@ -265,6 +265,12 @@ public TopNQueryBuilder postAggregators(Collection p) return this; } + public TopNQueryBuilder postAggregators(PostAggregator... postAggs) + { + postAggregatorSpecs = Arrays.asList(postAggs); + return this; + } + public TopNQueryBuilder context(Map c) { context = c; diff --git a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java index 35c6eb0258e6..fad50e3c9702 100644 --- a/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java +++ b/processing/src/test/java/org/apache/druid/query/DefaultQueryMetricsTest.java @@ -33,7 +33,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -61,7 +60,7 @@ public void testDefaultQueryMetricsQuery() )) .metric("count") .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .aggregators(new CountAggregatorFactory("count")) .threshold(5) .filters(new SelectorDimFilter("tags", "t3", null)) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java index 616a49c48d80..a9c8467134be 100644 --- a/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java +++ b/processing/src/test/java/org/apache/druid/query/MultiValuedDimensionTest.java @@ -1016,7 +1016,7 @@ public void testTopNWithDimFilterAndWithFilteredDimSpec() )) .metric("count") .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .aggregators(new CountAggregatorFactory("count")) .threshold(5) .filters(new SelectorDimFilter("tags", "t3", null)) .build(); @@ -1071,7 +1071,7 @@ public void testTopNExpression() ) .metric("count") .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .aggregators(new CountAggregatorFactory("count")) .threshold(15) .build(); @@ -1132,7 +1132,7 @@ public void testTopNExpressionAutoTransform() ) .metric("count") .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .aggregators(new CountAggregatorFactory("count")) .threshold(15) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java b/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java index 7d113602c824..4f59e561b1d4 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/DefaultTopNQueryMetricsTest.java @@ -35,7 +35,6 @@ import org.junit.Assert; import org.junit.Test; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -63,7 +62,7 @@ public void testDefaultTopNQueryMetricsQuery() )) .metric("count") .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("count"))) + .aggregators(new CountAggregatorFactory("count")) .threshold(5) .filters(new SelectorDimFilter("tags", "t3", null)) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index ad215914a5d5..fef64af875f3 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.query.filter.Filter; @@ -50,10 +51,20 @@ import org.junit.Test; import javax.annotation.Nullable; -import java.util.Collections; +import java.util.List; public class TopNMetricSpecOptimizationsTest { + private static final List aggs = Lists.newArrayList( + Iterables.concat( + QueryRunnerTestHelper.commonDoubleAggregators, + Lists.newArrayList( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ); + @Test public void testShouldOptimizeLexicographic() { @@ -67,18 +78,8 @@ public void testShouldOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonDoubleAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index") - ) - ) - ) - ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .aggregators(aggs) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); StorageAdapter adapter = @@ -111,18 +112,8 @@ public void testAlsoShouldOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-30T01:00:00Z") - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonDoubleAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index") - ) - ) - ) - ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .aggregators(aggs) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); StorageAdapter adapter = @@ -156,18 +147,8 @@ public void testShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-30T01:00:00Z") - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonDoubleAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index") - ) - ) - ) - ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .aggregators(aggs) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); StorageAdapter adapter = @@ -202,18 +183,8 @@ public void testAlsoShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonDoubleAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index") - ) - ) - ) - ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .aggregators(aggs) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); StorageAdapter adapter = @@ -246,18 +217,8 @@ public void testAgainShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators( - Lists.newArrayList( - Iterables.concat( - QueryRunnerTestHelper.commonDoubleAggregators, - Lists.newArrayList( - new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index") - ) - ) - ) - ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .aggregators(aggs) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java index 839ee6d5aba7..171f42285d2b 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerBenchmark.java @@ -40,7 +40,6 @@ import org.junit.Test; import java.nio.ByteBuffer; -import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -78,7 +77,7 @@ public enum TestCases ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); private static final Map testCaseMap = new HashMap<>(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java index 0eb48f5c7e5c..4b9fb51afb03 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryRunnerTest.java @@ -291,7 +291,7 @@ public void testEmptyTopN() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = ImmutableList.of( @@ -324,11 +324,11 @@ public void testFullOnTopN() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -385,7 +385,7 @@ public void testTopNOnMissingColumn() .metric("rows") .threshold(4) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("rows"))) + .aggregators(new CountAggregatorFactory("rows")) .build(); final HashMap resultMap = new HashMap<>(); @@ -411,7 +411,7 @@ public void testTopNOnMissingColumnWithExtractionFn() .metric("rows") .threshold(4) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Collections.singletonList(new CountAggregatorFactory("rows"))) + .aggregators(new CountAggregatorFactory("rows")) .build(); List> expectedResults = Collections.singletonList( @@ -451,11 +451,11 @@ public void testFullOnTopNOverPostAggs() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -515,19 +515,17 @@ public void testFullOnTopNOverPostAggsOnDimension() ) ) .postAggregators( - ImmutableList.of( - new ExpressionPostAggregator( - "dimPostAgg", - "market + 'x'", - null, - TestExprMacroTable.INSTANCE - ) + new ExpressionPostAggregator( + "dimPostAgg", + "market + 'x'", + null, + TestExprMacroTable.INSTANCE ) ) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -586,11 +584,11 @@ public void testFullOnTopNOverUniques() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -638,13 +636,11 @@ public void testTopNOverMissingUniques() .metric(QueryRunnerTestHelper.uniqueMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators( - Collections.singletonList(new HyperUniquesAggregatorFactory("uniques", "missingUniques")) - ) + .aggregators(new HyperUniquesAggregatorFactory("uniques", "missingUniques")) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -677,14 +673,12 @@ public void testTopNOverHyperUniqueFinalizingPostAggregator() .metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators( - Collections.singletonList(QueryRunnerTestHelper.qualityUniques) - ) + .aggregators(QueryRunnerTestHelper.qualityUniques) .postAggregators( - Collections.singletonList(new HyperUniqueFinalizingPostAggregator( + new HyperUniqueFinalizingPostAggregator( QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, QueryRunnerTestHelper.uniqueMetric - )) + ) ) .build(); @@ -725,16 +719,14 @@ public void testTopNOverHyperUniqueExpression() .metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators( - Collections.singletonList(QueryRunnerTestHelper.qualityUniques) - ) + .aggregators(QueryRunnerTestHelper.qualityUniques) .postAggregators( - Collections.singletonList(new ExpressionPostAggregator( + new ExpressionPostAggregator( QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, "uniques + 1", null, TestExprMacroTable.INSTANCE - )) + ) ) .build(); @@ -781,16 +773,14 @@ public void testTopNOverHyperUniqueExpressionRounded() .metric(QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric) .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators( - Collections.singletonList(QueryRunnerTestHelper.qualityUniquesRounded) - ) + .aggregators(QueryRunnerTestHelper.qualityUniquesRounded) .postAggregators( - Collections.singletonList(new ExpressionPostAggregator( + new ExpressionPostAggregator( QueryRunnerTestHelper.hyperUniqueFinalizingPostAggMetric, "uniques + 1", null, TestExprMacroTable.INSTANCE - )) + ) ) .build(); @@ -832,10 +822,8 @@ public void testTopNOverFirstLastAggregator() .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - new LongFirstAggregatorFactory("first", "index"), - new LongLastAggregatorFactory("last", "index") - ) + new LongFirstAggregatorFactory("first", "index"), + new LongLastAggregatorFactory("last", "index") ) .build(); @@ -943,10 +931,8 @@ public void testTopNOverFirstLastAggregatorChunkPeriod() .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - new LongFirstAggregatorFactory("first", "index"), - new LongLastAggregatorFactory("last", "index") - ) + new LongFirstAggregatorFactory("first", "index"), + new LongLastAggregatorFactory("last", "index") ) .context(ImmutableMap.of("chunkPeriod", "P1D")) .build(); @@ -1057,10 +1043,8 @@ public void testTopNOverFirstLastFloatAggregatorUsingDoubleColumn() .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - new FloatFirstAggregatorFactory("first", "index"), - new FloatLastAggregatorFactory("last", "index") - ) + new FloatFirstAggregatorFactory("first", "index"), + new FloatLastAggregatorFactory("last", "index") ) .build(); @@ -1168,10 +1152,8 @@ public void testTopNOverFirstLastFloatAggregatorUsingFloatColumn() .threshold(3) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - new FloatFirstAggregatorFactory("first", "indexFloat"), - new FloatLastAggregatorFactory("last", "indexFloat") - ) + new FloatFirstAggregatorFactory("first", "indexFloat"), + new FloatLastAggregatorFactory("last", "indexFloat") ) .build(); @@ -1284,13 +1266,13 @@ public void testTopNBySegment() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .context(specialContext) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1368,12 +1350,12 @@ public void testTopN() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1416,12 +1398,12 @@ public void testTopNByUniques() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1465,11 +1447,11 @@ public void testTopNWithOrFilter1() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1513,11 +1495,11 @@ public void testTopNWithOrFilter2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1554,11 +1536,11 @@ public void testTopNWithFilter1() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Collections.>singletonList( @@ -1588,11 +1570,11 @@ public void testTopNWithFilter2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1640,11 +1622,11 @@ public void testTopNWithFilter2OneDay() ) ) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1688,11 +1670,11 @@ public void testTopNWithNonExistentFilterInOr() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1729,7 +1711,7 @@ public void testTopNWithNonExistentFilter() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); assertExpectedResults( Collections.singletonList( @@ -1755,7 +1737,7 @@ public void testTopNWithNonExistentFilterMultiDim() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); assertExpectedResults( Collections.singletonList( @@ -1777,7 +1759,7 @@ public void testTopNWithMultiValueDimFilter1() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); assertExpectedResults( @@ -1791,7 +1773,7 @@ public void testTopNWithMultiValueDimFilter1() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build() ).toList(), query @@ -1810,7 +1792,7 @@ public void testTopNWithMultiValueDimFilter2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); assertExpectedResults( @@ -1829,7 +1811,7 @@ public void testTopNWithMultiValueDimFilter2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build() ).toList(), query @@ -1848,7 +1830,7 @@ public void testTopNWithMultiValueDimFilter3() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); final List> expectedResults = Collections.singletonList( @@ -1889,7 +1871,7 @@ public void testTopNWithMultiValueDimFilter4() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); final List> expectedResults = Collections.singletonList( @@ -1937,11 +1919,11 @@ public void testTopNWithMultiValueDimFilter5() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); final List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -1991,7 +1973,7 @@ public void testTopNWithNonExistentDimension() .threshold(1) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2025,7 +2007,7 @@ public void testTopNWithNonExistentDimensionAndActualFilter() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2059,7 +2041,7 @@ public void testTopNWithNonExistentDimensionAndNonExistentFilter() .threshold(1) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2092,7 +2074,7 @@ public void testTopNLexicographic() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2172,7 +2154,7 @@ public void testTopNLexicographicWithPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2212,7 +2194,7 @@ public void testTopNLexicographicWithNonExistingPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2252,11 +2234,11 @@ public void testTopNInvertedLexicographicWithPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -2292,11 +2274,11 @@ public void testTopNInvertedLexicographicWithNonExistingPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -2339,7 +2321,7 @@ public void testTopNDimExtractionToOne() .threshold(10) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2386,7 +2368,7 @@ public void testTopNDimExtractionTimeToOneLong() .threshold(10) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2428,12 +2410,10 @@ public void testTopNCollapsingDimExtraction() .threshold(2) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexDoubleSum - ) + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.indexDoubleSum ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2498,7 +2478,7 @@ public void testTopNDimExtraction() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2600,7 +2580,7 @@ public void testTopNDimExtractionFastTopNOptimalWithReplaceMissing() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2664,7 +2644,7 @@ public void testTopNDimExtractionFastTopNUnOptimalWithReplaceMissing() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2729,7 +2709,7 @@ public void testTopNDimExtractionFastTopNOptimal() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2796,7 +2776,7 @@ public void testTopNDimExtractionFastTopNUnOptimal() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -2862,11 +2842,11 @@ public void testTopNLexicographicDimExtractionOptimalNamespace() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -2928,11 +2908,11 @@ public void testTopNLexicographicDimExtractionUnOptimalNamespace() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -2995,11 +2975,11 @@ public void testTopNLexicographicDimExtractionOptimalNamespaceWithRunner() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3048,11 +3028,11 @@ public void testTopNLexicographicDimExtraction() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3101,11 +3081,11 @@ public void testInvertedTopNLexicographicDimExtraction2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3154,11 +3134,11 @@ public void testTopNLexicographicDimExtractionWithPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3225,11 +3205,11 @@ public ExtractionType getExtractionType() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3272,11 +3252,11 @@ public void testInvertedTopNLexicographicDimExtractionWithPreviousStop() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3318,11 +3298,11 @@ public void testInvertedTopNLexicographicDimExtractionWithPreviousStop2() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-04-01T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3384,7 +3364,7 @@ public ExtractionType getExtractionType() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .dimension( new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, @@ -3472,7 +3452,7 @@ public ExtractionType getExtractionType() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .dimension( new ExtractionDimensionSpec( QueryRunnerTestHelper.marketDimension, @@ -3529,7 +3509,7 @@ public void testInvertedTopNQuery() .threshold(3) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -3685,16 +3665,14 @@ public void testTopNDependentPostAgg() ) ) .postAggregators( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, - QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg - ) + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg ) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -3769,10 +3747,8 @@ public void testTopNBySegmentResults() ) ) .postAggregators( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg - ) + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg ) .context(ImmutableMap.of("finalize", true, "bySegment", true)) .build(); @@ -3835,11 +3811,9 @@ public void testTopNWithTimeColumn() .dataSource(QueryRunnerTestHelper.dataSource) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.jsCountIfTimeGreaterThan, - QueryRunnerTestHelper.__timeLongSum - ) + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.jsCountIfTimeGreaterThan, + QueryRunnerTestHelper.__timeLongSum ) .granularity(QueryRunnerTestHelper.allGran) .dimension(QueryRunnerTestHelper.marketDimension) @@ -3903,12 +3877,10 @@ public void testTopNTimeExtraction() .threshold(2) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .aggregators( - Arrays.asList( - QueryRunnerTestHelper.rowsCount, - QueryRunnerTestHelper.indexDoubleSum - ) + QueryRunnerTestHelper.rowsCount, + QueryRunnerTestHelper.indexDoubleSum ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -3956,7 +3928,7 @@ public void testTopNOverNullDimension() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); Map map = new HashMap<>(); @@ -4004,7 +3976,7 @@ public void testTopNOverNullDimensionWithFilter() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); Map map = new HashMap<>(); @@ -4223,8 +4195,7 @@ public void testTopNWithExtractionFilter() .threshold(3) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList( - QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .filters( new ExtractionDimFilter( QueryRunnerTestHelper.marketDimension, @@ -4293,7 +4264,7 @@ public void testTopNWithExtractionFilterAndFilteredAggregatorCaseNoExistingValue ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)); + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant); TopNQuery topNQueryWithNULLValueExtraction = topNQueryBuilder .filters(extractionFilter) .build(); @@ -4359,12 +4330,21 @@ public void testTopNWithExtractionFilterNoExistingValue() .metric(QueryRunnerTestHelper.indexMetric) .threshold(4) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators(Lists.newArrayList(Iterables.concat(commonAggregators, Lists.newArrayList( - new FilteredAggregatorFactory(new DoubleMaxAggregatorFactory("maxIndex", "index"), - extractionFilter), - //new DoubleMaxAggregatorFactory("maxIndex", "index"), - new DoubleMinAggregatorFactory("minIndex", "index"))))) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)); + .aggregators( + Lists.newArrayList( + Iterables.concat( + commonAggregators, + Lists.newArrayList( + new FilteredAggregatorFactory( + new DoubleMaxAggregatorFactory("maxIndex", "index"), + extractionFilter + ), + new DoubleMinAggregatorFactory("minIndex", "index") + ) + ) + ) + ) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant); TopNQuery topNQueryWithNULLValueExtraction = topNQueryBuilder .filters(extractionFilter) .build(); @@ -4414,11 +4394,11 @@ public void testFullOnTopNFloatColumn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4489,11 +4469,11 @@ public void testFullOnTopNFloatColumnWithExFn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4561,11 +4541,11 @@ public void testFullOnTopNFloatColumnAsString() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4633,11 +4613,11 @@ public void testFullOnTopNLongColumn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4705,12 +4685,12 @@ public void testFullOnTopNLongVirtualColumn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .virtualColumns(new ExpressionVirtualColumn("ql_expr", "qualityLong", ValueType.LONG, ExprMacroTable.nil())) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4776,7 +4756,7 @@ public void testTopNStringVirtualColumn() .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) .aggregators(commonAggregators) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( @@ -4836,11 +4816,11 @@ public void testFullOnTopNLongColumnWithExFn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4908,11 +4888,11 @@ public void testFullOnTopNLongColumnAsString() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -4980,11 +4960,11 @@ public void testFullOnTopNNumericStringColumnAsLong() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5052,11 +5032,11 @@ public void testFullOnTopNNumericStringColumnAsFloat() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5124,11 +5104,11 @@ public void testFullOnTopNLongTimeColumn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5331,11 +5311,11 @@ public void testFullOnTopNLongTimeColumnWithExFn() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5410,7 +5390,7 @@ public void testFullOnTopNDimExtractionAllNulls() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); Map expectedMap = new HashMap<>(); @@ -5423,7 +5403,7 @@ public void testFullOnTopNDimExtractionAllNulls() expectedMap.put("minIndex", 59.02102279663086D); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Collections.singletonList( @@ -5458,11 +5438,11 @@ public void testFullOnTopNStringOutputAsLong() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5536,11 +5516,11 @@ public void testFullOnTopNNumericStringColumnWithDecoration() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5605,11 +5585,11 @@ public void testFullOnTopNDecorationOnNumeric() ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( @@ -5713,7 +5693,7 @@ public void testFullOnTopNWithAggsOnNumericDims() } queryBuilder.metric(metric); if (hasIndexAggregator && hasRowsAggregator) { - queryBuilder.postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)); + queryBuilder.postAggregators(QueryRunnerTestHelper.addRowsIndexConstant); } TopNQuery query = queryBuilder.build(); @@ -5767,13 +5747,11 @@ public void testFullOnTopNBoundFilterAndLongSumMetric() .metric("Count") .threshold(5) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .aggregators( - Collections.singletonList(new LongSumAggregatorFactory("Count", "qualityLong")) - ) + .aggregators(new LongSumAggregatorFactory("Count", "qualityLong")) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue(Collections.emptyList()) ) @@ -5804,9 +5782,7 @@ public void testTopNWithNonBitmapFilter() .metric("count") .threshold(4) .intervals(QueryRunnerTestHelper.firstToThird) - .aggregators( - Collections.singletonList(new DoubleSumAggregatorFactory("count", "qualityDouble")) - ) + .aggregators(new DoubleSumAggregatorFactory("count", "qualityDouble")) .build(); // Don't check results, just the fact that the query could complete diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java index 2fc0db7a1c93..65496170dfe1 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNQueryTest.java @@ -37,7 +37,6 @@ import org.junit.Test; import java.io.IOException; -import java.util.Collections; public class TopNQueryTest { @@ -64,7 +63,7 @@ public void testQuerySerialization() throws IOException ) ) ) - .postAggregators(Collections.singletonList(QueryRunnerTestHelper.addRowsIndexConstant)) + .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); String json = jsonMapper.writeValueAsString(query); @@ -122,7 +121,7 @@ public void testQuerySerdeWithAlphaNumericTopNMetricSpec() throws IOException .metric(new DimensionTopNMetricSpec(null, StringComparators.ALPHANUMERIC)) .threshold(2) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec.getIntervals()) - .aggregators(Collections.singletonList(QueryRunnerTestHelper.rowsCount)) + .aggregators(QueryRunnerTestHelper.rowsCount) .build(); String jsonQuery = "{\n" + " \"queryType\": \"topN\",\n" diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java index 076bbda48fe3..6f6f9eeb5f5a 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java @@ -123,16 +123,14 @@ public void testTopNUnionQuery() ) ) .postAggregators( - Arrays.asList( - QueryRunnerTestHelper.addRowsIndexConstant, - QueryRunnerTestHelper.dependentPostAgg, - QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg - ) + QueryRunnerTestHelper.addRowsIndexConstant, + QueryRunnerTestHelper.dependentPostAgg, + QueryRunnerTestHelper.hyperUniqueFinalizingPostAgg ) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.>asList( diff --git a/processing/src/test/java/org/apache/druid/segment/AppendTest.java b/processing/src/test/java/org/apache/druid/segment/AppendTest.java index 5ed44fcb52ec..0bb20367f91d 100644 --- a/processing/src/test/java/org/apache/druid/segment/AppendTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AppendTest.java @@ -658,7 +658,7 @@ private TopNQuery makeTopNQuery() ) ) ) - .postAggregators(Collections.singletonList(addRowsIndexConstant)) + .postAggregators(addRowsIndexConstant) .build(); } @@ -688,7 +688,7 @@ private TopNQuery makeFilteredTopNQuery() ) ) ) - .postAggregators(Collections.singletonList(addRowsIndexConstant)) + .postAggregators(addRowsIndexConstant) .build(); } diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java index a48dba24887a..3786564d8f37 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestFullTest.java @@ -1524,7 +1524,7 @@ private void testFullOnTopN(QueryRunner runner, List> ex ) ) ) - .postAggregators(Collections.singletonList(addRowsIndexConstant)) + .postAggregators(addRowsIndexConstant) .build(); failMsg += " topN "; @@ -1557,7 +1557,7 @@ private void testFilteredTopN(QueryRunner runner, List> ) ) ) - .postAggregators(Collections.singletonList(addRowsIndexConstant)) + .postAggregators(addRowsIndexConstant) .build(); failMsg += " filtered topN "; diff --git a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java index 64e0c76990e3..b0fbf6273165 100644 --- a/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java +++ b/processing/src/test/java/org/apache/druid/segment/SchemalessTestSimpleTest.java @@ -190,11 +190,11 @@ public void testFullOnTopN() ) ) ) - .postAggregators(Collections.singletonList(addRowsIndexConstant)) + .postAggregators(addRowsIndexConstant) .build(); List> expectedResults = Collections.singletonList( - new Result( + new Result<>( DateTimes.of("2011-01-12T00:00:00.000Z"), new TopNResultValue( Arrays.asList( diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index 67a08b93f5a0..b3302f84fafe 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -356,7 +356,7 @@ public void testSingleValueTopN() throws IOException .dimension("sally") .metric("cnt") .threshold(10) - .aggregators(Collections.singletonList(new LongSumAggregatorFactory("cnt", "cnt"))) + .aggregators(new LongSumAggregatorFactory("cnt", "cnt")) .build(), new IncrementalIndexStorageAdapter(index), null diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java index 1158d57eaba5..899554fc1d45 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java @@ -360,9 +360,7 @@ private void testTopN( .dimension(VSTRING_DIM) .metric(COUNT) .threshold(1) - .aggregators( - Collections.singletonList(new CountAggregatorFactory(COUNT)) - ) + .aggregators(new CountAggregatorFactory(COUNT)) .virtualColumns(new DummyStringVirtualColumn( QueryRunnerTestHelper.marketDimension, VSTRING_DIM, diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 8170014c3e6a..ba9cf76dd41e 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -941,7 +941,7 @@ public void testOutOfOrderSequenceMerging() .dimension("a") .metric("b") .threshold(3) - .aggregators(Collections.singletonList(new CountAggregatorFactory("b"))) + .aggregators(new CountAggregatorFactory("b")) .build(), sequences ) diff --git a/server/src/test/java/org/apache/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java b/server/src/test/java/org/apache/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java index 277f56d6d835..ec1ddb9f66a6 100644 --- a/server/src/test/java/org/apache/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/router/JavaScriptTieredBrokerSelectorStrategyTest.java @@ -23,11 +23,9 @@ import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.js.JavaScriptConfig; import org.apache.druid.query.Druids; -import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -117,11 +115,7 @@ public LinkedHashMap getTierToBrokerMap() .dimension("bigdim") .metric("count") .threshold(1) - .aggregators( - ImmutableList.of( - new CountAggregatorFactory("count") - ) - ); + .aggregators(new CountAggregatorFactory("count")); Assert.assertEquals( Optional.absent(), @@ -145,11 +139,9 @@ public LinkedHashMap getTierToBrokerMap() STRATEGY.getBrokerServiceName( tieredBrokerConfig, queryBuilder.aggregators( - ImmutableList.of( - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("longSum", "a"), - new DoubleSumAggregatorFactory("doubleSum", "b") - ) + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("longSum", "a"), + new DoubleSumAggregatorFactory("doubleSum", "b") ).build() ) ); @@ -161,11 +153,9 @@ public LinkedHashMap getTierToBrokerMap() STRATEGY.getBrokerServiceName( tieredBrokerConfig, queryBuilder.aggregators( - ImmutableList.of( - new CountAggregatorFactory("count"), - new LongSumAggregatorFactory("longSum", "a"), - new DoubleSumAggregatorFactory("doubleSum", "b") - ) + new CountAggregatorFactory("count"), + new LongSumAggregatorFactory("longSum", "a"), + new DoubleSumAggregatorFactory("doubleSum", "b") ).build() ) ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 009c8ec103bf..d4c6473f7a07 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -1495,9 +1495,7 @@ public void testTopNWithSelectProjections() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim1", "d0")) - .postAggregators(ImmutableList.of( - expressionPostAgg("p0", "substring(\"d0\", 1, -1)") - )) + .postAggregators(expressionPostAgg("p0", "substring(\"d0\", 1, -1)")) .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) @@ -1532,10 +1530,10 @@ public void testTopNWithSelectAndOrderByProjections() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim1", "d0")) - .postAggregators(ImmutableList.of( + .postAggregators( expressionPostAgg("p0", "substring(\"d0\", 1, -1)"), expressionPostAgg("p1", "strlen(\"d0\")") - )) + ) .metric(new NumericTopNMetricSpec("p1")) .threshold(10) .context(QUERY_CONTEXT_DEFAULT) @@ -2542,15 +2540,11 @@ public void testGroupByWithSortOnPostAggregationDefault() throws Exception .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim1", "d0")) .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("p0"))) - .aggregators(aggregators( + .aggregators( new FloatMinAggregatorFactory("a0", "m1"), new FloatMaxAggregatorFactory("a1", "m1") - )) - .postAggregators( - ImmutableList.of( - expressionPostAgg("p0", "(\"a0\" + \"a1\")") - ) ) + .postAggregators(expressionPostAgg("p0", "(\"a0\" + \"a1\")")) .threshold(3) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -4650,7 +4644,7 @@ public void testTopNFilterJoin() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim2", "d0")) - .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) .metric(new NumericTopNMetricSpec("a0")) .threshold(2) .context(QUERY_CONTEXT_DEFAULT) @@ -4717,7 +4711,7 @@ public void testTopNFilterJoinWithProjection() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("dim2", "d0")) - .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) .metric(new NumericTopNMetricSpec("a0")) .threshold(2) .context(QUERY_CONTEXT_DEFAULT) @@ -7582,24 +7576,22 @@ public void testPostAggWithTopN() throws Exception .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("m1", "d0", ValueType.FLOAT)) .filters("dim2", "a") - .aggregators(aggregators( + .aggregators( new DoubleSumAggregatorFactory("a0:sum", "m2"), new CountAggregatorFactory("a0:count"), new DoubleSumAggregatorFactory("a1", "m1"), new DoubleSumAggregatorFactory("a2", "m2") - )) + ) .postAggregators( - ImmutableList.of( - new ArithmeticPostAggregator( - "a0", - "quotient", - ImmutableList.of( - new FieldAccessPostAggregator(null, "a0:sum"), - new FieldAccessPostAggregator(null, "a0:count") - ) - ), - expressionPostAgg("p0", "(\"a1\" + \"a2\")") - ) + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ), + expressionPostAgg("p0", "(\"a1\" + \"a2\")") ) .metric(new DimensionTopNMetricSpec(null, StringComparators.NUMERIC)) .threshold(5) From f38b888f68df181794af3f501d207ec50595d74d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 17 Jul 2019 16:25:18 -0700 Subject: [PATCH 7/8] use BinaryOperator, remove CombiningFunction, use Comparator instead of Ordering, other review adjustments --- .../CachingClusteredClientBenchmark.java | 23 +++++----- codestyle/druid-forbidden-apis.txt | 1 + .../druid/collections/CombiningFunction.java | 26 ----------- .../druid/collections/CombiningIterable.java | 43 +++++++++---------- .../druid/collections/CombiningIterator.java | 29 +++++++------ .../druid/common/guava/CombiningSequence.java | 24 +++++------ .../apache/druid/common/guava/GuavaUtils.java | 17 +++++++- .../java/util/common/guava/BaseSequence.java | 23 ++++------ .../collections/CombiningIteratorTest.java | 5 ++- .../common/guava/ComplexSequenceTest.java | 29 +++++++------ .../apache/druid/query/QueryToolChest.java | 14 +++--- .../druid/query/ResultMergeQueryRunner.java | 16 +++---- .../groupby/GroupByQueryQueryToolChest.java | 10 ++--- .../epinephelinae/GroupByBinaryFnV2.java | 4 +- .../groupby/strategy/GroupByStrategy.java | 14 +++--- .../groupby/strategy/GroupByStrategyV2.java | 10 ++--- .../SegmentMetadataQueryQueryToolChest.java | 4 +- .../druid/query/search/SearchBinaryFn.java | 6 +-- .../search/SearchQueryQueryToolChest.java | 13 +++--- .../druid/query/select/SelectBinaryFn.java | 4 +- .../select/SelectQueryQueryToolChest.java | 19 +++----- .../query/timeseries/TimeseriesBinaryFn.java | 4 +- .../TimeseriesQueryQueryToolChest.java | 19 +++----- .../apache/druid/query/topn/TopNBinaryFn.java | 4 +- .../query/topn/TopNQueryQueryToolChest.java | 12 +++--- .../collections/CombiningIterableTest.java | 9 ++-- .../topn/TopNMetricSpecOptimizationsTest.java | 12 +++--- .../apache/druid/client/BrokerServerView.java | 2 +- 28 files changed, 181 insertions(+), 215 deletions(-) delete mode 100644 core/src/main/java/org/apache/druid/collections/CombiningFunction.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 7b83b48adc01..b6807d068d1f 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -139,6 +139,11 @@ @Measurement(iterations = 30) public class CachingClusteredClientBenchmark { + private static final Logger LOG = new Logger(CachingClusteredClientBenchmark.class); + private static final int PROCESSING_BUFFER_SIZE = 10 * 1024 * 1024; // ~10MB + private static final String DATA_SOURCE = "ds"; + + public static final ObjectMapper JSON_MAPPER; @Param({"8"}) private int numServers; @@ -151,10 +156,6 @@ public class CachingClusteredClientBenchmark @Param({"all"}) private String queryGranularity; - private static final Logger log = new Logger(CachingClusteredClientBenchmark.class); - private static final String DATA_SOURCE = "ds"; - public static final ObjectMapper JSON_MAPPER; - private QueryToolChestWarehouse toolChestWarehouse; private QueryRunnerFactoryConglomerate conglomerate; private CachingClusteredClient cachingClusteredClient; @@ -197,7 +198,7 @@ public void setup() .shardSpec(new LinearShardSpec(i)) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); - log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); + LOG.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); final QueryableIndex index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment); queryableIndexes.put(dataSegment, index); } @@ -213,15 +214,13 @@ public String getFormatString() @Override public int intermediateComputeSizeBytes() { - return 10 * 1024 * 1024; + return PROCESSING_BUFFER_SIZE; } @Override public int getNumMergeBuffers() { - // Need 3 buffers for CalciteQueryTest.testDoubleNestedGroupby. - // Two buffers for the broker and one for the queryable - return 3; + return 1; } @Override @@ -248,7 +247,7 @@ public int getNumThreads() new TopNQueryRunnerFactory( new StupidPool<>( "TopNQueryRunnerFactory-bufferPool", - () -> ByteBuffer.allocate(10 * 1024 * 1024) + () -> ByteBuffer.allocate(PROCESSING_BUFFER_SIZE) ), new TopNQueryQueryToolChest( new TopNQueryConfig(), @@ -389,7 +388,7 @@ public void topNQuery(Blackhole blackhole) .aggregators(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential")) .granularity(Granularity.fromString(queryGranularity)) .metric("sumLongSequential") - .threshold(20480) + .threshold(10_000) // we are primarily measuring 'broker' merge time, so collect a significant number of results .build(); final List> results = runQuery(); @@ -560,7 +559,7 @@ private static DruidServer createServer(int nameSuiffix) "server_" + nameSuiffix, "127.0.0." + nameSuiffix, null, - 10240L, + Long.MAX_VALUE, ServerType.HISTORICAL, "default", 0 diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index c7f38fb47e42..7925b3b1a722 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -35,6 +35,7 @@ java.lang.Math#random() @ Use ThreadLocalRandom.current() java.util.regex.Pattern#matches(java.lang.String,java.lang.CharSequence) @ Use String.startsWith(), endsWith(), contains(), or compile and cache a Pattern explicitly org.apache.commons.io.FileUtils#getTempDirectory() @ Use org.junit.rules.TemporaryFolder for tests instead java.lang.Class#getCanonicalName() @ Class.getCanonicalName can return null for anonymous types, use Class.getName instead. +com.google.common.base.Objects#firstNonNull(java.lang.Object, java.lang.Object) @ Use org.apache.druid.common.guava.GuavaUtils#firstNonNull(java.lang.Object, java.lang.Object) instead (probably... the GuavaUtils method return object is nullable) @defaultMessage Use Locale.ENGLISH com.ibm.icu.text.DateFormatSymbols#() diff --git a/core/src/main/java/org/apache/druid/collections/CombiningFunction.java b/core/src/main/java/org/apache/druid/collections/CombiningFunction.java deleted file mode 100644 index 93d1390e2312..000000000000 --- a/core/src/main/java/org/apache/druid/collections/CombiningFunction.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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 org.apache.druid.collections; - -@FunctionalInterface -public interface CombiningFunction -{ - T apply(T arg1, T arg2); -} diff --git a/core/src/main/java/org/apache/druid/collections/CombiningIterable.java b/core/src/main/java/org/apache/druid/collections/CombiningIterable.java index 69eb48162117..5b1c17405042 100644 --- a/core/src/main/java/org/apache/druid/collections/CombiningIterable.java +++ b/core/src/main/java/org/apache/druid/collections/CombiningIterable.java @@ -19,14 +19,16 @@ package org.apache.druid.collections; +import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.java.util.common.guava.MergeIterable; import java.util.Comparator; import java.util.Iterator; +import java.util.function.BinaryOperator; /** */ -public class CombiningIterable implements Iterable +public class CombiningIterable implements Iterable { /** * Creates a CombiningIterable around a MergeIterable such that equivalent elements are thrown away @@ -37,45 +39,40 @@ public class CombiningIterable implements Iterable * * @param in An Iterable of Iterables to be merged * @param comparator the Comparator to determine sort and equality - * @param Type of object + * @param Type of object * @return An Iterable that is the merge of all Iterables from in such that there is only one instance of * equivalent objects. */ @SuppressWarnings("unchecked") - public static CombiningIterable createSplatted( - Iterable> in, - Comparator comparator + public static CombiningIterable createSplatted( + Iterable> in, + Comparator comparator ) { return create( - new MergeIterable(comparator, (Iterable>) in), + new MergeIterable<>(comparator, (Iterable>) in), comparator, - (arg1, arg2) -> { - if (arg1 == null) { - return arg2; - } - return arg1; - } + GuavaUtils::firstNonNull ); } - public static CombiningIterable create( - Iterable it, - Comparator comparator, - CombiningFunction fn + public static CombiningIterable create( + Iterable it, + Comparator comparator, + BinaryOperator fn ) { return new CombiningIterable<>(it, comparator, fn); } - private final Iterable it; - private final Comparator comparator; - private final CombiningFunction fn; + private final Iterable it; + private final Comparator comparator; + private final BinaryOperator fn; public CombiningIterable( - Iterable it, - Comparator comparator, - CombiningFunction fn + Iterable it, + Comparator comparator, + BinaryOperator fn ) { this.it = it; @@ -84,7 +81,7 @@ public CombiningIterable( } @Override - public Iterator iterator() + public Iterator iterator() { return CombiningIterator.create(it.iterator(), comparator, fn); } diff --git a/core/src/main/java/org/apache/druid/collections/CombiningIterator.java b/core/src/main/java/org/apache/druid/collections/CombiningIterator.java index 32ec4d45f6a9..fef6ad821db5 100644 --- a/core/src/main/java/org/apache/druid/collections/CombiningIterator.java +++ b/core/src/main/java/org/apache/druid/collections/CombiningIterator.java @@ -25,28 +25,29 @@ import java.util.Comparator; import java.util.Iterator; import java.util.NoSuchElementException; +import java.util.function.BinaryOperator; /** */ -public class CombiningIterator implements Iterator +public class CombiningIterator implements Iterator { - public static CombiningIterator create( - Iterator it, - Comparator comparator, - CombiningFunction fn + public static CombiningIterator create( + Iterator it, + Comparator comparator, + BinaryOperator fn ) { - return new CombiningIterator(it, comparator, fn); + return new CombiningIterator<>(it, comparator, fn); } - private final PeekingIterator it; - private final Comparator comparator; - private final CombiningFunction fn; + private final PeekingIterator it; + private final Comparator comparator; + private final BinaryOperator fn; public CombiningIterator( - Iterator it, - Comparator comparator, - CombiningFunction fn + Iterator it, + Comparator comparator, + BinaryOperator fn ) { this.it = Iterators.peekingIterator(it); @@ -61,13 +62,13 @@ public boolean hasNext() } @Override - public InType next() + public T next() { if (!hasNext()) { throw new NoSuchElementException(); } - InType res = null; + T res = null; while (hasNext()) { if (res == null) { diff --git a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java index f3eba7edaf32..9e9a7d77df57 100644 --- a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java +++ b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java @@ -19,8 +19,6 @@ package org.apache.druid.common.guava; -import com.google.common.collect.Ordering; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -28,6 +26,8 @@ import org.apache.druid.java.util.common.guava.YieldingAccumulator; import java.io.IOException; +import java.util.Comparator; +import java.util.function.BinaryOperator; /** */ @@ -35,21 +35,21 @@ public class CombiningSequence implements Sequence { public static CombiningSequence create( Sequence baseSequence, - Ordering ordering, - CombiningFunction mergeFn + Comparator ordering, + BinaryOperator mergeFn ) { return new CombiningSequence<>(baseSequence, ordering, mergeFn); } private final Sequence baseSequence; - private final Ordering ordering; - private final CombiningFunction mergeFn; + private final Comparator ordering; + private final BinaryOperator mergeFn; private CombiningSequence( Sequence baseSequence, - Ordering ordering, - CombiningFunction mergeFn + Comparator ordering, + BinaryOperator mergeFn ) { this.baseSequence = baseSequence; @@ -147,8 +147,8 @@ public void close() throws IOException private static class CombiningYieldingAccumulator extends YieldingAccumulator { - private final Ordering ordering; - private final CombiningFunction mergeFn; + private final Comparator ordering; + private final BinaryOperator mergeFn; private final YieldingAccumulator accumulator; private OutType retVal; @@ -156,8 +156,8 @@ private static class CombiningYieldingAccumulator extends YieldingAc private boolean accumulatedSomething = false; CombiningYieldingAccumulator( - Ordering ordering, - CombiningFunction mergeFn, + Comparator ordering, + BinaryOperator mergeFn, YieldingAccumulator accumulator ) { diff --git a/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java b/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java index 47d996033290..1f473b0060a9 100644 --- a/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java +++ b/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java @@ -31,7 +31,8 @@ public class GuavaUtils { /** - * To fix semantic difference of Longs.tryParse() from Long.parseLong (Longs.tryParse() returns null for '+' started value) + * To fix semantic difference of Longs.tryParse() from Long.parseLong (Longs.tryParse() returns null for '+' started + * value) */ @Nullable public static Long tryParseLong(@Nullable String string) @@ -62,4 +63,18 @@ public static > T getEnumIfPresent(final Class enumClass, f return null; } + + /** + * If first argument is not null, return it, else return the other argument. Sort of like + * {@link com.google.common.base.Objects#firstNonNull(Object, Object)} except will not explode if both arguments are + * null. + */ + @Nullable + public static T firstNonNull(T arg1, T arg2) + { + if (arg1 == null) { + return arg2; + } + return arg1; + } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java index f1184c49adcc..2ef6f4307738 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/BaseSequence.java @@ -26,18 +26,15 @@ */ public class BaseSequence> implements Sequence { - private final IteratorMaker maker; - public BaseSequence( - IteratorMaker maker - ) + public BaseSequence(IteratorMaker maker) { this.maker = maker; } @Override - public OutType accumulate(OutType initValue, final Accumulator fn) + public OutType accumulate(final OutType initValue, final Accumulator fn) { IterType iterator = maker.make(); OutType accumulated = initValue; @@ -61,7 +58,10 @@ public OutType accumulate(OutType initValue, final Accumulator Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) { final IterType iterator = maker.make(); @@ -80,7 +80,7 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat } private Yielder makeYielder( - OutType initValue, + final OutType initValue, final YieldingAccumulator accumulator, final IterType iter ) @@ -93,14 +93,7 @@ private Yielder makeYielder( if (!accumulator.yielded()) { return Yielders.done( retVal, - new Closeable() - { - @Override - public void close() - { - maker.cleanup(iter); - } - } + (Closeable) () -> maker.cleanup(iter) ); } diff --git a/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java b/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java index 57a2290b605c..2e25542efed6 100644 --- a/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java +++ b/core/src/test/java/org/apache/druid/collections/CombiningIteratorTest.java @@ -28,12 +28,13 @@ import java.util.Comparator; import java.util.NoSuchElementException; +import java.util.function.BinaryOperator; public class CombiningIteratorTest { private CombiningIterator testingIterator; private Comparator comparator; - private CombiningFunction combiningFunction; + private BinaryOperator combiningFunction; private PeekingIterator peekIterator; @Before @@ -41,7 +42,7 @@ public void setUp() { peekIterator = EasyMock.createMock(PeekingIterator.class); comparator = EasyMock.createMock(Comparator.class); - combiningFunction = EasyMock.createMock(CombiningFunction.class); + combiningFunction = EasyMock.createMock(BinaryOperator.class); testingIterator = CombiningIterator.create(peekIterator, comparator, combiningFunction); } diff --git a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java index 835afce0604c..ba5504f25cf1 100644 --- a/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/ComplexSequenceTest.java @@ -20,7 +20,6 @@ package org.apache.druid.common.guava; import com.google.common.primitives.Ints; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -32,9 +31,23 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.function.BinaryOperator; public class ComplexSequenceTest { + // Integer::sum with more nulls + private static final BinaryOperator PLUS_NULLABLE = (arg1, arg2) -> { + if (arg1 == null) { + return arg2; + } + + if (arg2 == null) { + return arg1; + } + + return arg1 + arg2; + }; + @Test public void testComplexSequence() { @@ -80,23 +93,11 @@ private Sequence simple(int... values) private Sequence combine(Sequence sequence) { - return CombiningSequence.create(sequence, Comparators.alwaysEqual(), plus); + return CombiningSequence.create(sequence, Comparators.alwaysEqual(), PLUS_NULLABLE); } private Sequence concat(Sequence... sequences) { return Sequences.concat(Arrays.asList(sequences)); } - - private final CombiningFunction plus = (arg1, arg2) -> { - if (arg1 == null) { - return arg2; - } - - if (arg2 == null) { - return arg1; - } - - return arg1 + arg2; - }; } diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index c2a9ea37a202..b414c8444bdf 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -23,15 +23,15 @@ import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.type.TypeFactory; import com.google.common.base.Function; -import com.google.common.collect.Ordering; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.aggregation.MetricManipulationFn; import org.apache.druid.timeline.LogicalSegment; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.List; +import java.util.function.BinaryOperator; /** * The broker-side (also used by server in some cases) API for a specific Query type. @@ -82,7 +82,7 @@ public final JavaType getBySegmentResultType() * * A default implementation constructs a {@link ResultMergeQueryRunner} which creates a * {@link org.apache.druid.common.guava.CombiningSequence} using the supplied {@link QueryRunner} with - * {@link QueryToolChest#createOrderingFn(Query)} and {@link QueryToolChest#createMergeFn(Query)}} supplied by this + * {@link QueryToolChest#createComparator(Query)} and {@link QueryToolChest#createMergeFn(Query)}} supplied by this * toolchest. * * @param runner A QueryRunner that provides a series of ResultType objects in time order (ascending or descending) @@ -91,7 +91,7 @@ public final JavaType getBySegmentResultType() */ public QueryRunner mergeResults(QueryRunner runner) { - return new ResultMergeQueryRunner<>(runner, this::createOrderingFn, this::createMergeFn); + return new ResultMergeQueryRunner<>(runner, this::createComparator, this::createMergeFn); } /** @@ -99,16 +99,16 @@ public QueryRunner mergeResults(QueryRunner runner) * function is used in the default {@link ResultMergeQueryRunner} provided by * {@link QueryToolChest#mergeResults(QueryRunner)} and can be used in additional future merge implementations */ - public CombiningFunction createMergeFn(Query query) + public BinaryOperator createMergeFn(Query query) { throw new UOE("%s doesn't support merge function", query.getClass().getName()); } /** - * Creates an ordering comparator that is used to order results. This ordering function is used in the defaul + * Creates an ordering comparator that is used to order results. This comparator is used in the defaul * {@link ResultMergeQueryRunner} provided by {@link QueryToolChest#mergeResults(QueryRunner)} */ - public Ordering createOrderingFn(Query query) + public Comparator createComparator(Query query) { throw new UOE("%s doesn't support ordering function", query.getClass().getName()); } diff --git a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java index efbdf0cc5190..f3f34c61e6d1 100644 --- a/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ResultMergeQueryRunner.java @@ -19,13 +19,13 @@ package org.apache.druid.query; -import com.google.common.collect.Ordering; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.common.guava.Sequence; +import java.util.Comparator; import java.util.Map; +import java.util.function.BinaryOperator; import java.util.function.Function; /** @@ -33,17 +33,17 @@ @PublicApi public class ResultMergeQueryRunner extends BySegmentSkippingQueryRunner { - private final Function, Ordering> orderingFnGenerator; - private final Function, CombiningFunction> mergeFnGenerator; + private final Function, Comparator> comparatorGenerator; + private final Function, BinaryOperator> mergeFnGenerator; public ResultMergeQueryRunner( QueryRunner baseRunner, - Function, Ordering> orderingFnGenerator, - Function, CombiningFunction> mergeFnGenerator + Function, Comparator> comparatorGenerator, + Function, BinaryOperator> mergeFnGenerator ) { super(baseRunner); - this.orderingFnGenerator = orderingFnGenerator; + this.comparatorGenerator = comparatorGenerator; this.mergeFnGenerator = mergeFnGenerator; } @@ -53,7 +53,7 @@ public Sequence doRun(QueryRunner baseRunner, QueryPlus queryPlus, Map< Query query = queryPlus.getQuery(); return CombiningSequence.create( baseRunner.run(queryPlus, context), - orderingFnGenerator.apply(query), + comparatorGenerator.apply(query), mergeFnGenerator.apply(query)); } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index bc613dfa77fc..c50a487cfc7a 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -30,9 +30,7 @@ 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.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.ISE; @@ -67,12 +65,14 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.function.BinaryOperator; /** */ @@ -131,15 +131,15 @@ public QueryRunner mergeResults(final QueryRunner runner) } @Override - public CombiningFunction createMergeFn(Query query) + public BinaryOperator createMergeFn(Query query) { return strategySelector.strategize((GroupByQuery) query).createMergeFn(query); } @Override - public Ordering createOrderingFn(Query query) + public Comparator createComparator(Query query) { - return strategySelector.strategize((GroupByQuery) query).createOrderingFn(query); + return strategySelector.strategize((GroupByQuery) query).createComparator(query); } private Sequence initAndMergeGroupByResults( diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java index e0e0d18a243a..6071adce6289 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByBinaryFnV2.java @@ -20,7 +20,6 @@ package org.apache.druid.query.groupby.epinephelinae; import com.google.common.collect.Maps; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.granularity.AllGranularity; @@ -30,8 +29,9 @@ import org.joda.time.DateTime; import java.util.Map; +import java.util.function.BinaryOperator; -public class GroupByBinaryFnV2 implements CombiningFunction +public class GroupByBinaryFnV2 implements BinaryOperator { private final GroupByQuery query; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index 417a5adc64f2..e2a8b25f2767 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -19,9 +19,7 @@ package org.apache.druid.query.groupby.strategy; -import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListeningExecutorService; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; @@ -35,8 +33,10 @@ import org.apache.druid.segment.StorageAdapter; import javax.annotation.Nullable; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.function.BinaryOperator; public interface GroupByStrategy { @@ -79,19 +79,19 @@ QueryRunner createIntervalChunkingRunner( * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy */ @Nullable - default CombiningFunction createMergeFn(Query query) + default BinaryOperator createMergeFn(Query query) { - throw new UOE("%s doesn't support merge function", this.getClass().getName()); + throw new UOE("%s doesn't provide a merge function", this.getClass().getName()); } /** - * See {@link org.apache.druid.query.QueryToolChest#createOrderingFn(Query)}, allows + * See {@link org.apache.druid.query.QueryToolChest#createComparator(Query)}, allows * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy */ @Nullable - default Ordering createOrderingFn(Query queryParam) + default Comparator createComparator(Query queryParam) { - throw new UOE("%s doesn't support merge function", this.getClass().getName()); + throw new UOE("%s doesn't provide a comparator", this.getClass().getName()); } Sequence applyPostProcessing(Sequence results, GroupByQuery query); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index 2f64f76209f9..b9b844e3f3a4 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -26,11 +26,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.inject.Inject; import org.apache.druid.collections.BlockingPool; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.collections.NonBlockingPool; import org.apache.druid.collections.ReferenceCountingResourceHolder; import org.apache.druid.data.input.MapBasedRow; @@ -76,9 +74,11 @@ import java.io.Closeable; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; import java.util.stream.Collectors; public class GroupByStrategyV2 implements GroupByStrategy @@ -214,13 +214,13 @@ public QueryRunner createIntervalChunkingRunner( } @Override - public Ordering createOrderingFn(Query queryParam) + public Comparator createComparator(Query queryParam) { return ((GroupByQuery) queryParam).getRowOrdering(true); } @Override - public CombiningFunction createMergeFn(Query queryParam) + public BinaryOperator createMergeFn(Query queryParam) { return new GroupByBinaryFnV2((GroupByQuery) queryParam); } @@ -236,7 +236,7 @@ public Sequence mergeResults( // involve materialization) final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( baseRunner, - this::createOrderingFn, + this::createComparator, this::createMergeFn ); diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java index cab0ce336328..b2d1a2a7ffa0 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryQueryToolChest.java @@ -30,7 +30,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import com.google.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.common.guava.CombiningSequence; import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.java.util.common.JodaUtils; @@ -67,6 +66,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeMap; +import java.util.function.BinaryOperator; public class SegmentMetadataQueryQueryToolChest extends QueryToolChest { @@ -137,7 +137,7 @@ private Ordering makeOrdering(SegmentMetadataQuery query) return query.getResultOrdering(); // No two elements should be equal, so it should never merge } - private CombiningFunction createMergeFn(final SegmentMetadataQuery inQ) + private BinaryOperator createMergeFn(final SegmentMetadataQuery inQ) { return (arg1, arg2) -> mergeAnalyses(arg1, arg2, inQ.isLenientAggregatorMerge()); } diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java b/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java index f11e98c6e7e4..8d5fa52fc74e 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchBinaryFn.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.Result; @@ -29,10 +28,11 @@ import java.util.Arrays; import java.util.List; +import java.util.function.BinaryOperator; /** */ -public class SearchBinaryFn implements CombiningFunction> +public class SearchBinaryFn implements BinaryOperator> { private final SearchSortSpec searchSortSpec; private final Granularity gran; @@ -112,6 +112,6 @@ public Result apply(Result arg1, Result(timestamp, new SearchResultValue(results)); + return new Result<>(timestamp, new SearchResultValue(results)); } } diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index d635efbf4d65..238c50e05bf5 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -26,10 +26,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; import com.google.common.primitives.Ints; import com.google.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -51,9 +49,11 @@ import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; /** */ @@ -94,7 +94,7 @@ public SearchQueryQueryToolChest( } @Override - public CombiningFunction> createMergeFn( + public BinaryOperator> createMergeFn( Query> query ) { @@ -103,12 +103,9 @@ public CombiningFunction> createMergeFn( } @Override - public Ordering> createOrderingFn(Query> query) + public Comparator> createComparator(Query> query) { - return ResultGranularTimestampComparator.create( - query.getGranularity(), - query.isDescending() - ); + return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java b/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java index b11b44b5da3c..9c758f35af75 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectBinaryFn.java @@ -20,7 +20,6 @@ package org.apache.druid.query.select; import com.google.common.collect.Sets; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.Result; @@ -28,10 +27,11 @@ import java.util.List; import java.util.Set; +import java.util.function.BinaryOperator; /** */ -public class SelectBinaryFn implements CombiningFunction> +public class SelectBinaryFn implements BinaryOperator> { private final Granularity gran; private final PagingSpec pagingSpec; diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index 732fb9787d03..8b1f5f52286b 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -26,9 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; import com.google.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Comparators; @@ -52,12 +50,14 @@ import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; +import java.util.function.BinaryOperator; /** */ @@ -99,25 +99,18 @@ public SelectQueryQueryToolChest( } @Override - public CombiningFunction> createMergeFn( + public BinaryOperator> createMergeFn( Query> query ) { final SelectQuery selectQuery = (SelectQuery) query; - return new SelectBinaryFn( - selectQuery.getGranularity(), - selectQuery.getPagingSpec(), - selectQuery.isDescending() - ); + return new SelectBinaryFn(selectQuery.getGranularity(), selectQuery.getPagingSpec(), selectQuery.isDescending()); } @Override - public Ordering> createOrderingFn(Query> query) + public Comparator> createComparator(Query> query) { - return ResultGranularTimestampComparator.create( - query.getGranularity(), - query.isDescending() - ); + return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java index bda96807dfcf..6436e3df7862 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesBinaryFn.java @@ -19,7 +19,6 @@ package org.apache.druid.query.timeseries; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.Result; @@ -28,10 +27,11 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; /** */ -public class TimeseriesBinaryFn implements CombiningFunction> +public class TimeseriesBinaryFn implements BinaryOperator> { private final Granularity gran; private final List aggregations; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index d706377b8ceb..9547ef582e69 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -27,9 +27,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.common.collect.Ordering; import com.google.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; @@ -54,10 +52,12 @@ import org.joda.time.DateTime; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; /** */ @@ -100,7 +100,7 @@ public QueryRunner> mergeResults( { final QueryRunner> resultMergeQueryRunner = new ResultMergeQueryRunner>( queryRunner, - this::createOrderingFn, + this::createComparator, this::createMergeFn ) { @@ -195,23 +195,18 @@ public Sequence> doRun( } @Override - public CombiningFunction> createMergeFn( + public BinaryOperator> createMergeFn( Query> query ) { TimeseriesQuery timeseriesQuery = (TimeseriesQuery) query; - return new TimeseriesBinaryFn( - timeseriesQuery.getGranularity(), - timeseriesQuery.getAggregatorSpecs() - ); + return new TimeseriesBinaryFn(timeseriesQuery.getGranularity(), timeseriesQuery.getAggregatorSpecs()); } @Override - public Ordering> createOrderingFn(Query> query) + public Comparator> createComparator(Query> query) { - return ResultGranularTimestampComparator.create( - query.getGranularity(), query.isDescending() - ); + return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } private Result getNullTimeseriesResultValue(TimeseriesQuery query) diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java b/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java index ea29bff51150..37a500d3c3c4 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNBinaryFn.java @@ -19,7 +19,6 @@ package org.apache.druid.query.topn; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.Result; @@ -33,10 +32,11 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; /** */ -public class TopNBinaryFn implements CombiningFunction> +public class TopNBinaryFn implements BinaryOperator> { private final DimensionSpec dimSpec; private final Granularity gran; diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 0307a8adb729..ab26a5070da5 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -25,9 +25,7 @@ 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.inject.Inject; -import org.apache.druid.collections.CombiningFunction; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; @@ -52,9 +50,11 @@ import org.apache.druid.segment.DimensionHandlerUtils; import org.joda.time.DateTime; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.function.BinaryOperator; /** */ @@ -108,7 +108,7 @@ private static List prunePostAggregators(TopNQuery query) } @Override - public CombiningFunction> createMergeFn( + public BinaryOperator> createMergeFn( Query> query ) { @@ -124,11 +124,9 @@ public CombiningFunction> createMergeFn( } @Override - public Ordering> createOrderingFn(Query> query) + public Comparator> createComparator(Query> query) { - return ResultGranularTimestampComparator.create( - ((TopNQuery) query).getGranularity(), query.isDescending() - ); + return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } @Override diff --git a/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java b/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java index 497ddec08dd2..9093876bff53 100644 --- a/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java +++ b/processing/src/test/java/org/apache/druid/collections/CombiningIterableTest.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; @@ -37,17 +38,17 @@ public class CombiningIterableTest public void testMerge() { List> resultsBefore = Arrays.asList( - new Result(DateTimes.of("2011-01-01"), 1L), - new Result(DateTimes.of("2011-01-01"), 2L) + new Result<>(DateTimes.of("2011-01-01"), 1L), + new Result<>(DateTimes.of("2011-01-01"), 2L) ); Iterable> expectedResults = Collections.singletonList( - new Result(DateTimes.of("2011-01-01"), 3L) + new Result<>(DateTimes.of("2011-01-01"), 3L) ); Iterable> resultsAfter = CombiningIterable.create( resultsBefore, - (r1, r2) -> r1.getTimestamp().compareTo(r2.getTimestamp()), + Comparator.comparing(Result::getTimestamp), (arg1, arg2) -> { if (arg1 == null) { return arg2; diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index fef64af875f3..1c184ff2f074 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -55,7 +55,7 @@ public class TopNMetricSpecOptimizationsTest { - private static final List aggs = Lists.newArrayList( + private static final List AGGS = Lists.newArrayList( Iterables.concat( QueryRunnerTestHelper.commonDoubleAggregators, Lists.newArrayList( @@ -78,7 +78,7 @@ public void testShouldOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators(aggs) + .aggregators(AGGS) .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); @@ -112,7 +112,7 @@ public void testAlsoShouldOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-30T01:00:00Z") - .aggregators(aggs) + .aggregators(AGGS) .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); @@ -147,7 +147,7 @@ public void testShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-30T01:00:00Z") - .aggregators(aggs) + .aggregators(AGGS) .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); @@ -183,7 +183,7 @@ public void testAlsoShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators(aggs) + .aggregators(AGGS) .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); @@ -217,7 +217,7 @@ public void testAgainShouldNotOptimizeLexicographic() .metric(QueryRunnerTestHelper.indexMetric) .threshold(threshold) .intervals("2018-05-30T00:00:00Z/2018-05-31T00:00:00Z") - .aggregators(aggs) + .aggregators(AGGS) .postAggregators(QueryRunnerTestHelper.addRowsIndexConstant) .build(); diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 2745be95ee96..05b182b214c7 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -108,7 +108,7 @@ public BrokerServerView( this.selectors = new HashMap<>(); this.timelines = new HashMap<>(); - this.segmentFilter = metadataAndSegment -> { + this.segmentFilter = (Pair metadataAndSegment) -> { if (segmentWatcherConfig.getWatchedTiers() != null && !segmentWatcherConfig.getWatchedTiers().contains(metadataAndSegment.lhs.getTier())) { return false; From 211cab9f0295b398c11a9c7b88ca6afc9e15d337 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 18 Jul 2019 11:15:05 -0700 Subject: [PATCH 8/8] rename createComparator to createResultComparator, fix typo, firstNonNull nullable parameters --- .../org/apache/druid/common/guava/GuavaUtils.java | 2 +- .../java/org/apache/druid/query/QueryToolChest.java | 12 ++++++------ .../query/groupby/GroupByQueryQueryToolChest.java | 4 ++-- .../query/groupby/strategy/GroupByStrategy.java | 6 +++--- .../query/groupby/strategy/GroupByStrategyV2.java | 4 ++-- .../query/search/SearchQueryQueryToolChest.java | 2 +- .../query/select/SelectQueryQueryToolChest.java | 2 +- .../timeseries/TimeseriesQueryQueryToolChest.java | 4 ++-- .../druid/query/topn/TopNQueryQueryToolChest.java | 2 +- 9 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java b/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java index 1f473b0060a9..fa69f8286d1a 100644 --- a/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java +++ b/core/src/main/java/org/apache/druid/common/guava/GuavaUtils.java @@ -70,7 +70,7 @@ public static > T getEnumIfPresent(final Class enumClass, f * null. */ @Nullable - public static T firstNonNull(T arg1, T arg2) + public static T firstNonNull(@Nullable T arg1, @Nullable T arg2) { if (arg1 == null) { return arg2; diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index b414c8444bdf..a8a20a670336 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -82,7 +82,7 @@ public final JavaType getBySegmentResultType() * * A default implementation constructs a {@link ResultMergeQueryRunner} which creates a * {@link org.apache.druid.common.guava.CombiningSequence} using the supplied {@link QueryRunner} with - * {@link QueryToolChest#createComparator(Query)} and {@link QueryToolChest#createMergeFn(Query)}} supplied by this + * {@link QueryToolChest#createResultComparator(Query)} and {@link QueryToolChest#createMergeFn(Query)}} supplied by this * toolchest. * * @param runner A QueryRunner that provides a series of ResultType objects in time order (ascending or descending) @@ -91,7 +91,7 @@ public final JavaType getBySegmentResultType() */ public QueryRunner mergeResults(QueryRunner runner) { - return new ResultMergeQueryRunner<>(runner, this::createComparator, this::createMergeFn); + return new ResultMergeQueryRunner<>(runner, this::createResultComparator, this::createMergeFn); } /** @@ -101,16 +101,16 @@ public QueryRunner mergeResults(QueryRunner runner) */ public BinaryOperator createMergeFn(Query query) { - throw new UOE("%s doesn't support merge function", query.getClass().getName()); + throw new UOE("%s doesn't provide a merge function", query.getClass().getName()); } /** - * Creates an ordering comparator that is used to order results. This comparator is used in the defaul + * Creates an ordering comparator that is used to order results. This comparator is used in the default * {@link ResultMergeQueryRunner} provided by {@link QueryToolChest#mergeResults(QueryRunner)} */ - public Comparator createComparator(Query query) + public Comparator createResultComparator(Query query) { - throw new UOE("%s doesn't support ordering function", query.getClass().getName()); + throw new UOE("%s doesn't provide a result comparator", query.getClass().getName()); } /** diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index c50a487cfc7a..b81b2551d14c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -137,9 +137,9 @@ public BinaryOperator createMergeFn(Query query) } @Override - public Comparator createComparator(Query query) + public Comparator createResultComparator(Query query) { - return strategySelector.strategize((GroupByQuery) query).createComparator(query); + return strategySelector.strategize((GroupByQuery) query).createResultComparator(query); } private Sequence initAndMergeGroupByResults( diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java index e2a8b25f2767..56035a423c0c 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategy.java @@ -85,13 +85,13 @@ default BinaryOperator createMergeFn(Query query) } /** - * See {@link org.apache.druid.query.QueryToolChest#createComparator(Query)}, allows + * See {@link org.apache.druid.query.QueryToolChest#createResultComparator(Query)}, allows * {@link GroupByQueryQueryToolChest} to delegate implementation to the strategy */ @Nullable - default Comparator createComparator(Query queryParam) + default Comparator createResultComparator(Query queryParam) { - throw new UOE("%s doesn't provide a comparator", this.getClass().getName()); + throw new UOE("%s doesn't provide a result comparator", this.getClass().getName()); } Sequence applyPostProcessing(Sequence results, GroupByQuery query); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java index b9b844e3f3a4..3167f5f69905 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/strategy/GroupByStrategyV2.java @@ -214,7 +214,7 @@ public QueryRunner createIntervalChunkingRunner( } @Override - public Comparator createComparator(Query queryParam) + public Comparator createResultComparator(Query queryParam) { return ((GroupByQuery) queryParam).getRowOrdering(true); } @@ -236,7 +236,7 @@ public Sequence mergeResults( // involve materialization) final ResultMergeQueryRunner mergingQueryRunner = new ResultMergeQueryRunner<>( baseRunner, - this::createComparator, + this::createResultComparator, this::createMergeFn ); diff --git a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java index 238c50e05bf5..5011eed601dc 100644 --- a/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/search/SearchQueryQueryToolChest.java @@ -103,7 +103,7 @@ public BinaryOperator> createMergeFn( } @Override - public Comparator> createComparator(Query> query) + public Comparator> createResultComparator(Query> query) { return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } diff --git a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java index 8b1f5f52286b..e0a5855b8c74 100644 --- a/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/select/SelectQueryQueryToolChest.java @@ -108,7 +108,7 @@ public BinaryOperator> createMergeFn( } @Override - public Comparator> createComparator(Query> query) + public Comparator> createResultComparator(Query> query) { return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 9547ef582e69..023e017c7f95 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -100,7 +100,7 @@ public QueryRunner> mergeResults( { final QueryRunner> resultMergeQueryRunner = new ResultMergeQueryRunner>( queryRunner, - this::createComparator, + this::createResultComparator, this::createMergeFn ) { @@ -204,7 +204,7 @@ public BinaryOperator> createMergeFn( } @Override - public Comparator> createComparator(Query> query) + public Comparator> createResultComparator(Query> query) { return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index ab26a5070da5..c0f12b1bab64 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -124,7 +124,7 @@ public BinaryOperator> createMergeFn( } @Override - public Comparator> createComparator(Query> query) + public Comparator> createResultComparator(Query> query) { return ResultGranularTimestampComparator.create(query.getGranularity(), query.isDescending()); }