From 10e57d5f9ed003e032c82240045125002903a5bb Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Feb 2019 14:04:13 -0800 Subject: [PATCH 01/91] Moved Scan Builder to Druids class and started on Scan Benchmark setup --- .../druid/benchmark/query/ScanBenchmark.java | 371 ++++++++++++++++++ .../benchmark/query/SelectBenchmark.java | 1 - .../java/org/apache/druid/query/Druids.java | 157 ++++++++ .../apache/druid/query/scan/ScanQuery.java | 171 +------- .../apache/druid/query/DoubleStorageTest.java | 14 +- .../query/scan/MultiSegmentScanQueryTest.java | 5 +- .../druid/query/scan/ScanQueryRunnerTest.java | 5 +- .../sql/calcite/BaseCalciteQueryTest.java | 5 +- 8 files changed, 550 insertions(+), 179 deletions(-) create mode 100644 benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java new file mode 100644 index 000000000000..4085467a3a96 --- /dev/null +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -0,0 +1,371 @@ +/* + * 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.ObjectMapper; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.Files; +import org.apache.commons.io.FileUtils; +import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; +import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; +import org.apache.druid.benchmark.datagen.BenchmarkSchemas; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.hll.HyperLogLogHash; +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.guava.Sequence; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.Druids; +import org.apache.druid.query.FinalizeResultsQueryRunner; +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.QueryToolChest; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import org.apache.druid.query.extraction.DimExtractionFn; +import org.apache.druid.query.extraction.IdentityExtractionFn; +import org.apache.druid.query.extraction.LowerExtractionFn; +import org.apache.druid.query.extraction.StrlenExtractionFn; +import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.extraction.UpperExtractionFn; +import org.apache.druid.query.filter.AndDimFilter; +import org.apache.druid.query.filter.BoundDimFilter; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.scan.ScanQueryConfig; +import org.apache.druid.query.scan.ScanQueryEngine; +import org.apache.druid.query.scan.ScanQueryQueryToolChest; +import org.apache.druid.query.scan.ScanQueryRunnerFactory; +import org.apache.druid.query.search.SearchQueryConfig; +import org.apache.druid.query.search.SearchQueryQueryToolChest; +import org.apache.druid.query.search.SearchQueryRunnerFactory; +import org.apache.druid.query.search.SearchStrategySelector; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.IndexMergerV9; +import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.serde.ComplexMetrics; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +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 java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 10) +@Measurement(iterations = 25) +public class ScanBenchmark +{ + @Param({"1"}) + private int numSegments; + + @Param({"750000"}) + private int rowsPerSegment; + + @Param({"basic.A"}) + private String schemaAndQuery; + + @Param({"1000"}) + private int limit; + + private static final Logger log = new Logger(ScanBenchmark.class); + private static final ObjectMapper JSON_MAPPER; + private static final IndexMergerV9 INDEX_MERGER_V9; + private static final IndexIO INDEX_IO; + + private List incIndexes; + private List qIndexes; + + private QueryRunnerFactory factory; + private BenchmarkSchemaInfo schemaInfo; + private Druids.ScanQueryBuilder queryBuilder; + private ScanQuery query; + private File tmpDir; + + private ExecutorService executorService; + + static { + JSON_MAPPER = new DefaultObjectMapper(); + INDEX_IO = new IndexIO( + JSON_MAPPER, + () -> 0 + ); + INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); + } + + private static final Map> SCHEMA_QUERY_MAP = new LinkedHashMap<>(); + + private void setupQueries() + { + // queries for the basic schema + final Map basicQueries = new LinkedHashMap<>(); + final BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + + final List queryTypes = ImmutableList.of("A", "B", "C", "D"); + for (final String eachType : queryTypes) { + basicQueries.put(eachType, makeQuery(eachType, basicSchema)); + } + + SCHEMA_QUERY_MAP.put("basic", basicQueries); + } + + private static Druids.ScanQueryBuilder makeQuery(final String name, final BenchmarkSchemaInfo basicSchema) + { + switch (name) { + case "A": + return basicA(basicSchema); + case "B": + return basicB(basicSchema); + case "C": + return basicC(basicSchema); + case "D": + return basicD(basicSchema); + default: + return null; + } + } + + private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) + { + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + + return Druids.newScanQueryBuilder() + .dataSource("blah") + .intervals(intervalSpec) + .query("123"); + } + + private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) + { + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + + final List dimUniformFilterVals = new ArrayList<>(); + int resultNum = (int) (100000 * 0.1); + int step = 100000 / resultNum; + for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { + dimUniformFilterVals.add(String.valueOf(i)); + } + + List dimHyperUniqueFilterVals = new ArrayList<>(); + resultNum = (int) (100000 * 0.1); + step = 100000 / resultNum; + for (int i = 0; i < 100001 && dimHyperUniqueFilterVals.size() < resultNum; i += step) { + dimHyperUniqueFilterVals.add(String.valueOf(i)); + } + + final List dimFilters = new ArrayList<>(); + dimFilters.add(new InDimFilter("dimUniform", dimUniformFilterVals, null)); + dimFilters.add(new InDimFilter("dimHyperUnique", dimHyperUniqueFilterVals, null)); + + return Druids.newScanQueryBuilder(); // TODO + } + + private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) + { + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + + final List dimUniformFilterVals = new ArrayList<>(); + final int resultNum = (int) (100000 * 0.1); + final int step = 100000 / resultNum; + for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { + dimUniformFilterVals.add(String.valueOf(i)); + } + + final String dimName = "dimUniform"; + final List dimFilters = new ArrayList<>(); + dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, IdentityExtractionFn.getInstance())); + dimFilters.add(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())); + dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, new DimExtractionFn() + { + @Override + public byte[] getCacheKey() + { + return new byte[]{0xF}; + } + + @Override + public String apply(String value) + { + return String.valueOf(Long.parseLong(value) + 1); + } + + @Override + public boolean preservesOrdering() + { + return false; + } + + @Override + public ExtractionType getExtractionType() + { + return ExtractionType.ONE_TO_ONE; + } + }, null)); + dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new LowerExtractionFn(null))); + dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new UpperExtractionFn(null))); + dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new SubstringDimExtractionFn(1, 3))); + + return Druids.newScanQueryBuilder(); // TODO + } + + private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) + { + final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList(basicSchema.getDataInterval()) + ); + + final List dimUniformFilterVals = new ArrayList<>(); + final int resultNum = (int) (100000 * 0.1); + final int step = 100000 / resultNum; + for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { + dimUniformFilterVals.add(String.valueOf(i)); + } + + final String dimName = "dimUniform"; + final List dimFilters = new ArrayList<>(); + dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, null)); + dimFilters.add(new SelectorDimFilter(dimName, "3", null)); + dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)); + + return Druids.newScanQueryBuilder(); // TODO + } + + @Setup + public void setup() throws IOException + { + log.info("SETUP CALLED AT " + +System.currentTimeMillis()); + + if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { + ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault())); + } + executorService = Execs.multiThreaded(numSegments, "SearchThreadPool"); + + setupQueries(); + + String[] schemaQuery = schemaAndQuery.split("\\."); + String schemaName = schemaQuery[0]; + String queryName = schemaQuery[1]; + + schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + queryBuilder = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); + queryBuilder.limit(limit); + query = queryBuilder.build(); + + incIndexes = new ArrayList<>(); + for (int i = 0; i < numSegments; i++) { + log.info("Generating rows for segment " + i); + BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + schemaInfo.getColumnSchemas(), + System.currentTimeMillis(), + schemaInfo.getDataInterval(), + rowsPerSegment + ); + + IncrementalIndex incIndex = makeIncIndex(); + + for (int j = 0; j < rowsPerSegment; j++) { + InputRow row = gen.nextRow(); + if (j % 10000 == 0) { + log.info(j + " rows generated."); + } + incIndex.add(row); + } + incIndexes.add(incIndex); + } + + tmpDir = Files.createTempDir(); + log.info("Using temp dir: " + tmpDir.getAbsolutePath()); + + qIndexes = new ArrayList<>(); + for (int i = 0; i < numSegments; i++) { + File indexFile = INDEX_MERGER_V9.persist( + incIndexes.get(i), + tmpDir, + new IndexSpec(), + null + ); + + QueryableIndex qIndex = INDEX_IO.loadIndex(indexFile); + qIndexes.add(qIndex); + } + + final ScanQueryConfig config = new ScanQueryConfig().setLegacy(false); + factory = new ScanQueryRunnerFactory( + new ScanQueryQueryToolChest( + config, + DefaultGenericQueryMetricsFactory.instance() + ), + new ScanQueryEngine() + ); + } + + @TearDown + public void tearDown() throws IOException + { + FileUtils.deleteDirectory(tmpDir); + } + + private IncrementalIndex makeIncIndex() + { + return new IncrementalIndex.Builder() + .setSimpleTestingIndexSchema(schemaInfo.getAggsArray()) + .setReportParseExceptions(false) + .setMaxRowCount(rowsPerSegment) + .buildOnheap(); + } + + private static List runQuery(QueryRunnerFactory factory, QueryRunner runner, Query query) + { + QueryToolChest toolChest = factory.getToolchest(); + QueryRunner theRunner = new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(toolChest.preMergeQueryDecoration(runner)), + toolChest + ); + + Sequence queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>()); + return queryResult.toList(); + } +} diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java index bd2126465664..dbc694215483 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SelectBenchmark.java @@ -315,7 +315,6 @@ public void queryIncrementalIndex(Blackhole blackhole) } } - @Benchmark @BenchmarkMode(Mode.AverageTime) @OutputTimeUnit(TimeUnit.MICROSECONDS) diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 2c2a6537b85e..6ef7f93afb61 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -36,6 +36,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.metadata.metadata.ColumnIncluderator; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; +import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.search.ContainsSearchQuerySpec; import org.apache.druid.query.search.FragmentSearchQuerySpec; import org.apache.druid.query.search.InsensitiveContainsSearchQuerySpec; @@ -896,6 +897,162 @@ public static SelectQueryBuilder newSelectQueryBuilder() return new SelectQueryBuilder(); } + /** + * A Builder for ScanQuery. + *

+ * Required: dataSource(), intervals() must be called before build() + *

+ * Usage example: + *


+   *   ScanQuery query = new ScanQueryBuilder()
+   *                                  .dataSource("Example")
+   *                                  .interval("2010/2013")
+   *                                  .build();
+   * 
+ * + * @see ScanQuery + */ + public static class ScanQueryBuilder + { + private DataSource dataSource; + private QuerySegmentSpec querySegmentSpec; + private VirtualColumns virtualColumns; + private Map context; + private String resultFormat; + private int batchSize; + private long limit; + private DimFilter dimFilter; + private List columns; + private Boolean legacy; + + public ScanQueryBuilder() + { + dataSource = null; + querySegmentSpec = null; + virtualColumns = null; + context = null; + resultFormat = null; + batchSize = 0; + limit = 0; + dimFilter = null; + columns = new ArrayList<>(); + legacy = null; + } + + public ScanQuery build() + { + return new ScanQuery( + dataSource, + querySegmentSpec, + virtualColumns, + resultFormat, + batchSize, + limit, + dimFilter, + columns, + legacy, + context + ); + } + + public static ScanQueryBuilder copy(ScanQuery query) + { + return new ScanQueryBuilder() + .dataSource(query.getDataSource()) + .intervals(query.getQuerySegmentSpec()) + .virtualColumns(query.getVirtualColumns()) + .resultFormat(query.getResultFormat()) + .batchSize(query.getBatchSize()) + .limit(query.getLimit()) + .filters(query.getFilter()) + .columns(query.getColumns()) + .legacy(query.isLegacy()) + .context(query.getContext()); + } + + public ScanQueryBuilder dataSource(String ds) + { + dataSource = new TableDataSource(ds); + return this; + } + + public ScanQueryBuilder dataSource(DataSource ds) + { + dataSource = ds; + return this; + } + + public ScanQueryBuilder intervals(QuerySegmentSpec q) + { + querySegmentSpec = q; + return this; + } + + public ScanQueryBuilder virtualColumns(VirtualColumns virtualColumns) + { + this.virtualColumns = virtualColumns; + return this; + } + + public ScanQueryBuilder virtualColumns(VirtualColumn... virtualColumns) + { + return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns))); + } + + public ScanQueryBuilder context(Map c) + { + context = c; + return this; + } + + public ScanQueryBuilder resultFormat(String r) + { + resultFormat = r; + return this; + } + + public ScanQueryBuilder batchSize(int b) + { + batchSize = b; + return this; + } + + public ScanQueryBuilder limit(long l) + { + limit = l; + return this; + } + + public ScanQueryBuilder filters(DimFilter f) + { + dimFilter = f; + return this; + } + + public ScanQueryBuilder columns(List c) + { + columns = c; + return this; + } + + public ScanQueryBuilder columns(String... c) + { + columns = Arrays.asList(c); + return this; + } + + public ScanQueryBuilder legacy(Boolean legacy) + { + this.legacy = legacy; + return this; + } + } + + public static ScanQueryBuilder newScanQueryBuilder() + { + return new ScanQueryBuilder(); + } + /** * A Builder for DataSourceMetadataQuery. *

diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index e780d36e10bd..c6d1583b59c2 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -24,15 +24,12 @@ import com.google.common.base.Preconditions; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; +import org.apache.druid.query.Druids; import org.apache.druid.query.Query; -import org.apache.druid.query.TableDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; -import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -137,30 +134,30 @@ public Boolean isLegacy() public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig) { - return ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build(); + return Druids.ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build(); } @Override public Query withQuerySegmentSpec(QuerySegmentSpec querySegmentSpec) { - return ScanQueryBuilder.copy(this).intervals(querySegmentSpec).build(); + return Druids.ScanQueryBuilder.copy(this).intervals(querySegmentSpec).build(); } @Override public Query withDataSource(DataSource dataSource) { - return ScanQueryBuilder.copy(this).dataSource(dataSource).build(); + return Druids.ScanQueryBuilder.copy(this).dataSource(dataSource).build(); } @Override public Query withOverriddenContext(Map contextOverrides) { - return ScanQueryBuilder.copy(this).context(computeOverriddenContext(getContext(), contextOverrides)).build(); + return Druids.ScanQueryBuilder.copy(this).context(computeOverriddenContext(getContext(), contextOverrides)).build(); } public ScanQuery withDimFilter(DimFilter dimFilter) { - return ScanQueryBuilder.copy(this).filters(dimFilter).build(); + return Druids.ScanQueryBuilder.copy(this).filters(dimFilter).build(); } @Override @@ -206,160 +203,4 @@ public String toString() ", legacy=" + legacy + '}'; } - - /** - * A Builder for ScanQuery. - *

- * Required: dataSource(), intervals() must be called before build() - *

- * Usage example: - *


-   *   ScanQuery query = new ScanQueryBuilder()
-   *                                  .dataSource("Example")
-   *                                  .interval("2010/2013")
-   *                                  .build();
-   * 
- * - * @see ScanQuery - */ - public static class ScanQueryBuilder - { - private DataSource dataSource; - private QuerySegmentSpec querySegmentSpec; - private VirtualColumns virtualColumns; - private Map context; - private String resultFormat; - private int batchSize; - private long limit; - private DimFilter dimFilter; - private List columns; - private Boolean legacy; - - public ScanQueryBuilder() - { - dataSource = null; - querySegmentSpec = null; - virtualColumns = null; - context = null; - resultFormat = null; - batchSize = 0; - limit = 0; - dimFilter = null; - columns = new ArrayList<>(); - legacy = null; - } - - public ScanQuery build() - { - return new ScanQuery( - dataSource, - querySegmentSpec, - virtualColumns, - resultFormat, - batchSize, - limit, - dimFilter, - columns, - legacy, - context - ); - } - - public static ScanQueryBuilder copy(ScanQuery query) - { - return new ScanQueryBuilder() - .dataSource(query.getDataSource()) - .intervals(query.getQuerySegmentSpec()) - .virtualColumns(query.getVirtualColumns()) - .resultFormat(query.getResultFormat()) - .batchSize(query.getBatchSize()) - .limit(query.getLimit()) - .filters(query.getFilter()) - .columns(query.getColumns()) - .legacy(query.isLegacy()) - .context(query.getContext()); - } - - public ScanQueryBuilder dataSource(String ds) - { - dataSource = new TableDataSource(ds); - return this; - } - - public ScanQueryBuilder dataSource(DataSource ds) - { - dataSource = ds; - return this; - } - - public ScanQueryBuilder intervals(QuerySegmentSpec q) - { - querySegmentSpec = q; - return this; - } - - public ScanQueryBuilder virtualColumns(VirtualColumns virtualColumns) - { - this.virtualColumns = virtualColumns; - return this; - } - - public ScanQueryBuilder virtualColumns(VirtualColumn... virtualColumns) - { - return virtualColumns(VirtualColumns.create(Arrays.asList(virtualColumns))); - } - - public ScanQueryBuilder context(Map c) - { - context = c; - return this; - } - - public ScanQueryBuilder resultFormat(String r) - { - resultFormat = r; - return this; - } - - public ScanQueryBuilder batchSize(int b) - { - batchSize = b; - return this; - } - - public ScanQueryBuilder limit(long l) - { - limit = l; - return this; - } - - public ScanQueryBuilder filters(DimFilter f) - { - dimFilter = f; - return this; - } - - public ScanQueryBuilder columns(List c) - { - columns = c; - return this; - } - - public ScanQueryBuilder columns(String... c) - { - columns = Arrays.asList(c); - return this; - } - - public ScanQueryBuilder legacy(Boolean legacy) - { - this.legacy = legacy; - return this; - } - } - - public static ScanQueryBuilder newScanQueryBuilder() - { - return new ScanQueryBuilder(); - } } diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index e6f0b5db0c55..a85b41d61165 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -97,14 +97,14 @@ public class DoubleStorageTest new ScanQueryEngine() ); - private ScanQuery.ScanQueryBuilder newTestQuery() + private Druids.ScanQueryBuilder newTestQuery() { - return ScanQuery.newScanQueryBuilder() - .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) - .columns(Collections.emptyList()) - .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) - .limit(Integer.MAX_VALUE) - .legacy(false); + return Druids.newScanQueryBuilder() + .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) + .columns(Collections.emptyList()) + .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) + .limit(Integer.MAX_VALUE) + .legacy(false); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index e4f837ba1c57..e138d73a1fe8 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -31,6 +31,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; @@ -177,9 +178,9 @@ public MultiSegmentScanQueryTest(int limit, int batchSize) this.batchSize = batchSize; } - private ScanQuery.ScanQueryBuilder newBuilder() + private Druids.ScanQueryBuilder newBuilder() { - return ScanQuery.newScanQueryBuilder() + return Druids.newScanQueryBuilder() .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) .intervals(SelectQueryRunnerTest.I_0112_0114_SPEC) .batchSize(batchSize) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index faeea4f83808..ec18bdbcdc68 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.Druids; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; @@ -136,9 +137,9 @@ public ScanQueryRunnerTest(final QueryRunner runner, final boolean legacy) this.legacy = legacy; } - private ScanQuery.ScanQueryBuilder newTestQuery() + private Druids.ScanQueryBuilder newTestQuery() { - return ScanQuery.newScanQueryBuilder() + return Druids.newScanQueryBuilder() .dataSource(new TableDataSource(QueryRunnerTestHelper.dataSource)) .columns(Collections.emptyList()) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index b5d083100c75..0d25a92e8c83 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -32,6 +32,7 @@ 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.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryRunnerFactoryConglomerate; @@ -359,9 +360,9 @@ public static ExpressionPostAggregator EXPRESSION_POST_AGG(final String name, fi return new ExpressionPostAggregator(name, expression, null, CalciteTests.createExprMacroTable()); } - public static ScanQuery.ScanQueryBuilder newScanQueryBuilder() + public static Druids.ScanQueryBuilder newScanQueryBuilder() { - return new ScanQuery.ScanQueryBuilder().resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) .legacy(false); } From dd4ec1ac9c1194144e3ec98b811adc59598c8d8c Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Feb 2019 15:12:17 -0800 Subject: [PATCH 02/91] Need to form queries --- .../druid/benchmark/query/ScanBenchmark.java | 90 ++++++++++++++++--- 1 file changed, 79 insertions(+), 11 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 4085467a3a96..e177065d3bbd 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -20,19 +20,17 @@ package org.apache.druid.benchmark.query; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import com.google.common.io.Files; import org.apache.commons.io.FileUtils; import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.Row; import org.apache.druid.hll.HyperLogLogHash; 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.guava.Sequence; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; @@ -43,6 +41,7 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; import org.apache.druid.query.extraction.DimExtractionFn; import org.apache.druid.query.extraction.IdentityExtractionFn; @@ -50,7 +49,6 @@ import org.apache.druid.query.extraction.StrlenExtractionFn; import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.extraction.UpperExtractionFn; -import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; @@ -60,28 +58,32 @@ import org.apache.druid.query.scan.ScanQueryEngine; import org.apache.druid.query.scan.ScanQueryQueryToolChest; import org.apache.druid.query.scan.ScanQueryRunnerFactory; -import org.apache.druid.query.search.SearchQueryConfig; -import org.apache.druid.query.search.SearchQueryQueryToolChest; -import org.apache.druid.query.search.SearchQueryRunnerFactory; -import org.apache.druid.query.search.SearchStrategySelector; +import org.apache.druid.query.scan.ScanResultValue; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.timeline.SegmentId; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; 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 java.io.File; import java.io.IOException; @@ -92,6 +94,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; @State(Scope.Benchmark) @Fork(value = 1) @@ -99,9 +102,12 @@ @Measurement(iterations = 25) public class ScanBenchmark { - @Param({"1"}) + @Param({"1", "4"}) private int numSegments; + @Param({"1", "2"}) + private int numProcessingThreads; + @Param({"750000"}) private int rowsPerSegment; @@ -281,7 +287,7 @@ public void setup() throws IOException if (ComplexMetrics.getSerdeForType("hyperUnique") == null) { ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(HyperLogLogHash.getDefault())); } - executorService = Execs.multiThreaded(numSegments, "SearchThreadPool"); + executorService = Execs.multiThreaded(numProcessingThreads, "ScanThreadPool"); setupQueries(); @@ -368,4 +374,66 @@ private static List runQuery(QueryRunnerFactory factory, QueryRunner runn Sequence queryResult = theRunner.run(QueryPlus.wrap(query), new HashMap<>()); return queryResult.toList(); } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleIncrementalIndex(Blackhole blackhole) + { + QueryRunner runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + SegmentId.dummy("incIndex"), + new IncrementalIndexSegment(incIndexes.get(0), SegmentId.dummy("incIndex")) + ); + + List results = ScanBenchmark.runQuery(factory, runner, query); + blackhole.consume(results); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void querySingleQueryableIndex(Blackhole blackhole) + { + final QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + SegmentId.dummy("qIndex"), + new QueryableIndexSegment(qIndexes.get(0), SegmentId.dummy("qIndex")) + ); + + List results = ScanBenchmark.runQuery(factory, runner, query); + blackhole.consume(results); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void queryMultiQueryableIndex(Blackhole blackhole) + { + List> runners = new ArrayList<>(); + QueryToolChest toolChest = factory.getToolchest(); + for (int i = 0; i < numSegments; i++) { + String segmentName = "qIndex" + i; + final QueryRunner> runner = QueryBenchmarkUtil.makeQueryRunner( + factory, + SegmentId.dummy(segmentName), + new QueryableIndexSegment(qIndexes.get(i), SegmentId.dummy(segmentName)) + ); + runners.add(toolChest.preMergeQueryDecoration(runner)); + } + + QueryRunner theRunner = toolChest.postMergeQueryDecoration( + new FinalizeResultsQueryRunner<>( + toolChest.mergeResults(factory.mergeRunners(executorService, runners)), + toolChest + ) + ); + + Sequence> queryResult = theRunner.run( + QueryPlus.wrap(query), + new HashMap<>() + ); + List> results = queryResult.toList(); + blackhole.consume(results); + } } From 26930f8d2021d1d62322c54e0ec35e260137ab1d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Feb 2019 16:38:49 -0800 Subject: [PATCH 03/91] It runs. --- .../druid/benchmark/query/ScanBenchmark.java | 71 +++++-------------- 1 file changed, 18 insertions(+), 53 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index e177065d3bbd..4c66e8373565 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -43,12 +43,7 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; -import org.apache.druid.query.extraction.DimExtractionFn; -import org.apache.druid.query.extraction.IdentityExtractionFn; -import org.apache.druid.query.extraction.LowerExtractionFn; import org.apache.druid.query.extraction.StrlenExtractionFn; -import org.apache.druid.query.extraction.SubstringDimExtractionFn; -import org.apache.druid.query.extraction.UpperExtractionFn; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; @@ -96,10 +91,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +/* Works with 4GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 10) -@Measurement(iterations = 25) +@Warmup(iterations = 5) +@Measurement(iterations = 5) public class ScanBenchmark { @Param({"1", "4"}) @@ -108,7 +104,7 @@ public class ScanBenchmark @Param({"1", "2"}) private int numProcessingThreads; - @Param({"750000"}) + @Param({"250000"}) private int rowsPerSegment; @Param({"basic.A"}) @@ -174,14 +170,15 @@ private static Druids.ScanQueryBuilder makeQuery(final String name, final Benchm } } + /* Just get everything */ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); return Druids.newScanQueryBuilder() .dataSource("blah") - .intervals(intervalSpec) - .query("123"); + .intervals(intervalSpec); + } private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) @@ -202,11 +199,11 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch dimHyperUniqueFilterVals.add(String.valueOf(i)); } - final List dimFilters = new ArrayList<>(); - dimFilters.add(new InDimFilter("dimUniform", dimUniformFilterVals, null)); - dimFilters.add(new InDimFilter("dimHyperUnique", dimHyperUniqueFilterVals, null)); + DimFilter filter = new InDimFilter("dimHyperUnique", dimHyperUniqueFilterVals, null); - return Druids.newScanQueryBuilder(); // TODO + return Druids.newScanQueryBuilder() + .filters(filter) + .intervals(intervalSpec); } private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) @@ -221,40 +218,9 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch } final String dimName = "dimUniform"; - final List dimFilters = new ArrayList<>(); - dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, IdentityExtractionFn.getInstance())); - dimFilters.add(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())); - dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, new DimExtractionFn() - { - @Override - public byte[] getCacheKey() - { - return new byte[]{0xF}; - } - - @Override - public String apply(String value) - { - return String.valueOf(Long.parseLong(value) + 1); - } - - @Override - public boolean preservesOrdering() - { - return false; - } - - @Override - public ExtractionType getExtractionType() - { - return ExtractionType.ONE_TO_ONE; - } - }, null)); - dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new LowerExtractionFn(null))); - dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new UpperExtractionFn(null))); - dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, new SubstringDimExtractionFn(1, 3))); - - return Druids.newScanQueryBuilder(); // TODO + return Druids.newScanQueryBuilder() + .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) + .intervals(intervalSpec); // TODO } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -271,12 +237,11 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch } final String dimName = "dimUniform"; - final List dimFilters = new ArrayList<>(); - dimFilters.add(new InDimFilter(dimName, dimUniformFilterVals, null)); - dimFilters.add(new SelectorDimFilter(dimName, "3", null)); - dimFilters.add(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)); - return Druids.newScanQueryBuilder(); // TODO + + return Druids.newScanQueryBuilder() + .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) + .intervals(intervalSpec); // TODO } @Setup From 7a6080f636ab2ead5ff85a29ea6b9cc04d93b353 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Feb 2019 18:00:58 -0800 Subject: [PATCH 04/91] Stuff for time-ordered scan query --- docs/content/querying/scan-query.md | 71 ++--- .../apache/druid/query/scan/ScanQuery.java | 29 +- .../druid/query/scan/ScanQueryConfig.java | 9 + .../druid/query/scan/ScanQueryEngine.java | 3 +- .../query/scan/ScanQueryQueryToolChest.java | 128 ++++++++- .../druid/query/scan/ScanQueryRunnerTest.java | 258 +++++++++++++++++- .../druid/query/scan/ScanQuerySpecTest.java | 4 +- .../query/scan/ScanResultValueSerdeTest.java | 71 +++++ .../druid/client/CachingClusteredClient.java | 2 +- .../druid/sql/calcite/rel/DruidQuery.java | 3 +- 10 files changed, 520 insertions(+), 58 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index cb4791f7ec06..d7866abbae7d 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -24,7 +24,13 @@ title: "Scan query" # Scan query -Scan query returns raw Druid rows in streaming mode. +The Scan query returns raw Druid rows in streaming mode. The biggest difference between the Select query and the Scan +query is that the Scan query does not retain all the returned rows in memory before they are returned to the client +(except when time-ordering is used). The Select query _will_ retain the rows in memory, causing memory pressure if too +many rows are returned. The Scan query can return all the rows without issuing another pagination query, which is +extremely useful when directly querying against historical or realtime nodes. + +An example Scan query object is shown below: ```json { @@ -36,28 +42,29 @@ Scan query returns raw Druid rows in streaming mode. "2013-01-01/2013-01-02" ], "batchSize":20480, - "limit":5 + "limit":3 } ``` -There are several main parts to a scan query: +The following are the main parameters for Scan queries: |property|description|required?| |--------|-----------|---------| |queryType|This String should always be "scan"; this is the first thing Druid looks at to figure out how to interpret the query|yes| |dataSource|A String or Object defining the data source to query, very similar to a table in a relational database. See [DataSource](../querying/datasource.html) for more information.|yes| |intervals|A JSON Object representing ISO-8601 Intervals. This defines the time ranges to run the query over.|yes| -|resultFormat|How result represented, list or compactedList or valueVector. Currently only `list` and `compactedList` are supported. Default is `list`|no| +|resultFormat|How the results are represented: list, compactedList or valueVector. Currently only `list` and `compactedList` are supported. Default is `list`|no| |filter|See [Filters](../querying/filters.html)|no| |columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no| |batchSize|How many rows buffered before return to client. Default is `20480`|no| |limit|How many rows to return. If not specified, all rows will be returned.|no| +|timeOrder|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsTimeOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsTimeOrderedInMemory` will not be time-ordered and default to a timeOrder of "none".|none| |legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no| |context|An additional JSON Object which can be used to specify certain flags.|no| ## Example results -The format of the result when resultFormat equals to `list`: +The format of the result when resultFormat equals `list`: ```json [{ @@ -123,41 +130,11 @@ The format of the result when resultFormat equals to `list`: "delta" : 77.0, "variation" : 77.0, "deleted" : 0.0 - }, { - "timestamp" : "2013-01-01T00:00:00.000Z", - "robot" : "0", - "namespace" : "article", - "anonymous" : "0", - "unpatrolled" : "0", - "page" : "113_U.S._73", - "language" : "en", - "newpage" : "1", - "user" : "MZMcBride", - "count" : 1.0, - "added" : 70.0, - "delta" : 70.0, - "variation" : 70.0, - "deleted" : 0.0 - }, { - "timestamp" : "2013-01-01T00:00:00.000Z", - "robot" : "0", - "namespace" : "article", - "anonymous" : "0", - "unpatrolled" : "0", - "page" : "113_U.S._756", - "language" : "en", - "newpage" : "1", - "user" : "MZMcBride", - "count" : 1.0, - "added" : 68.0, - "delta" : 68.0, - "variation" : 68.0, - "deleted" : 0.0 } ] } ] ``` -The format of the result when resultFormat equals to `compactedList`: +The format of the result when resultFormat equals `compactedList`: ```json [{ @@ -168,17 +145,18 @@ The format of the result when resultFormat equals to `compactedList`: "events" : [ ["2013-01-01T00:00:00.000Z", "1", "article", "0", "0", "11._korpus_(NOVJ)", "sl", "0", "EmausBot", 1.0, 39.0, 39.0, 39.0, 0.0], ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "112_U.S._580", "en", "1", "MZMcBride", 1.0, 70.0, 70.0, 70.0, 0.0], - ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._243", "en", "1", "MZMcBride", 1.0, 77.0, 77.0, 77.0, 0.0], - ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._73", "en", "1", "MZMcBride", 1.0, 70.0, 70.0, 70.0, 0.0], - ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._756", "en", "1", "MZMcBride", 1.0, 68.0, 68.0, 68.0, 0.0] + ["2013-01-01T00:00:00.000Z", "0", "article", "0", "0", "113_U.S._243", "en", "1", "MZMcBride", 1.0, 77.0, 77.0, 77.0, 0.0] ] } ] ``` -The biggest difference between select query and scan query is that, scan query doesn't retain all rows in memory before rows can be returned to client. -It will cause memory pressure if too many rows required by select query. -Scan query doesn't have this issue. -Scan query can return all rows without issuing another pagination query, which is extremely useful when query against Historical or realtime node directly. +## Time Ordering + +The Scan query currently supports ordering based on timestamp for non-legacy queries where the limit is less than +`druid.query.scan.maxRowsTimeOrderedInMemory` rows. The default value of `druid.query.scan.maxRowsTimeOrderedInMemory` +is 100000 rows. The reasoning behind this limit is that the current implementation of time ordering sorts all returned +records in memory. Attempting to load too many rows into memory runs the risk of Broker nodes running out of memory. +The limit can be configured based on server memory and number of dimensions being queried. ## Legacy mode @@ -194,3 +172,10 @@ Legacy mode can be triggered either by passing `"legacy" : true` in your query J `druid.query.scan.legacy = true` on your Druid nodes. If you were previously using the scan-query contrib extension, the best way to migrate is to activate legacy mode during a rolling upgrade, then switch it off after the upgrade is complete. + +## Configuration Properties + +|property|description|values|default| +|--------|-----------|------|-------| +|druid.query.scan.maxRowsTimeOrderedInMemory|An integer in the range [0, 2147483647]|100000| +|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index e780d36e10bd..db4d2eb29fdb 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -43,6 +43,10 @@ public class ScanQuery extends BaseQuery public static final String RESULT_FORMAT_COMPACTED_LIST = "compactedList"; public static final String RESULT_FORMAT_VALUE_VECTOR = "valueVector"; + public static final String TIME_ORDER_ASCENDING = "ascending"; + public static final String TIME_ORDER_DESCENDING = "descending"; + public static final String TIME_ORDER_NONE = "none"; + private final VirtualColumns virtualColumns; private final String resultFormat; private final int batchSize; @@ -50,6 +54,7 @@ public class ScanQuery extends BaseQuery private final DimFilter dimFilter; private final List columns; private final Boolean legacy; + private final String timeOrder; @JsonCreator public ScanQuery( @@ -62,7 +67,8 @@ public ScanQuery( @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("columns") List columns, @JsonProperty("legacy") Boolean legacy, - @JsonProperty("context") Map context + @JsonProperty("context") Map context, + @JsonProperty("timeOrder") String timeOrder ) { super(dataSource, querySegmentSpec, false, context); @@ -75,6 +81,7 @@ public ScanQuery( this.dimFilter = dimFilter; this.columns = columns; this.legacy = legacy; + this.timeOrder = timeOrder == null ? TIME_ORDER_NONE : timeOrder; } @JsonProperty @@ -101,6 +108,12 @@ public long getLimit() return limit; } + @JsonProperty + public String getTimeOrder() + { + return timeOrder; + } + @Override public boolean hasFilters() { @@ -234,6 +247,7 @@ public static class ScanQueryBuilder private DimFilter dimFilter; private List columns; private Boolean legacy; + private String timeOrder; public ScanQueryBuilder() { @@ -247,6 +261,7 @@ public ScanQueryBuilder() dimFilter = null; columns = new ArrayList<>(); legacy = null; + timeOrder = null; } public ScanQuery build() @@ -261,7 +276,8 @@ public ScanQuery build() dimFilter, columns, legacy, - context + context, + timeOrder ); } @@ -277,7 +293,8 @@ public static ScanQueryBuilder copy(ScanQuery query) .filters(query.getFilter()) .columns(query.getColumns()) .legacy(query.isLegacy()) - .context(query.getContext()); + .context(query.getContext()) + .timeOrder(query.getTimeOrder()); } public ScanQueryBuilder dataSource(String ds) @@ -339,6 +356,12 @@ public ScanQueryBuilder filters(DimFilter f) return this; } + public ScanQueryBuilder timeOrder(String t) + { + timeOrder = t; + return this; + } + public ScanQueryBuilder columns(List c) { columns = c; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 272bddbeaecc..b11e67a6f63d 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -40,6 +40,15 @@ public ScanQueryConfig setLegacy(final boolean legacy) return this; } + // int should suffice here because no one should be sorting greater than 2B rows in memory + @JsonProperty + private int maxRowsTimeOrderedInMemory = 100000; + + public int getMaxRowsTimeOrderedInMemory() + { + return maxRowsTimeOrderedInMemory; + } + @Override public boolean equals(final Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index ba98ad956cf3..069b2dfe8d5b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -131,7 +131,8 @@ public Sequence process( intervals.get(0), query.getVirtualColumns(), Granularities.ALL, - query.isDescending(), + query.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING) || + (query.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) && query.isDescending()), null ) .map(cursor -> new BaseSequence<>( diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index b031a84a82c5..8330da59fa8b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -22,10 +22,13 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.primitives.Longs; import com.google.inject.Inject; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.BaseSequence; 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.parsers.CloseableIterator; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; @@ -33,8 +36,16 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.segment.column.ColumnHolder; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.PriorityQueue; public class ScanQueryQueryToolChest extends QueryToolChest { @@ -44,6 +55,7 @@ public class ScanQueryQueryToolChest extends QueryToolChest run( // the same way, even if they have different default legacy values. final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - - if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(queryPlusWithNonNullLegacy, responseContext); - } - return new BaseSequence<>( + BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = new BaseSequence.IteratorMaker() { @Override @@ -87,8 +96,77 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) { CloseQuietly.close(iterFromMake); } + }; + + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) || + scanQuery.getLimit() > maxRowsForInMemoryTimeOrdering) { + if (scanQuery.getLimit() == Long.MAX_VALUE) { + return runner.run(queryPlusWithNonNullLegacy, responseContext); + } + return new BaseSequence<>(scanQueryLimitRowIteratorMaker); + } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || + scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { + Comparator priorityQComparator = (val1, val2) -> { + int comparison; + ScanResultValue val1SRV = (ScanResultValue) val1, + val2SRV = (ScanResultValue) val2; + if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { + comparison = Longs.compare( + (Long) ((Map) ((List) val1SRV.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), + (Long) ((Map) ((List) val2SRV.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) + ); + } else if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { + int val1TimeColumnIndex = val1SRV.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); + int val2TimeColumnIndex = val2SRV.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); + List event1 = (List) ((List) val1SRV.getEvents()).get(0); + List event2 = (List) ((List) val2SRV.getEvents()).get(0); + comparison = Longs.compare( + (Long) event1.get(val1TimeColumnIndex), + (Long) event2.get(val2TimeColumnIndex) + ); + } else { + throw new UOE("Result format [%s] is not supported", scanQuery.getResultFormat()); + } + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { + return comparison * -1; + } + return comparison; + }; + + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + while (scanResultIterator.hasNext()) { + ScanResultValue next = scanResultIterator.next(); + List events = (List) next.getEvents(); + for (Object event : events) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); } - ); + } + + Iterator queueIterator = q.iterator(); + + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public ScanBatchedTimeOrderedQueueIterator make() + { + return new ScanBatchedTimeOrderedQueueIterator(queueIterator, scanQuery.getBatchSize()); + } + + @Override + public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); + } else { + throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); + } } }; } @@ -131,4 +209,42 @@ public Sequence run(QueryPlus queryPlus, Map + { + private final Iterator itr; + private final int batchSize; + + public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) + { + itr = iterator; + this.batchSize = batchSize; + } + + @Override + public void close() throws IOException + { + } + + @Override + public boolean hasNext() + { + return itr.hasNext(); + } + + @Override + public ScanResultValue next() + { + // Create new scanresultvalue from event map + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && itr.hasNext()) { + ScanResultValue srv = itr.next(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(((List) srv.getEvents()).get(0)); + } + return new ScanResultValue(null, columns, eventsToAdd); + } + } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index faeea4f83808..c8056989e298 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -26,6 +26,7 @@ import com.google.common.collect.ObjectArrays; import com.google.common.collect.Sets; import com.google.common.hash.Hashing; +import org.apache.commons.lang.ArrayUtils; import org.apache.druid.hll.HyperLogLogCollector; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; @@ -64,6 +65,7 @@ import java.util.Set; /** + * */ @RunWith(Parameterized.class) public class ScanQueryRunnerTest @@ -234,7 +236,11 @@ public void testSelectWithUnderscoreUnderscoreTime() { ScanQuery query = newTestQuery() .intervals(I_0112_0114) - .columns(ColumnHolder.TIME_COLUMN_NAME, QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric) + .columns( + ColumnHolder.TIME_COLUMN_NAME, + QueryRunnerTestHelper.marketDimension, + QueryRunnerTestHelper.indexMetric + ) .build(); HashMap context = new HashMap(); @@ -508,6 +514,249 @@ public void testFullSelectNoDimensionAndMetric() verify(expectedResults, results); } + @Test + public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() + { + // limits + for (int limit : new int[]{3, 1, 5, 7, 0}) { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .limit(limit) + .timeOrder("ascending") + .build(); + + HashMap context = new HashMap<>(); + Iterable results = runner.run(QueryPlus.wrap(query), context).toList(); + String[] seg1Results = new String[]{ + "2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t100.000000" + }; + String[] seg2Results = new String[]{ + "2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t94.874713", + "2011-01-13T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t103.629399", + "2011-01-13T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t110.087299", + "2011-01-13T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t114.947403", + "2011-01-13T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t104.465767", + "2011-01-13T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t102.851683", + "2011-01-13T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t108.863011", + "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", + "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" + }; + final List>> ascendingEvents = toEvents( + new String[]{ + legacy ? getTimestampName() + ":TIME" : null, + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":DOUBLE" + }, + (String[]) ArrayUtils.addAll(seg1Results, seg2Results) + ); + List ascendingExpectedResults = toExpected( + ascendingEvents, + legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), + 0, + limit + ); + verify(ascendingExpectedResults, results); + } + } + + @Test + public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() + { + // limits + for (int limit : new int[]{3, 1, 5, 7, 0}) { + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .limit(limit) + .timeOrder("descending") + .build(); + + HashMap context = new HashMap<>(); + Iterable results = runner.run(QueryPlus.wrap(query), context).toList(); + String[] seg1Results = new String[]{ + "2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t100.000000" + }; + String[] seg2Results = new String[]{ + "2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t94.874713", + "2011-01-13T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t103.629399", + "2011-01-13T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t110.087299", + "2011-01-13T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t114.947403", + "2011-01-13T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t104.465767", + "2011-01-13T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t102.851683", + "2011-01-13T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t108.863011", + "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", + "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" + }; + String[] expectedRet = (String[]) ArrayUtils.addAll(seg1Results, seg2Results); + ArrayUtils.reverse(expectedRet); + final List>> descendingEvents = toEvents( + new String[]{ + legacy ? getTimestampName() + ":TIME" : null, + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":DOUBLE" + }, + expectedRet + ); + List descendingExpectedResults = toExpected( + descendingEvents, + legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), + 0, + limit + ); + verify(descendingExpectedResults, results); + } + } + + @Test + public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedListFormat() + { + String[] seg1Results = new String[]{ + "2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t100.000000" + }; + String[] seg2Results = new String[]{ + "2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t94.874713", + "2011-01-13T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t103.629399", + "2011-01-13T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t110.087299", + "2011-01-13T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t114.947403", + "2011-01-13T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t104.465767", + "2011-01-13T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t102.851683", + "2011-01-13T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t108.863011", + "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", + "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" + }; + // limits + for (int limit : new int[]{3, 1, 5, 7, 0}) { + /* Ascending */ + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .resultFormat("compactedList") + .timeOrder("ascending") + .limit(limit) + .build(); + + HashMap context = new HashMap<>(); + Iterable results = runner.run(QueryPlus.wrap(query), context).toList(); + final List>> ascendingEvents = toEvents( + new String[]{ + legacy ? getTimestampName() + ":TIME" : null, + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":DOUBLE" + }, + (String[]) ArrayUtils.addAll(seg1Results, seg2Results) + ); + List ascendingExpectedResults = toExpected( + ascendingEvents, + legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), + 0, + limit + ); + results = compactedListToRow(results); + verify(ascendingExpectedResults, results); + } + } + + @Test + public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedListFormat() + { + String[] seg1Results = new String[]{ + "2011-01-12T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t100.000000", + "2011-01-12T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t100.000000" + }; + String[] seg2Results = new String[]{ + "2011-01-13T00:00:00.000Z\tspot\tautomotive\tpreferred\tapreferred\t94.874713", + "2011-01-13T00:00:00.000Z\tspot\tbusiness\tpreferred\tbpreferred\t103.629399", + "2011-01-13T00:00:00.000Z\tspot\tentertainment\tpreferred\tepreferred\t110.087299", + "2011-01-13T00:00:00.000Z\tspot\thealth\tpreferred\thpreferred\t114.947403", + "2011-01-13T00:00:00.000Z\tspot\tmezzanine\tpreferred\tmpreferred\t104.465767", + "2011-01-13T00:00:00.000Z\tspot\tnews\tpreferred\tnpreferred\t102.851683", + "2011-01-13T00:00:00.000Z\tspot\tpremium\tpreferred\tppreferred\t108.863011", + "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", + "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" + }; + // limits + for (int limit : new int[]{3, 1, 5, 7, 0}) { + /* Descending */ + ScanQuery query = newTestQuery() + .intervals(I_0112_0114) + .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) + .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) + .resultFormat("compactedList") + .timeOrder("descending") + .limit(limit) + .build(); + + HashMap context = new HashMap<>(); + Iterable results = runner.run(QueryPlus.wrap(query), context).toList(); + String[] expectedRet = (String[]) ArrayUtils.addAll(seg1Results, seg2Results); + ArrayUtils.reverse(expectedRet); + final List>> descendingEvents = toEvents( + new String[]{ + legacy ? getTimestampName() + ":TIME" : null, + null, + QueryRunnerTestHelper.qualityDimension + ":STRING", + null, + null, + QueryRunnerTestHelper.indexMetric + ":DOUBLE" + }, + expectedRet //segments in reverse order from above + ); + List descendingExpectedResults = toExpected( + descendingEvents, + legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), + 0, + limit + ); + results = compactedListToRow(results); + verify(descendingExpectedResults, results); + } + } + + private List>> toFullEvents(final String[]... valueSet) { return toEvents( @@ -686,7 +935,12 @@ public static void verify( Object exValue = ex.getValue(); if (exValue instanceof Double || exValue instanceof Float) { final double expectedDoubleValue = ((Number) exValue).doubleValue(); - Assert.assertEquals("invalid value for " + ex.getKey(), expectedDoubleValue, ((Number) actVal).doubleValue(), expectedDoubleValue * 1e-6); + Assert.assertEquals( + "invalid value for " + ex.getKey(), + expectedDoubleValue, + ((Number) actVal).doubleValue(), + expectedDoubleValue * 1e-6 + ); } else { Assert.assertEquals("invalid value for " + ex.getKey(), ex.getValue(), actVal); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index ec5069546a4d..e1e322649fbe 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -53,6 +53,7 @@ public void testSerializationLegacyString() throws Exception + "\"resultFormat\":\"list\"," + "\"batchSize\":20480," + "\"limit\":3," + + "\"timeOrder\":\"none\"," + "\"filter\":null," + "\"columns\":[\"market\",\"quality\",\"index\"]," + "\"legacy\":null," @@ -70,7 +71,8 @@ public void testSerializationLegacyString() throws Exception null, Arrays.asList("market", "quality", "index"), null, - null + null, + "none" ); String actual = jsonMapper.writeValueAsString(query); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java new file mode 100644 index 000000000000..572f1d8615c4 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java @@ -0,0 +1,71 @@ +/* + * 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.query.scan; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ScanResultValueSerdeTest +{ + private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testSerdeScanResultValueCompactedList() throws IOException + { + String segmentId = "some_segment_id"; + List columns = new ArrayList<>(Arrays.asList("col1", "col2", "col3")); + List event = new ArrayList<>(Arrays.asList( + "prop1", + "prop2", + "prop3" + )); + List> events = new ArrayList<>(Collections.singletonList(event)); + ScanResultValue srv = new ScanResultValue(segmentId, columns, events); + String serialized = jsonMapper.writeValueAsString(srv); + ScanResultValue deserialized = jsonMapper.readValue(serialized, ScanResultValue.class); + Assert.assertEquals(srv, deserialized); + } + + @Test + public void testSerdeScanResultValueNonCompactedList() throws IOException + { + String segmentId = "some_segment_id"; + List columns = new ArrayList<>(Arrays.asList("col1", "col2", "col3")); + Map event = new HashMap<>(); + event.put("key1", new Integer(4)); + event.put("key2", "some_string"); + event.put("key3", new Double(4.1)); + List> events = new ArrayList<>(Collections.singletonList(event)); + ScanResultValue srv = new ScanResultValue(segmentId, columns, events); + String serialized = jsonMapper.writeValueAsString(srv); + ScanResultValue deserialized = jsonMapper.readValue(serialized, ScanResultValue.class); + Assert.assertEquals(srv, deserialized); + } +} diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 253aa0f68062..eaa08d656c5a 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -199,7 +199,7 @@ public Sequence run(final QueryPlus queryPlus, final Map r } /** - * This class essentially incapsulates the major part of the logic of {@link CachingClusteredClient}. It's state and + * This class essentially encapsulates the major part of the logic of {@link CachingClusteredClient}. It's state and * methods couldn't belong to {@link CachingClusteredClient} itself, because they depend on the specific query object * being run, but {@link QuerySegmentWalker} API is designed so that implementations should be able to accept * arbitrary queries. diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 647e213dd194..5bdc37c3d667 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -967,7 +967,8 @@ public ScanQuery toScanQuery() filtration.getDimFilter(), Ordering.natural().sortedCopy(ImmutableSet.copyOf(outputRowSignature.getRowOrder())), false, - ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()), + null // Will default to "none" ); } From 79e8319383eddbb49ecb4c1785dcd3eed14a0634 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Feb 2019 18:22:58 -0800 Subject: [PATCH 05/91] Move ScanResultValue timestamp comparator to a separate class for testing --- .../query/scan/ScanQueryQueryToolChest.java | 28 +-------- .../ScanResultValueTimestampComparator.java | 59 +++++++++++++++++++ 2 files changed, 61 insertions(+), 26 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 8330da59fa8b..c8504d3648af 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -106,37 +106,13 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) return new BaseSequence<>(scanQueryLimitRowIteratorMaker); } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { - Comparator priorityQComparator = (val1, val2) -> { - int comparison; - ScanResultValue val1SRV = (ScanResultValue) val1, - val2SRV = (ScanResultValue) val2; - if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { - comparison = Longs.compare( - (Long) ((Map) ((List) val1SRV.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), - (Long) ((Map) ((List) val2SRV.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) - ); - } else if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { - int val1TimeColumnIndex = val1SRV.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); - int val2TimeColumnIndex = val2SRV.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); - List event1 = (List) ((List) val1SRV.getEvents()).get(0); - List event2 = (List) ((List) val2SRV.getEvents()).get(0); - comparison = Longs.compare( - (Long) event1.get(val1TimeColumnIndex), - (Long) event2.get(val2TimeColumnIndex) - ); - } else { - throw new UOE("Result format [%s] is not supported", scanQuery.getResultFormat()); - } - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { - return comparison * -1; - } - return comparison; - }; + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + while (scanResultIterator.hasNext()) { ScanResultValue next = scanResultIterator.next(); List events = (List) next.getEvents(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java new file mode 100644 index 000000000000..53f7949bb351 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -0,0 +1,59 @@ +package org.apache.druid.query.scan; + +import com.google.common.primitives.Longs; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.segment.column.ColumnHolder; + +import java.util.Comparator; +import java.util.List; +import java.util.Map; + +public class ScanResultValueTimestampComparator implements Comparator +{ + private ScanQuery scanQuery; + + public ScanResultValueTimestampComparator(ScanQuery scanQuery) { + this.scanQuery = scanQuery; + } + + @Override + public int compare(ScanResultValue o1, ScanResultValue o2) + { + int comparison; + if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { + comparison = Longs.compare( + (Long) ((Map) ((List) o1.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), + (Long) ((Map) ((List) o2.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) + ); + } else if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { + int val1TimeColumnIndex = o1.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); + int val2TimeColumnIndex = o2.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); + List event1 = (List) ((List) o1.getEvents()).get(0); + List event2 = (List) ((List) o2.getEvents()).get(0); + comparison = Longs.compare( + (Long) event1.get(val1TimeColumnIndex), + (Long) event2.get(val2TimeColumnIndex) + ); + } else { + throw new UOE("Result format [%s] is not supported", scanQuery.getResultFormat()); + } + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { + return comparison * -1; + } + return comparison; + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + ScanResultValueTimestampComparator comp = (ScanResultValueTimestampComparator) obj + return this.scanQuery.equals(comp.scanQuery); + } + +} From 7b584713946b538d15da591e306ca4c0a7a378e3 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Sat, 2 Feb 2019 03:48:18 -0800 Subject: [PATCH 06/91] Licensing stuff --- .../query/scan/ScanQueryQueryToolChest.java | 2 +- .../ScanResultValueTimestampComparator.java | 23 ++++++++++-- ...canResultValueTimestampComparatorTest.java | 35 +++++++++++++++++++ 3 files changed, 57 insertions(+), 3 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index c8504d3648af..dbd14da40f4c 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -112,7 +112,7 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); - + while (scanResultIterator.hasNext()) { ScanResultValue next = scanResultIterator.next(); List events = (List) next.getEvents(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 53f7949bb351..05c2b90a3afd 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -1,3 +1,22 @@ +/* + * 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.query.scan; import com.google.common.primitives.Longs; @@ -10,7 +29,7 @@ public class ScanResultValueTimestampComparator implements Comparator { - private ScanQuery scanQuery; + private final ScanQuery scanQuery; public ScanResultValueTimestampComparator(ScanQuery scanQuery) { this.scanQuery = scanQuery; @@ -52,7 +71,7 @@ public boolean equals(Object obj) if (obj == null || getClass() != obj.getClass()) { return false; } - ScanResultValueTimestampComparator comp = (ScanResultValueTimestampComparator) obj + ScanResultValueTimestampComparator comp = (ScanResultValueTimestampComparator) obj; return this.scanQuery.equals(comp.scanQuery); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java new file mode 100644 index 000000000000..cb9a6f106333 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -0,0 +1,35 @@ +/* + * 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.query.scan; + +import org.junit.Test; + +public class ScanResultValueTimestampComparatorTest +{ + @Test + public void comparisonTest() { + + } + + @Test + public void priorityQueueTest() { + + } +} From ad731a362b465e9b4ca0c9ad7050fc6555606d52 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 10:55:56 -0800 Subject: [PATCH 07/91] Change benchmark --- .../druid/benchmark/query/ScanBenchmark.java | 12 ++++++++---- .../main/java/org/apache/druid/query/Druids.java | 14 ++++++++++++-- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 4c66e8373565..e72bc5496a08 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -177,7 +177,8 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .dataSource("blah") - .intervals(intervalSpec); + .intervals(intervalSpec) + .timeOrder("none"); } @@ -203,7 +204,8 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(filter) - .intervals(intervalSpec); + .intervals(intervalSpec) + .timeOrder("none"); } private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) @@ -220,7 +222,8 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) - .intervals(intervalSpec); // TODO + .intervals(intervalSpec) + .timeOrder("none"); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -241,7 +244,8 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) - .intervals(intervalSpec); // TODO + .intervals(intervalSpec) + .timeOrder("none"); } @Setup diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 6ef7f93afb61..dd8b35c3f873 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -924,6 +924,7 @@ public static class ScanQueryBuilder private DimFilter dimFilter; private List columns; private Boolean legacy; + private String timeOrder; public ScanQueryBuilder() { @@ -937,6 +938,7 @@ public ScanQueryBuilder() dimFilter = null; columns = new ArrayList<>(); legacy = null; + timeOrder = null; } public ScanQuery build() @@ -951,7 +953,8 @@ public ScanQuery build() dimFilter, columns, legacy, - context + context, + timeOrder ); } @@ -967,7 +970,8 @@ public static ScanQueryBuilder copy(ScanQuery query) .filters(query.getFilter()) .columns(query.getColumns()) .legacy(query.isLegacy()) - .context(query.getContext()); + .context(query.getContext()) + .timeOrder(query.getTimeOrder()); } public ScanQueryBuilder dataSource(String ds) @@ -1046,6 +1050,12 @@ public ScanQueryBuilder legacy(Boolean legacy) this.legacy = legacy; return this; } + + public ScanQueryBuilder timeOrder(String timeOrder) + { + this.timeOrder = timeOrder; + return this; + } } public static ScanQueryBuilder newScanQueryBuilder() From e66339cd76cdb7f08a291e8488e3415518f3df63 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 10:56:41 -0800 Subject: [PATCH 08/91] Remove todos --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 4c66e8373565..bfa7c2898598 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -220,7 +220,7 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) - .intervals(intervalSpec); // TODO + .intervals(intervalSpec); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -241,7 +241,7 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) - .intervals(intervalSpec); // TODO + .intervals(intervalSpec); } @Setup From 12e51a272124c7a75628fe5b2f65ddc00e34ba27 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 12:02:13 -0800 Subject: [PATCH 09/91] Added TimestampComparator tests --- .../druid/benchmark/query/ScanBenchmark.java | 4 +- .../query/scan/ScanQueryQueryToolChest.java | 2 - .../ScanResultValueTimestampComparator.java | 3 +- ...canResultValueTimestampComparatorTest.java | 152 +++++++++++++++++- 4 files changed, 153 insertions(+), 8 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index e72bc5496a08..18116575809a 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -94,8 +94,8 @@ /* Works with 4GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 5) -@Measurement(iterations = 5) +@Warmup(iterations = 10) +@Measurement(iterations = 25) public class ScanBenchmark { @Param({"1", "4"}) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index dbd14da40f4c..b20afef0970d 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.primitives.Longs; import com.google.inject.Inject; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -36,7 +35,6 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.aggregation.MetricManipulationFn; -import org.apache.druid.segment.column.ColumnHolder; import java.io.IOException; import java.util.ArrayList; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 05c2b90a3afd..163c6a104496 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -31,7 +31,8 @@ public class ScanResultValueTimestampComparator implements Comparator> events1 = new ArrayList<>(); + HashMap event1 = new HashMap<>(); + event1.put(ColumnHolder.TIME_COLUMN_NAME, new Long(42)); + events1.add(event1); + + ScanResultValue s1 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events1); + + ArrayList> events2 = new ArrayList<>(); + HashMap event2 = new HashMap<>(); + event2.put(ColumnHolder.TIME_COLUMN_NAME, new Long(43)); + events2.add(event2); + + ScanResultValue s2 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events2); + + Assert.assertEquals(1, comparator.compare(s1, s2)); + } + + @Test + public void comparisonAscendingListTest() + { + ScanQuery query = Druids.newScanQueryBuilder() + .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .resultFormat(ScanQuery.RESULT_FORMAT_LIST) + .dataSource("some src") + .intervals(intervalSpec) + .build(); + + ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); + + ArrayList> events1 = new ArrayList<>(); + HashMap event1 = new HashMap<>(); + event1.put(ColumnHolder.TIME_COLUMN_NAME, new Long(42)); + events1.add(event1); + + ScanResultValue s1 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events1); + + ArrayList> events2 = new ArrayList<>(); + HashMap event2 = new HashMap<>(); + event2.put(ColumnHolder.TIME_COLUMN_NAME, new Long(43)); + events2.add(event2); + + ScanResultValue s2 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events2); + + Assert.assertEquals(-1, comparator.compare(s1, s2)); + } + @Test - public void comparisonTest() { + public void comparisonDescendingCompactedListTest() + { + ScanQuery query = Druids.newScanQueryBuilder() + .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .dataSource("some src") + .intervals(intervalSpec) + .build(); + + ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); + + List> events1 = new ArrayList<>(); + List event1 = Collections.singletonList(new Long(42)); + events1.add(event1); + + ScanResultValue s1 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events1); + List> events2 = new ArrayList<>(); + List event2 = Collections.singletonList(new Long(43)); + events2.add(event2); + + ScanResultValue s2 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events2); + + Assert.assertEquals(1, comparator.compare(s1, s2)); } @Test - public void priorityQueueTest() { - + public void comparisonAscendingCompactedListTest() + { + ScanQuery query = Druids.newScanQueryBuilder() + .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .dataSource("some src") + .intervals(intervalSpec) + .build(); + + ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); + + List> events1 = new ArrayList<>(); + List event1 = Collections.singletonList(new Long(42)); + events1.add(event1); + + ScanResultValue s1 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events1); + + List> events2 = new ArrayList<>(); + List event2 = Collections.singletonList(new Long(43)); + events2.add(event2); + + ScanResultValue s2 = new ScanResultValue( + "segmentId", + Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), + events2); + + Assert.assertEquals(-1, comparator.compare(s1, s2)); } } From 432acaf08575c451ea02e8ec8d6318678dcf20cb Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 12:03:14 -0800 Subject: [PATCH 10/91] Change number of benchmark iterations --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index bfa7c2898598..1c4d61d3757d 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -94,8 +94,8 @@ /* Works with 4GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 5) -@Measurement(iterations = 5) +@Warmup(iterations = 10) +@Measurement(iterations = 25) public class ScanBenchmark { @Param({"1", "4"}) From 01b25ed11293f472dac78d4f793f2941c3b22a18 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 14:36:18 -0800 Subject: [PATCH 11/91] Added time ordering to the scan benchmark --- .../druid/benchmark/query/ScanBenchmark.java | 17 +-- .../query/scan/ScanQueryQueryToolChest.java | 119 +++++++++--------- 2 files changed, 66 insertions(+), 70 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 18116575809a..95ba66bb6681 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -91,7 +91,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -/* Works with 4GB heap size or greater. Otherwise there's a good chance of an OOME. */ +/* Works with 8GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 10) @@ -104,15 +104,18 @@ public class ScanBenchmark @Param({"1", "2"}) private int numProcessingThreads; - @Param({"250000"}) + @Param({"750000"}) private int rowsPerSegment; @Param({"basic.A"}) private String schemaAndQuery; - @Param({"1000"}) + @Param({"1000", "99999"}) private int limit; + @Param({"none", "descending", "ascending"}) + private static String timeOrdering; + private static final Logger log = new Logger(ScanBenchmark.class); private static final ObjectMapper JSON_MAPPER; private static final IndexMergerV9 INDEX_MERGER_V9; @@ -178,7 +181,7 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .dataSource("blah") .intervals(intervalSpec) - .timeOrder("none"); + .timeOrder(timeOrdering); } @@ -205,7 +208,7 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(filter) .intervals(intervalSpec) - .timeOrder("none"); + .timeOrder(timeOrdering); } private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) @@ -223,7 +226,7 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) .intervals(intervalSpec) - .timeOrder("none"); + .timeOrder(timeOrdering); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -245,7 +248,7 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) .intervals(intervalSpec) - .timeOrder("none"); + .timeOrder(timeOrdering); } @Setup diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index b20afef0970d..f2fab980b4c2 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -69,78 +69,71 @@ public ScanQueryQueryToolChest( @Override public QueryRunner mergeResults(final QueryRunner runner) { - return new QueryRunner() - { - @Override - public Sequence run( - final QueryPlus queryPlus, final Map responseContext - ) - { - // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it - // the same way, even if they have different default legacy values. - final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); - final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = - new BaseSequence.IteratorMaker() + return (queryPlus, responseContext) -> { + // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it + // the same way, even if they have different default legacy values. + final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); + final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); + BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() + { + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } + + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }; + + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) || + scanQuery.getLimit() > maxRowsForInMemoryTimeOrdering) { + if (scanQuery.getLimit() == Long.MAX_VALUE) { + return runner.run(queryPlusWithNonNullLegacy, responseContext); + } + return new BaseSequence<>(scanQueryLimitRowIteratorMaker); + } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || + scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); + + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + + while (scanResultIterator.hasNext()) { + ScanResultValue next = scanResultIterator.next(); + List events = (List) next.getEvents(); + for (Object event : events) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); + } + } + + Iterator queueIterator = q.iterator(); + + return new BaseSequence( + new BaseSequence.IteratorMaker() { @Override - public ScanQueryLimitRowIterator make() + public ScanBatchedTimeOrderedQueueIterator make() { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + return new ScanBatchedTimeOrderedQueueIterator(queueIterator, scanQuery.getBatchSize()); } @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) + public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) { CloseQuietly.close(iterFromMake); } - }; - - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) || - scanQuery.getLimit() > maxRowsForInMemoryTimeOrdering) { - if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(queryPlusWithNonNullLegacy, responseContext); - } - return new BaseSequence<>(scanQueryLimitRowIteratorMaker); - } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || - scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { - Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); - - // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch - // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); - Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); - - while (scanResultIterator.hasNext()) { - ScanResultValue next = scanResultIterator.next(); - List events = (List) next.getEvents(); - for (Object event : events) { - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); - } - } - - Iterator queueIterator = q.iterator(); - - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public ScanBatchedTimeOrderedQueueIterator make() - { - return new ScanBatchedTimeOrderedQueueIterator(queueIterator, scanQuery.getBatchSize()); - } - - @Override - public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); - } else { - throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); - } + }); + } else { + throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); } }; } From 9e6e71616bdcd9a7eea56e4bc1ef869c08bcf83c Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 15:31:21 -0800 Subject: [PATCH 12/91] Changed benchmark params --- .../org/apache/druid/benchmark/query/ScanBenchmark.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 1c4d61d3757d..b9f90bbbd9ac 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -91,7 +91,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; -/* Works with 4GB heap size or greater. Otherwise there's a good chance of an OOME. */ +/* Works with 8GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) @Warmup(iterations = 10) @@ -104,13 +104,13 @@ public class ScanBenchmark @Param({"1", "2"}) private int numProcessingThreads; - @Param({"250000"}) + @Param({"750000"}) private int rowsPerSegment; @Param({"basic.A"}) private String schemaAndQuery; - @Param({"1000"}) + @Param({"1000", "99999"}) private int limit; private static final Logger log = new Logger(ScanBenchmark.class); From 20c36644dbbf46df1a9209a635e661c01aeec627 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 15:36:35 -0800 Subject: [PATCH 13/91] More param changes --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index b9f90bbbd9ac..16dc90e218ae 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -98,10 +98,10 @@ @Measurement(iterations = 25) public class ScanBenchmark { - @Param({"1", "4"}) + @Param({"2", "4"}) private int numSegments; - @Param({"1", "2"}) + @Param({"2"}) private int numProcessingThreads; @Param({"750000"}) From 796083f2bb188421f68858111bb39c988cb2f71c Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Feb 2019 15:37:42 -0800 Subject: [PATCH 14/91] Benchmark param change --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 95ba66bb6681..bb492561835a 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -98,10 +98,10 @@ @Measurement(iterations = 25) public class ScanBenchmark { - @Param({"1", "4"}) + @Param({"2", "4"}) private int numSegments; - @Param({"1", "2"}) + @Param({"2"}) private int numProcessingThreads; @Param({"750000"}) From 737a83321d74cd0b1f7b4ca800509c36056d08ff Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 5 Feb 2019 10:15:32 -0800 Subject: [PATCH 15/91] Made Jon's changes and removed TODOs --- .../druid/benchmark/query/ScanBenchmark.java | 29 ++----------------- 1 file changed, 3 insertions(+), 26 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 16dc90e218ae..410bb7b0bcd6 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -178,24 +178,16 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .dataSource("blah") .intervals(intervalSpec); - } private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); - final List dimUniformFilterVals = new ArrayList<>(); - int resultNum = (int) (100000 * 0.1); - int step = 100000 / resultNum; - for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { - dimUniformFilterVals.add(String.valueOf(i)); - } - List dimHyperUniqueFilterVals = new ArrayList<>(); - resultNum = (int) (100000 * 0.1); - step = 100000 / resultNum; - for (int i = 0; i < 100001 && dimHyperUniqueFilterVals.size() < resultNum; i += step) { + int numResults = (int) (100000 * 0.1); + int step = 100000 / numResults; + for (int i = 0; i < 100001 && dimHyperUniqueFilterVals.size() < numResults; i += step) { dimHyperUniqueFilterVals.add(String.valueOf(i)); } @@ -210,13 +202,6 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); - final List dimUniformFilterVals = new ArrayList<>(); - final int resultNum = (int) (100000 * 0.1); - final int step = 100000 / resultNum; - for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { - dimUniformFilterVals.add(String.valueOf(i)); - } - final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) @@ -229,16 +214,8 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch Collections.singletonList(basicSchema.getDataInterval()) ); - final List dimUniformFilterVals = new ArrayList<>(); - final int resultNum = (int) (100000 * 0.1); - final int step = 100000 / resultNum; - for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) { - dimUniformFilterVals.add(String.valueOf(i)); - } - final String dimName = "dimUniform"; - return Druids.newScanQueryBuilder() .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) .intervals(intervalSpec); From 86c5eee13b6ce18b33c723cd0c4e464eaf41f010 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 5 Feb 2019 10:31:27 -0800 Subject: [PATCH 16/91] Broke some long lines into two lines --- .../org/apache/druid/benchmark/query/ScanBenchmark.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 410bb7b0bcd6..c68f961981e9 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -173,7 +173,8 @@ private static Druids.ScanQueryBuilder makeQuery(final String name, final Benchm /* Just get everything */ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = + new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); return Druids.newScanQueryBuilder() .dataSource("blah") @@ -182,7 +183,8 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = + new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List dimHyperUniqueFilterVals = new ArrayList<>(); int numResults = (int) (100000 * 0.1); @@ -200,7 +202,8 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) { - final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); + final QuerySegmentSpec intervalSpec = + new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() From d1a1793f36d4c9c910f84318f2bbbd355533c977 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 5 Feb 2019 12:40:26 -0800 Subject: [PATCH 17/91] nit --- .../org/apache/druid/query/scan/ScanQueryQueryToolChest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 65513c950126..04d66f900cb9 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -74,7 +74,7 @@ public QueryRunner mergeResults(final QueryRunner queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = + final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = new BaseSequence.IteratorMaker() { @Override From b6d4df3864e3910fa406dcc83f6644f45f496c5f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 5 Feb 2019 16:45:20 -0800 Subject: [PATCH 18/91] Decrease segment size for less memory usage --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index c68f961981e9..3ac7bb2f2c79 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -104,7 +104,7 @@ public class ScanBenchmark @Param({"2"}) private int numProcessingThreads; - @Param({"750000"}) + @Param({"250000"}) private int rowsPerSegment; @Param({"basic.A"}) From 8b7d5f50818b00730965a55b1bf8ed27860bd6a4 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 11:55:09 -0800 Subject: [PATCH 19/91] Wrote tests for heapsort scan result values and fixed bug where iterator wasn't returning elements in correct order --- .../query/scan/ScanQueryQueryToolChest.java | 52 ++++-- .../scan/ScanQueryQueryToolChestTest.java | 159 ++++++++++++++++++ 2 files changed, 193 insertions(+), 18 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 04d66f900cb9..337b016e14f6 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -20,6 +20,7 @@ package org.apache.druid.query.scan; import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; import com.google.inject.Inject; @@ -98,32 +99,18 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) return new BaseSequence(scanQueryLimitRowIteratorMaker); } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { - Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); - - // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch - // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - PriorityQueue q = new PriorityQueue(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); - while (scanResultIterator.hasNext()) { - ScanResultValue next = scanResultIterator.next(); - List events = (List) next.getEvents(); - for (Object event : events) { - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); - } - } - - Iterator queueIterator = q.iterator(); - return new BaseSequence( new BaseSequence.IteratorMaker() { @Override public ScanBatchedTimeOrderedQueueIterator make() { - return new ScanBatchedTimeOrderedQueueIterator(queueIterator, scanQuery.getBatchSize()); + return new ScanBatchedTimeOrderedQueueIterator( + heapsortScanResultValues(scanResultIterator, scanQuery), + scanQuery.getBatchSize() + ); } @Override @@ -177,6 +164,35 @@ public Sequence run(QueryPlus queryPlus, Map heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) { + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); + + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + + PriorityQueue q = new PriorityQueue<> + (Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + + while (inputIterator.hasNext()) { + + ScanResultValue next = inputIterator.next(); + List events = (List) next.getEvents(); + for (Object event : events) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); + } + } + // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained + List sortedElements = new ArrayList<>(q.size()); + while (q.size() != 0 ) { + sortedElements.add(q.poll()); + } + return sortedElements.iterator(); + } + private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator { private final Iterator itr; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java new file mode 100644 index 000000000000..89d846c54aa6 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -0,0 +1,159 @@ +/* + * 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.query.scan; + +import org.apache.druid.query.Druids; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.segment.column.ColumnHolder; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.easymock.EasyMock.createNiceMock; +import static org.easymock.EasyMock.expect; +import static org.easymock.EasyMock.replay; + +public class ScanQueryQueryToolChestTest +{ + private static ScanQueryQueryToolChest chest; + private static ScanQueryConfig config; + + @Before + public void setup() + { + config = createNiceMock(ScanQueryConfig.class); + expect(config.getMaxRowsTimeOrderedInMemory()).andReturn(100000); + replay(config); + chest = new ScanQueryQueryToolChest(config, null); + } + + @Test + public void testDescendingHeapsortListScanResultValues() + { + List inputs = new ArrayList<>(); + for (long i = 0; i < 1000; i++) { + HashMap event = new HashMap<>(); + event.put("__time", i * 1000); + inputs.add( + new ScanResultValue( + "some segment id", + Collections.singletonList("__time"), + Collections.singletonList(event) + ) + ); + } + ScanQuery scanQuery = new Druids.ScanQueryBuilder() + .resultFormat("list") + .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .dataSource("some data source") + .intervals(new QuerySegmentSpec() + { + @Override + public List getIntervals() + { + return null; + } + + @Override + public QueryRunner lookup( + Query query, QuerySegmentWalker walker + ) + { + return null; + } + }) + .limit(99999) + .build(); + Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + + Long previousTime = Long.MAX_VALUE; + while (sorted.hasNext()) { + ScanResultValue curr = sorted.next(); + Long currentTime = (Long) + ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); + Assert.assertTrue("Event timestamp is less than that of the previous event", + currentTime < previousTime); + previousTime = currentTime; + } + } + + @Test + public void testAscendingHeapsortListScanResultValues() + { + List inputs = new ArrayList<>(); + for (long i = 1000; i > 0; i--) { + HashMap event = new HashMap<>(); + event.put("__time", i * 1000); + inputs.add( + new ScanResultValue( + "some segment id", + Collections.singletonList("__time"), + Collections.singletonList(event) + ) + ); + } + ScanQuery scanQuery = new Druids.ScanQueryBuilder() + .resultFormat("list") + .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .dataSource("some data source") + .intervals(new QuerySegmentSpec() + { + @Override + public List getIntervals() + { + return null; + } + + @Override + public QueryRunner lookup( + Query query, QuerySegmentWalker walker + ) + { + return null; + } + }) + .limit(99999) + .build(); + Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + + Long previousTime = -1L; + while (sorted.hasNext()) { + ScanResultValue curr = sorted.next(); + Long currentTime = (Long) + ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); + Assert.assertTrue( + "Event timestamp is greater than that of the previous event", + currentTime > previousTime + ); + previousTime = currentTime; + } + } +} From 4f51024b318bf744eddb9d2f9638f7590872cf14 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 12:08:12 -0800 Subject: [PATCH 20/91] Wrote more tests for scan result value sort --- .../scan/ScanQueryQueryToolChestTest.java | 119 ++++++++++++++++-- 1 file changed, 112 insertions(+), 7 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 89d846c54aa6..6a556ea5d2cc 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -45,6 +45,7 @@ public class ScanQueryQueryToolChestTest { private static ScanQueryQueryToolChest chest; private static ScanQueryConfig config; + private static int numElements; @Before public void setup() @@ -53,13 +54,14 @@ public void setup() expect(config.getMaxRowsTimeOrderedInMemory()).andReturn(100000); replay(config); chest = new ScanQueryQueryToolChest(config, null); + numElements = 1000; } @Test public void testDescendingHeapsortListScanResultValues() { List inputs = new ArrayList<>(); - for (long i = 0; i < 1000; i++) { + for (long i = 0; i < numElements; i++) { HashMap event = new HashMap<>(); event.put("__time", i * 1000); inputs.add( @@ -94,22 +96,24 @@ public QueryRunner lookup( .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + int count = 0; Long previousTime = Long.MAX_VALUE; while (sorted.hasNext()) { + count++; ScanResultValue curr = sorted.next(); Long currentTime = (Long) ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); - Assert.assertTrue("Event timestamp is less than that of the previous event", - currentTime < previousTime); + Assert.assertTrue(currentTime < previousTime); previousTime = currentTime; } + Assert.assertEquals(numElements, count); } @Test public void testAscendingHeapsortListScanResultValues() { List inputs = new ArrayList<>(); - for (long i = 1000; i > 0; i--) { + for (long i = numElements; i > 0; i--) { HashMap event = new HashMap<>(); event.put("__time", i * 1000); inputs.add( @@ -144,16 +148,117 @@ public QueryRunner lookup( .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + int count = 0; Long previousTime = -1L; while (sorted.hasNext()) { + count++; ScanResultValue curr = sorted.next(); Long currentTime = (Long) ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); - Assert.assertTrue( - "Event timestamp is greater than that of the previous event", - currentTime > previousTime + Assert.assertTrue(currentTime > previousTime); + previousTime = currentTime; + } + Assert.assertEquals(numElements, count); + } + + @Test + public void testDescendingHeapsortCompactedListScanResultValues() + { + List inputs = new ArrayList<>(); + for (long i = 0; i < numElements; i++) { + inputs.add( + new ScanResultValue( + "some segment id", + Collections.singletonList("__time"), + Collections.singletonList(Collections.singletonList(new Long(i * 1000))) + ) + ); + } + ScanQuery scanQuery = new Druids.ScanQueryBuilder() + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .dataSource("some data source") + .intervals(new QuerySegmentSpec() + { + @Override + public List getIntervals() + { + return null; + } + + @Override + public QueryRunner lookup( + Query query, QuerySegmentWalker walker + ) + { + return null; + } + }) + .limit(99999) + .build(); + Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + + Long previousTime = Long.MAX_VALUE; + int count = 0 ; + while (sorted.hasNext()) { + count++; + ScanResultValue curr = sorted.next(); + Long currentTime = (Long) + ((List) (((List) curr.getEvents()).get(0))).get(0); + Assert.assertTrue(currentTime < previousTime); + previousTime = currentTime; + } + Assert.assertEquals(numElements, count); + } + + @Test + public void testAscendingHeapsortCompactedListScanResultValues() + { + List inputs = new ArrayList<>(); + for (long i = numElements; i > 0; i--) { + inputs.add( + new ScanResultValue( + "some segment id", + Collections.singletonList("__time"), + Collections.singletonList(Collections.singletonList(new Long(i * 1000))) + ) ); + } + ScanQuery scanQuery = new Druids.ScanQueryBuilder() + .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .dataSource("some data source") + .intervals(new QuerySegmentSpec() + { + @Override + public List getIntervals() + { + return null; + } + + @Override + public QueryRunner lookup( + Query query, QuerySegmentWalker walker + ) + { + return null; + } + }) + .limit(99999) + .build(); + Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + + Long previousTime = -1L; + int count = 0; + while (sorted.hasNext()) { + count++; + ScanResultValue curr = sorted.next(); + Long currentTime = (Long) + ((List) (((List) curr.getEvents()).get(0))).get(0); + Assert.assertTrue(currentTime > previousTime); previousTime = currentTime; } + Assert.assertEquals(numElements, count); } + } From 60b7684db725387b4d843385d9c61d50f2ed6744 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 13:02:13 -0800 Subject: [PATCH 21/91] Committing a param change to kick teamcity --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 3ac7bb2f2c79..511de6bca9ec 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -104,7 +104,7 @@ public class ScanBenchmark @Param({"2"}) private int numProcessingThreads; - @Param({"250000"}) + @Param({"200000"}) private int rowsPerSegment; @Param({"basic.A"}) From dfe4aa9681d04b8a31dcc1486e0447f29f6eb7bd Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 13:41:18 -0800 Subject: [PATCH 22/91] Fixed codestyle and forbidden API errors --- .../query/scan/ScanQueryQueryToolChest.java | 8 ++-- .../scan/ScanQueryQueryToolChestTest.java | 14 ++++--- ...canResultValueTimestampComparatorTest.java | 38 +++++++++++++------ 3 files changed, 39 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 337b016e14f6..0681ad714120 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -165,14 +165,14 @@ public Sequence run(QueryPlus queryPlus, Map heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) { + Iterator heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) + { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - PriorityQueue q = new PriorityQueue<> - (Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); while (inputIterator.hasNext()) { @@ -187,7 +187,7 @@ Iterator heapsortScanResultValues(Iterator inp // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order // will be maintained List sortedElements = new ArrayList<>(q.size()); - while (q.size() != 0 ) { + while (q.size() != 0) { sortedElements.add(q.poll()); } return sortedElements.iterator(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 6a556ea5d2cc..f3c69b5e53da 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -86,7 +86,8 @@ public List getIntervals() @Override public QueryRunner lookup( - Query query, QuerySegmentWalker walker + Query query, + QuerySegmentWalker walker ) { return null; @@ -138,7 +139,8 @@ public List getIntervals() @Override public QueryRunner lookup( - Query query, QuerySegmentWalker walker + Query query, + QuerySegmentWalker walker ) { return null; @@ -188,7 +190,8 @@ public List getIntervals() @Override public QueryRunner lookup( - Query query, QuerySegmentWalker walker + Query query, + QuerySegmentWalker walker ) { return null; @@ -199,7 +202,7 @@ public QueryRunner lookup( Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); Long previousTime = Long.MAX_VALUE; - int count = 0 ; + int count = 0; while (sorted.hasNext()) { count++; ScanResultValue curr = sorted.next(); @@ -238,7 +241,8 @@ public List getIntervals() @Override public QueryRunner lookup( - Query query, QuerySegmentWalker walker + Query query, + QuerySegmentWalker walker ) { return null; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index d7d860fb00c7..9492269e8ba0 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.query.scan; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.Druids; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -38,8 +39,13 @@ public class ScanResultValueTimestampComparatorTest private static QuerySegmentSpec intervalSpec; @BeforeClass - public void setup() { - intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(new Interval(0, 1))); + public void setup() + { + intervalSpec = new MultipleIntervalSegmentSpec( + Collections.singletonList( + new Interval(DateTimes.of("2012-01-01"), DateTimes.of("2012-01-01").plusHours(1)) + ) + ); } @@ -55,7 +61,7 @@ public void comparisonDescendingListTest() ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); - ArrayList> events1 = new ArrayList<>(); + ArrayList> events1 = new ArrayList<>();` ` HashMap event1 = new HashMap<>(); event1.put(ColumnHolder.TIME_COLUMN_NAME, new Long(42)); events1.add(event1); @@ -63,7 +69,8 @@ public void comparisonDescendingListTest() ScanResultValue s1 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events1); + events1 + ); ArrayList> events2 = new ArrayList<>(); HashMap event2 = new HashMap<>(); @@ -73,7 +80,8 @@ public void comparisonDescendingListTest() ScanResultValue s2 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events2); + events2 + ); Assert.assertEquals(1, comparator.compare(s1, s2)); } @@ -90,7 +98,7 @@ public void comparisonAscendingListTest() ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); - ArrayList> events1 = new ArrayList<>(); + ArrayList> events1 = new ArrayList<>(); HashMap event1 = new HashMap<>(); event1.put(ColumnHolder.TIME_COLUMN_NAME, new Long(42)); events1.add(event1); @@ -98,7 +106,8 @@ public void comparisonAscendingListTest() ScanResultValue s1 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events1); + events1 + ); ArrayList> events2 = new ArrayList<>(); HashMap event2 = new HashMap<>(); @@ -108,7 +117,8 @@ public void comparisonAscendingListTest() ScanResultValue s2 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events2); + events2 + ); Assert.assertEquals(-1, comparator.compare(s1, s2)); } @@ -132,7 +142,8 @@ public void comparisonDescendingCompactedListTest() ScanResultValue s1 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events1); + events1 + ); List> events2 = new ArrayList<>(); List event2 = Collections.singletonList(new Long(43)); @@ -141,7 +152,8 @@ public void comparisonDescendingCompactedListTest() ScanResultValue s2 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events2); + events2 + ); Assert.assertEquals(1, comparator.compare(s1, s2)); } @@ -165,7 +177,8 @@ public void comparisonAscendingCompactedListTest() ScanResultValue s1 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events1); + events1 + ); List> events2 = new ArrayList<>(); List event2 = Collections.singletonList(new Long(43)); @@ -174,7 +187,8 @@ public void comparisonAscendingCompactedListTest() ScanResultValue s2 = new ScanResultValue( "segmentId", Collections.singletonList(ColumnHolder.TIME_COLUMN_NAME), - events2); + events2 + ); Assert.assertEquals(-1, comparator.compare(s1, s2)); } From 10b5e0ca93a529d1b0e018c11fafc9c63071b8cd Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 13:42:12 -0800 Subject: [PATCH 23/91] . --- .../query/scan/ScanResultValueTimestampComparatorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 9492269e8ba0..440da59e5aa2 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -61,7 +61,7 @@ public void comparisonDescendingListTest() ScanResultValueTimestampComparator comparator = new ScanResultValueTimestampComparator(query); - ArrayList> events1 = new ArrayList<>();` ` + ArrayList> events1 = new ArrayList<>(); HashMap event1 = new HashMap<>(); event1.put(ColumnHolder.TIME_COLUMN_NAME, new Long(42)); events1.add(event1); From 8212a21cafc2ed4002607362f0661f4b5f6bef9d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 14:40:35 -0800 Subject: [PATCH 24/91] Improved conciseness --- .../query/scan/ScanQueryQueryToolChest.java | 2 +- .../ScanResultValueTimestampComparator.java | 1 - .../scan/ScanQueryQueryToolChestTest.java | 91 +++++-------------- .../druid/query/scan/ScanQueryRunnerTest.java | 1 - ...canResultValueTimestampComparatorTest.java | 1 - 5 files changed, 23 insertions(+), 73 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 0681ad714120..77ac6206be2d 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -200,7 +200,7 @@ private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator iterator, int batchSize) { - itr = iterator; + this.itr = iterator; this.batchSize = batchSize; } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 163c6a104496..98271e66dfdd 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -75,5 +75,4 @@ public boolean equals(Object obj) ScanResultValueTimestampComparator comp = (ScanResultValueTimestampComparator) obj; return this.scanQuery.equals(comp.scanQuery); } - } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index f3c69b5e53da..453baac036fd 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -46,6 +46,7 @@ public class ScanQueryQueryToolChestTest private static ScanQueryQueryToolChest chest; private static ScanQueryConfig config; private static int numElements; + private static QuerySegmentSpec emptySegmentSpec; @Before public void setup() @@ -55,6 +56,23 @@ public void setup() replay(config); chest = new ScanQueryQueryToolChest(config, null); numElements = 1000; + emptySegmentSpec = new QuerySegmentSpec() + { + @Override + public List getIntervals() + { + return null; + } + + @Override + public QueryRunner lookup( + Query query, + QuerySegmentWalker walker + ) + { + return null; + } + }; } @Test @@ -76,23 +94,7 @@ public void testDescendingHeapsortListScanResultValues() .resultFormat("list") .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) .dataSource("some data source") - .intervals(new QuerySegmentSpec() - { - @Override - public List getIntervals() - { - return null; - } - - @Override - public QueryRunner lookup( - Query query, - QuerySegmentWalker walker - ) - { - return null; - } - }) + .intervals(emptySegmentSpec) .limit(99999) .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); @@ -129,23 +131,7 @@ public void testAscendingHeapsortListScanResultValues() .resultFormat("list") .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) .dataSource("some data source") - .intervals(new QuerySegmentSpec() - { - @Override - public List getIntervals() - { - return null; - } - - @Override - public QueryRunner lookup( - Query query, - QuerySegmentWalker walker - ) - { - return null; - } - }) + .intervals(emptySegmentSpec) .limit(99999) .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); @@ -180,23 +166,7 @@ public void testDescendingHeapsortCompactedListScanResultValues() .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) .dataSource("some data source") - .intervals(new QuerySegmentSpec() - { - @Override - public List getIntervals() - { - return null; - } - - @Override - public QueryRunner lookup( - Query query, - QuerySegmentWalker walker - ) - { - return null; - } - }) + .intervals(emptySegmentSpec) .limit(99999) .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); @@ -231,23 +201,7 @@ public void testAscendingHeapsortCompactedListScanResultValues() .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) .dataSource("some data source") - .intervals(new QuerySegmentSpec() - { - @Override - public List getIntervals() - { - return null; - } - - @Override - public QueryRunner lookup( - Query query, - QuerySegmentWalker walker - ) - { - return null; - } - }) + .intervals(emptySegmentSpec) .limit(99999) .build(); Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); @@ -264,5 +218,4 @@ public QueryRunner lookup( } Assert.assertEquals(numElements, count); } - } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index ccc2df005802..750b76ea06cd 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -66,7 +66,6 @@ import java.util.Set; /** - * */ @RunWith(Parameterized.class) public class ScanQueryRunnerTest diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 440da59e5aa2..0de0c34ea7c8 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -48,7 +48,6 @@ public void setup() ); } - @Test public void comparisonDescendingListTest() { From 305876a4346c292296db623c1fcea688a29c0bb8 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 15:02:02 -0800 Subject: [PATCH 25/91] nit --- .../apache/druid/query/scan/ScanQueryQueryToolChestTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 453baac036fd..7535e7364767 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -27,7 +27,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.joda.time.Interval; import org.junit.Assert; -import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import java.util.ArrayList; @@ -48,7 +48,7 @@ public class ScanQueryQueryToolChestTest private static int numElements; private static QuerySegmentSpec emptySegmentSpec; - @Before + @BeforeClass public void setup() { config = createNiceMock(ScanQueryConfig.class); From 7e872a8ebcea0d3a141addd122dd9f8b6629ead6 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 15:36:24 -0800 Subject: [PATCH 26/91] Created an error message for when someone tries to time order a result set > threshold limit --- .../query/scan/ScanQueryQueryToolChest.java | 371 +++++++++--------- 1 file changed, 189 insertions(+), 182 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 77ac6206be2d..7b8d341698c3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -48,186 +48,193 @@ public class ScanQueryQueryToolChest extends QueryToolChest { - private static final TypeReference TYPE_REFERENCE = new TypeReference() - { - }; - - private final ScanQueryConfig scanQueryConfig; - private final GenericQueryMetricsFactory queryMetricsFactory; - private final long maxRowsForInMemoryTimeOrdering; - - @Inject - public ScanQueryQueryToolChest( - final ScanQueryConfig scanQueryConfig, - final GenericQueryMetricsFactory queryMetricsFactory - ) - { - this.scanQueryConfig = scanQueryConfig; - this.queryMetricsFactory = queryMetricsFactory; - this.maxRowsForInMemoryTimeOrdering = scanQueryConfig.getMaxRowsTimeOrderedInMemory(); - } - - @Override - public QueryRunner mergeResults(final QueryRunner runner) - { - return (queryPlus, responseContext) -> { - // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it - // the same way, even if they have different default legacy values. - final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); - final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = - new BaseSequence.IteratorMaker() - { - @Override - public ScanQueryLimitRowIterator make() - { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } - - @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }; - - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) || - scanQuery.getLimit() > maxRowsForInMemoryTimeOrdering) { - if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(queryPlusWithNonNullLegacy, responseContext); - } - return new BaseSequence(scanQueryLimitRowIteratorMaker); - } else if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || - scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { - Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); - - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public ScanBatchedTimeOrderedQueueIterator make() - { - return new ScanBatchedTimeOrderedQueueIterator( - heapsortScanResultValues(scanResultIterator, scanQuery), - scanQuery.getBatchSize() - ); - } - - @Override - public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); - } else { - throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); - } - }; - } - - @Override - public QueryMetrics> makeMetrics(ScanQuery query) - { - return queryMetricsFactory.makeMetrics(query); - } - - @Override - public Function makePreComputeManipulatorFn( - ScanQuery query, - MetricManipulationFn fn - ) - { - return Functions.identity(); - } - - @Override - public TypeReference getResultTypeReference() - { - return TYPE_REFERENCE; - } - - @Override - public QueryRunner preMergeQueryDecoration(final QueryRunner runner) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) - { - ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); - if (scanQuery.getFilter() != null) { - scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); - queryPlus = queryPlus.withQuery(scanQuery); - } - return runner.run(queryPlus, responseContext); - } - }; - } - - @VisibleForTesting - Iterator heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) - { - Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); - - // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch - // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - - PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); - - while (inputIterator.hasNext()) { - - ScanResultValue next = inputIterator.next(); - List events = (List) next.getEvents(); - for (Object event : events) { - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); - } - } - // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order - // will be maintained - List sortedElements = new ArrayList<>(q.size()); - while (q.size() != 0) { - sortedElements.add(q.poll()); - } - return sortedElements.iterator(); - } - - private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator - { - private final Iterator itr; - private final int batchSize; - - public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) - { - this.itr = iterator; - this.batchSize = batchSize; - } - - @Override - public void close() throws IOException - { - } - - @Override - public boolean hasNext() - { - return itr.hasNext(); - } - - @Override - public ScanResultValue next() - { - // Create new scanresultvalue from event map - List eventsToAdd = new ArrayList<>(batchSize); - List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && itr.hasNext()) { - ScanResultValue srv = itr.next(); - // Only replace once using the columns from the first event - columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(((List) srv.getEvents()).get(0)); - } - return new ScanResultValue(null, columns, eventsToAdd); - } - } + private static final TypeReference TYPE_REFERENCE = new TypeReference() + { + }; + + private final ScanQueryConfig scanQueryConfig; + private final GenericQueryMetricsFactory queryMetricsFactory; + private final long maxRowsForInMemoryTimeOrdering; + + @Inject + public ScanQueryQueryToolChest( + final ScanQueryConfig scanQueryConfig, + final GenericQueryMetricsFactory queryMetricsFactory + ) + { + this.scanQueryConfig = scanQueryConfig; + this.queryMetricsFactory = queryMetricsFactory; + this.maxRowsForInMemoryTimeOrdering = scanQueryConfig.getMaxRowsTimeOrderedInMemory(); + } + + @Override + public QueryRunner mergeResults(final QueryRunner runner) + { + return (queryPlus, responseContext) -> { + // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it + // the same way, even if they have different default legacy values. + final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); + final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); + final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() + { + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } + + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }; + + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE)) { + if (scanQuery.getLimit() == Long.MAX_VALUE) { + return runner.run(queryPlusWithNonNullLegacy, responseContext); + } + return new BaseSequence(scanQueryLimitRowIteratorMaker); + } else if ((scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || + scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) + && scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public ScanBatchedTimeOrderedQueueIterator make() + { + return new ScanBatchedTimeOrderedQueueIterator( + heapsortScanResultValues(scanResultIterator, scanQuery), + scanQuery.getBatchSize() + ); + } + + @Override + public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); + } else if (scanQuery.getLimit() > scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + throw new UOE( + "Time ordering for result set limit of %s is not supported. Try lowering the " + + "result set size to less than or equal to the time ordering limit of %s.", + scanQuery.getLimit(), + scanQueryConfig.getMaxRowsTimeOrderedInMemory() + ); + } else { + throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); + } + }; + } + + @Override + public QueryMetrics> makeMetrics(ScanQuery query) + { + return queryMetricsFactory.makeMetrics(query); + } + + @Override + public Function makePreComputeManipulatorFn( + ScanQuery query, + MetricManipulationFn fn + ) + { + return Functions.identity(); + } + + @Override + public TypeReference getResultTypeReference() + { + return TYPE_REFERENCE; + } + + @Override + public QueryRunner preMergeQueryDecoration(final QueryRunner runner) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); + if (scanQuery.getFilter() != null) { + scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); + queryPlus = queryPlus.withQuery(scanQuery); + } + return runner.run(queryPlus, responseContext); + } + }; + } + + @VisibleForTesting + Iterator heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) + { + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); + + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + + PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + + while (inputIterator.hasNext()) { + + ScanResultValue next = inputIterator.next(); + List events = (List) next.getEvents(); + for (Object event : events) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); + } + } + // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained + List sortedElements = new ArrayList<>(q.size()); + while (q.size() != 0) { + sortedElements.add(q.poll()); + } + return sortedElements.iterator(); + } + + private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator + { + private final Iterator itr; + private final int batchSize; + + public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) + { + this.itr = iterator; + this.batchSize = batchSize; + } + + @Override + public void close() throws IOException + { + } + + @Override + public boolean hasNext() + { + return itr.hasNext(); + } + + @Override + public ScanResultValue next() + { + // Create new scanresultvalue from event map + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && itr.hasNext()) { + ScanResultValue srv = itr.next(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(((List) srv.getEvents()).get(0)); + } + return new ScanResultValue(null, columns, eventsToAdd); + } + } } From 85e72a614ef49736d1142ce82d26b533b609c911 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 15:42:02 -0800 Subject: [PATCH 27/91] Set to spaces over tabs --- .../query/scan/ScanQueryQueryToolChest.java | 378 +++++++++--------- 1 file changed, 189 insertions(+), 189 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 7b8d341698c3..a1b9e149e4a7 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -48,193 +48,193 @@ public class ScanQueryQueryToolChest extends QueryToolChest { - private static final TypeReference TYPE_REFERENCE = new TypeReference() - { - }; - - private final ScanQueryConfig scanQueryConfig; - private final GenericQueryMetricsFactory queryMetricsFactory; - private final long maxRowsForInMemoryTimeOrdering; - - @Inject - public ScanQueryQueryToolChest( - final ScanQueryConfig scanQueryConfig, - final GenericQueryMetricsFactory queryMetricsFactory - ) - { - this.scanQueryConfig = scanQueryConfig; - this.queryMetricsFactory = queryMetricsFactory; - this.maxRowsForInMemoryTimeOrdering = scanQueryConfig.getMaxRowsTimeOrderedInMemory(); - } - - @Override - public QueryRunner mergeResults(final QueryRunner runner) - { - return (queryPlus, responseContext) -> { - // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it - // the same way, even if they have different default legacy values. - final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); - final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = - new BaseSequence.IteratorMaker() - { - @Override - public ScanQueryLimitRowIterator make() - { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } - - @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }; - - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE)) { - if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(queryPlusWithNonNullLegacy, responseContext); - } - return new BaseSequence(scanQueryLimitRowIteratorMaker); - } else if ((scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || - scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) - && scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { - Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); - - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public ScanBatchedTimeOrderedQueueIterator make() - { - return new ScanBatchedTimeOrderedQueueIterator( - heapsortScanResultValues(scanResultIterator, scanQuery), - scanQuery.getBatchSize() - ); - } - - @Override - public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); - } else if (scanQuery.getLimit() > scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { - throw new UOE( - "Time ordering for result set limit of %s is not supported. Try lowering the " - + "result set size to less than or equal to the time ordering limit of %s.", - scanQuery.getLimit(), - scanQueryConfig.getMaxRowsTimeOrderedInMemory() - ); - } else { - throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); - } - }; - } - - @Override - public QueryMetrics> makeMetrics(ScanQuery query) - { - return queryMetricsFactory.makeMetrics(query); - } - - @Override - public Function makePreComputeManipulatorFn( - ScanQuery query, - MetricManipulationFn fn - ) - { - return Functions.identity(); - } - - @Override - public TypeReference getResultTypeReference() - { - return TYPE_REFERENCE; - } - - @Override - public QueryRunner preMergeQueryDecoration(final QueryRunner runner) - { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) - { - ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); - if (scanQuery.getFilter() != null) { - scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); - queryPlus = queryPlus.withQuery(scanQuery); - } - return runner.run(queryPlus, responseContext); - } - }; - } - - @VisibleForTesting - Iterator heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) - { - Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); - - // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch - // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - - PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); - - while (inputIterator.hasNext()) { - - ScanResultValue next = inputIterator.next(); - List events = (List) next.getEvents(); - for (Object event : events) { - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); - } - } - // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order - // will be maintained - List sortedElements = new ArrayList<>(q.size()); - while (q.size() != 0) { - sortedElements.add(q.poll()); - } - return sortedElements.iterator(); - } - - private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator - { - private final Iterator itr; - private final int batchSize; - - public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) - { - this.itr = iterator; - this.batchSize = batchSize; - } - - @Override - public void close() throws IOException - { - } - - @Override - public boolean hasNext() - { - return itr.hasNext(); - } - - @Override - public ScanResultValue next() - { - // Create new scanresultvalue from event map - List eventsToAdd = new ArrayList<>(batchSize); - List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && itr.hasNext()) { - ScanResultValue srv = itr.next(); - // Only replace once using the columns from the first event - columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(((List) srv.getEvents()).get(0)); - } - return new ScanResultValue(null, columns, eventsToAdd); - } - } + private static final TypeReference TYPE_REFERENCE = new TypeReference() + { + }; + + private final ScanQueryConfig scanQueryConfig; + private final GenericQueryMetricsFactory queryMetricsFactory; + private final long maxRowsForInMemoryTimeOrdering; + + @Inject + public ScanQueryQueryToolChest( + final ScanQueryConfig scanQueryConfig, + final GenericQueryMetricsFactory queryMetricsFactory + ) + { + this.scanQueryConfig = scanQueryConfig; + this.queryMetricsFactory = queryMetricsFactory; + this.maxRowsForInMemoryTimeOrdering = scanQueryConfig.getMaxRowsTimeOrderedInMemory(); + } + + @Override + public QueryRunner mergeResults(final QueryRunner runner) + { + return (queryPlus, responseContext) -> { + // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it + // the same way, even if they have different default legacy values. + final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); + final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); + final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() + { + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } + + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }; + + if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE)) { + if (scanQuery.getLimit() == Long.MAX_VALUE) { + return runner.run(queryPlusWithNonNullLegacy, responseContext); + } + return new BaseSequence(scanQueryLimitRowIteratorMaker); + } else if ((scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || + scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) + && scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public ScanBatchedTimeOrderedQueueIterator make() + { + return new ScanBatchedTimeOrderedQueueIterator( + heapsortScanResultValues(scanResultIterator, scanQuery), + scanQuery.getBatchSize() + ); + } + + @Override + public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); + } else if (scanQuery.getLimit() > scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + throw new UOE( + "Time ordering for result set limit of %s is not supported. Try lowering the " + + "result set size to less than or equal to the time ordering limit of %s.", + scanQuery.getLimit(), + scanQueryConfig.getMaxRowsTimeOrderedInMemory() + ); + } else { + throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); + } + }; + } + + @Override + public QueryMetrics> makeMetrics(ScanQuery query) + { + return queryMetricsFactory.makeMetrics(query); + } + + @Override + public Function makePreComputeManipulatorFn( + ScanQuery query, + MetricManipulationFn fn + ) + { + return Functions.identity(); + } + + @Override + public TypeReference getResultTypeReference() + { + return TYPE_REFERENCE; + } + + @Override + public QueryRunner preMergeQueryDecoration(final QueryRunner runner) + { + return new QueryRunner() + { + @Override + public Sequence run(QueryPlus queryPlus, Map responseContext) + { + ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); + if (scanQuery.getFilter() != null) { + scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); + queryPlus = queryPlus.withQuery(scanQuery); + } + return runner.run(queryPlus, responseContext); + } + }; + } + + @VisibleForTesting + Iterator heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) + { + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); + + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + + PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + + while (inputIterator.hasNext()) { + + ScanResultValue next = inputIterator.next(); + List events = (List) next.getEvents(); + for (Object event : events) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); + } + } + // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained + List sortedElements = new ArrayList<>(q.size()); + while (q.size() != 0) { + sortedElements.add(q.poll()); + } + return sortedElements.iterator(); + } + + private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator + { + private final Iterator itr; + private final int batchSize; + + public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) + { + this.itr = iterator; + this.batchSize = batchSize; + } + + @Override + public void close() throws IOException + { + } + + @Override + public boolean hasNext() + { + return itr.hasNext(); + } + + @Override + public ScanResultValue next() + { + // Create new scanresultvalue from event map + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && itr.hasNext()) { + ScanResultValue srv = itr.next(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(((List) srv.getEvents()).get(0)); + } + return new ScanResultValue(null, columns, eventsToAdd); + } + } } From b2c8c77ad4ee5a9a273ee7a7870fb2d9b0ec9dd4 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 17:39:48 -0800 Subject: [PATCH 28/91] Fixing tests WIP --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 2 +- processing/src/main/java/org/apache/druid/query/Druids.java | 4 ++-- .../src/main/java/org/apache/druid/query/scan/ScanQuery.java | 4 ++-- .../org/apache/druid/query/scan/ScanQueryQueryToolChest.java | 4 +--- .../apache/druid/query/scan/ScanQueryQueryToolChestTest.java | 2 +- .../java/org/apache/druid/query/scan/ScanQuerySpecTest.java | 4 ++-- .../query/scan/ScanResultValueTimestampComparatorTest.java | 2 +- .../java/org/apache/druid/sql/calcite/rel/DruidQuery.java | 5 +++-- .../java/org/apache/druid/sql/calcite/CalciteQueryTest.java | 2 +- 9 files changed, 14 insertions(+), 15 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index bb492f2d9f77..267cea544a3e 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -94,7 +94,7 @@ /* Works with 8GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 10) +@Warmup(iterations = 25) @Measurement(iterations = 25) public class ScanBenchmark { diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index dd8b35c3f873..b0fe8edfecc5 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -950,11 +950,11 @@ public ScanQuery build() resultFormat, batchSize, limit, + timeOrder, dimFilter, columns, legacy, - context, - timeOrder + context ); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 0b1ee2356260..323c0a3246a5 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -61,11 +61,11 @@ public ScanQuery( @JsonProperty("resultFormat") String resultFormat, @JsonProperty("batchSize") int batchSize, @JsonProperty("limit") long limit, + @JsonProperty("timeOrder") String timeOrder, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("columns") List columns, @JsonProperty("legacy") Boolean legacy, - @JsonProperty("context") Map context, - @JsonProperty("timeOrder") String timeOrder + @JsonProperty("context") Map context ) { super(dataSource, querySegmentSpec, false, context); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index a1b9e149e4a7..43a55ea10e93 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -54,7 +54,6 @@ public class ScanQueryQueryToolChest extends QueryToolChest heapsortScanResultValues(Iterator inp return sortedElements.iterator(); } - private class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator + private static class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 7535e7364767..10364ae2743b 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -49,7 +49,7 @@ public class ScanQueryQueryToolChestTest private static QuerySegmentSpec emptySegmentSpec; @BeforeClass - public void setup() + public static void setup() { config = createNiceMock(ScanQueryConfig.class); expect(config.getMaxRowsTimeOrderedInMemory()).andReturn(100000); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index e1e322649fbe..9cfbade42224 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -68,11 +68,11 @@ public void testSerializationLegacyString() throws Exception null, 0, 3, + "none", null, Arrays.asList("market", "quality", "index"), null, - null, - "none" + null ); String actual = jsonMapper.writeValueAsString(query); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 0de0c34ea7c8..005a820ec759 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -39,7 +39,7 @@ public class ScanResultValueTimestampComparatorTest private static QuerySegmentSpec intervalSpec; @BeforeClass - public void setup() + public static void setup() { intervalSpec = new MultipleIntervalSegmentSpec( Collections.singletonList( diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 5bdc37c3d667..6cb4eafa4127 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -964,11 +964,12 @@ public ScanQuery toScanQuery() ScanQuery.RESULT_FORMAT_COMPACTED_LIST, 0, scanLimit, + null, // Will default to "none" filtration.getDimFilter(), Ordering.natural().sortedCopy(ImmutableSet.copyOf(outputRowSignature.getRowOrder())), false, - ImmutableSortedMap.copyOf(plannerContext.getQueryContext()), - null // Will default to "none" + ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) + ); } 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 b89ae45a1d47..a0bf8f74fc8b 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 @@ -770,7 +770,7 @@ public void testExplainSelfJoinWithFallback() throws Exception + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":" + emptyStringEq + ",\"extractionFn\":null}},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"; + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"; testQuery( PLANNER_CONFIG_FALLBACK, From b432beaf84de5b363454fd8058ff653a097c713d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Feb 2019 17:45:59 -0800 Subject: [PATCH 29/91] Fixed failing calcite tests --- .../java/org/apache/druid/sql/calcite/CalciteQueryTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 a0bf8f74fc8b..7ffb44965e3a 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 @@ -511,7 +511,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" } ) ); @@ -767,7 +767,7 @@ public void testExplainSelfJoinWithFallback() throws Exception String emptyStringEq = NullHandling.replaceWithDefault() ? null : "\"\""; final String explanation = "BindableJoin(condition=[=($0, $2)], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":" + emptyStringEq + ",\"extractionFn\":null}},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"; @@ -6645,7 +6645,7 @@ public void testUsingSubqueryAsPartOfOrFilter() throws Exception + " BindableFilter(condition=[OR(=($0, 'xxx'), CAST(AND(IS NOT NULL($4), <>($2, 0), IS NOT NULL($1))):BOOLEAN)])\n" + " BindableJoin(condition=[=($1, $3)], joinType=[left])\n" + " BindableJoin(condition=[true], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n" + " DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"}}], signature=[{a0:LONG}])\n" + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"d1:v\",\"expression\":\"1\",\"outputType\":\"LONG\"}],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"d1:v\",\"outputName\":\"d1\",\"outputType\":\"LONG\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false}], signature=[{d0:STRING, d1:LONG}])\n"; From ab00eade9f0b8e8642da40905214653c04cba4d4 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Feb 2019 09:42:48 -0800 Subject: [PATCH 30/91] Kicking travis with change to benchmark param --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 267cea544a3e..bb492f2d9f77 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -94,7 +94,7 @@ /* Works with 8GB heap size or greater. Otherwise there's a good chance of an OOME. */ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 25) +@Warmup(iterations = 10) @Measurement(iterations = 25) public class ScanBenchmark { From d3b335af42602a771063bd8a63c89acf5c715938 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Feb 2019 11:08:07 -0800 Subject: [PATCH 31/91] added all query types to scan benchmark --- .../java/org/apache/druid/benchmark/query/ScanBenchmark.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index bb492f2d9f77..df5e7ca1b719 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -107,7 +107,7 @@ public class ScanBenchmark @Param({"200000"}) private int rowsPerSegment; - @Param({"basic.A"}) + @Param({"basic.A", "basic.B", "basic.C", "basic.D"}) private String schemaAndQuery; @Param({"1000", "99999"}) From 2e3577cd3d7b43e140d36aad944536f49287fbfa Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Feb 2019 13:01:25 -0800 Subject: [PATCH 32/91] Fixed benchmark queries --- .../druid/benchmark/query/ScanBenchmark.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index df5e7ca1b719..dc8ecc58ea30 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -201,6 +201,7 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .filters(filter) + .dataSource("blah") .intervals(intervalSpec) .timeOrder(timeOrdering); } @@ -212,9 +213,10 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() - .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) - .intervals(intervalSpec) - .timeOrder(timeOrdering); + .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) + .intervals(intervalSpec) + .dataSource("blah") + .timeOrder(timeOrdering); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -226,9 +228,10 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch final String dimName = "dimUniform"; return Druids.newScanQueryBuilder() - .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) - .intervals(intervalSpec) - .timeOrder(timeOrdering); + .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) + .intervals(intervalSpec) + .dataSource("blah") + .timeOrder(timeOrdering); } @Setup From 134041c47965a8a199862ca33ef2119e29f67287 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 8 Feb 2019 13:13:54 -0800 Subject: [PATCH 33/91] Renamed sort function --- .../apache/druid/query/scan/ScanQueryQueryToolChest.java | 5 ++--- .../druid/query/scan/ScanQueryQueryToolChestTest.java | 8 ++++---- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 43a55ea10e93..863f4b944cd1 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -106,7 +106,7 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) public ScanBatchedTimeOrderedQueueIterator make() { return new ScanBatchedTimeOrderedQueueIterator( - heapsortScanResultValues(scanResultIterator, scanQuery), + sortScanResultValues(scanResultIterator, scanQuery), scanQuery.getBatchSize() ); } @@ -170,7 +170,7 @@ public Sequence run(QueryPlus queryPlus, Map heapsortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) + Iterator sortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -180,7 +180,6 @@ Iterator heapsortScanResultValues(Iterator inp PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); while (inputIterator.hasNext()) { - ScanResultValue next = inputIterator.next(); List events = (List) next.getEvents(); for (Object event : events) { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 10364ae2743b..8a057924d896 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -97,7 +97,7 @@ public void testDescendingHeapsortListScanResultValues() .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); int count = 0; Long previousTime = Long.MAX_VALUE; @@ -134,7 +134,7 @@ public void testAscendingHeapsortListScanResultValues() .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); int count = 0; Long previousTime = -1L; @@ -169,7 +169,7 @@ public void testDescendingHeapsortCompactedListScanResultValues() .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); Long previousTime = Long.MAX_VALUE; int count = 0; @@ -204,7 +204,7 @@ public void testAscendingHeapsortCompactedListScanResultValues() .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.heapsortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); Long previousTime = -1L; int count = 0; From 93e1636287f45d38c80f275e4644c0b3222c65e7 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Feb 2019 10:03:14 -0800 Subject: [PATCH 34/91] Added javadoc on ScanResultValueTimestampComparator --- .../query/scan/ScanResultValueTimestampComparator.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 98271e66dfdd..99098a3974ca 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -21,12 +21,21 @@ import com.google.common.primitives.Longs; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.column.ColumnHolder; import java.util.Comparator; +import java.util.Iterator; import java.util.List; import java.util.Map; +/** + * This comparator class supports comparisons of ScanResultValues based on the timestamp of their first event. Since + * only the first event is looked at, this Comparator is especially useful for unbatched ScanResultValues (such as in + * {@link ScanQueryQueryToolChest#mergeResults(QueryRunner )}. The comparator takes a scanQuery as + * a parameter so that it knows the result format (list or compactedList) of Object ScanResultValue#events. It uses + * this result format to perform a bunch of type casts on the object to get the timestamp then compares the timestamps. + */ public class ScanResultValueTimestampComparator implements Comparator { private final ScanQuery scanQuery; From 5f92dd7325aeff0b2e3f87003263e083ba2b427d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Feb 2019 10:05:58 -0800 Subject: [PATCH 35/91] Unused import --- .../druid/query/scan/ScanResultValueTimestampComparator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 99098a3974ca..66894ae35574 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -25,7 +25,6 @@ import org.apache.druid.segment.column.ColumnHolder; import java.util.Comparator; -import java.util.Iterator; import java.util.List; import java.util.Map; From f0eddee66598095a767a1570516c5af59e58e2f6 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Feb 2019 10:18:45 -0800 Subject: [PATCH 36/91] Added more javadoc --- .../query/scan/ScanQueryQueryToolChest.java | 17 +++++++++++------ .../ScanResultValueTimestampComparator.java | 4 ++-- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 863f4b944cd1..309d29a23723 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -100,19 +100,19 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); return new BaseSequence( - new BaseSequence.IteratorMaker() + new BaseSequence.IteratorMaker() { @Override - public ScanBatchedTimeOrderedQueueIterator make() + public ScanBatchedTimeOrderedIterator make() { - return new ScanBatchedTimeOrderedQueueIterator( + return new ScanBatchedTimeOrderedIterator( sortScanResultValues(scanResultIterator, scanQuery), scanQuery.getBatchSize() ); } @Override - public void cleanup(ScanBatchedTimeOrderedQueueIterator iterFromMake) + public void cleanup(ScanBatchedTimeOrderedIterator iterFromMake) { CloseQuietly.close(iterFromMake); } @@ -197,12 +197,17 @@ Iterator sortScanResultValues(Iterator inputIt return sortedElements.iterator(); } - private static class ScanBatchedTimeOrderedQueueIterator implements CloseableIterator + /** + * This iterator supports iteration through any Iterable and aggregates events in the Iterable into ScanResultValues + * with {int batchSize} events. The columns from the first event per ScanResultValue will be used to populate the + * column section. + */ + private static class ScanBatchedTimeOrderedIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; - public ScanBatchedTimeOrderedQueueIterator(Iterator iterator, int batchSize) + public ScanBatchedTimeOrderedIterator(Iterator iterator, int batchSize) { this.itr = iterator; this.batchSize = batchSize; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 66894ae35574..289a1cddf806 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -50,8 +50,8 @@ public int compare(ScanResultValue o1, ScanResultValue o2) int comparison; if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { comparison = Longs.compare( - (Long) ((Map) ((List) o1.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), - (Long) ((Map) ((List) o2.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) + (Long) ((Map) ((List) o1.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), + (Long) ((Map) ((List) o2.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) ); } else if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { int val1TimeColumnIndex = o1.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); From ecb0f483a9525ffc2844cb01a0daafe6bc4d2161 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Feb 2019 10:37:11 -0800 Subject: [PATCH 37/91] improved doc --- .../apache/druid/query/scan/ScanQueryQueryToolChest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 309d29a23723..575b9cc3f589 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -198,9 +198,9 @@ Iterator sortScanResultValues(Iterator inputIt } /** - * This iterator supports iteration through any Iterable and aggregates events in the Iterable into ScanResultValues - * with {int batchSize} events. The columns from the first event per ScanResultValue will be used to populate the - * column section. + * This iterator supports iteration through any Iterable of unbatched ScanResultValues (1 event/SRV) and aggregates + * events into ScanResultValues with {int batchSize} events. The columns from the first event per ScanResultValue + * will be used to populate the column section. */ private static class ScanBatchedTimeOrderedIterator implements CloseableIterator { @@ -227,7 +227,7 @@ public boolean hasNext() @Override public ScanResultValue next() { - // Create new scanresultvalue from event map + // Create new ScanResultValue from event map List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); while (eventsToAdd.size() < batchSize && itr.hasNext()) { From 4e69276d57de4a9042b927efa5a864411aedacb4 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 11 Feb 2019 12:09:54 -0800 Subject: [PATCH 38/91] Removed unused import to satisfy PMD check --- .../druid/query/scan/ScanResultValueTimestampComparator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 289a1cddf806..c2edea613120 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -21,7 +21,6 @@ import com.google.common.primitives.Longs; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.query.QueryRunner; import org.apache.druid.segment.column.ColumnHolder; import java.util.Comparator; @@ -31,7 +30,7 @@ /** * This comparator class supports comparisons of ScanResultValues based on the timestamp of their first event. Since * only the first event is looked at, this Comparator is especially useful for unbatched ScanResultValues (such as in - * {@link ScanQueryQueryToolChest#mergeResults(QueryRunner )}. The comparator takes a scanQuery as + * {@link ScanQueryQueryToolChest#mergeResults(org.apache.druid.query.QueryRunner )}. The comparator takes a scanQuery as * a parameter so that it knows the result format (list or compactedList) of Object ScanResultValue#events. It uses * this result format to perform a bunch of type casts on the object to get the timestamp then compares the timestamps. */ From 35150fe1a63c5143f564c4435461929e619a0de2 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 15 Feb 2019 15:57:53 -0800 Subject: [PATCH 39/91] Small changes --- .../druid/benchmark/query/ScanBenchmark.java | 2 +- .../java/org/apache/druid/query/Druids.java | 4 +- .../apache/druid/query/scan/ScanQuery.java | 16 ++++---- .../druid/query/scan/ScanQueryConfig.java | 1 - .../druid/query/scan/ScanQueryEngine.java | 4 +- .../query/scan/ScanQueryQueryToolChest.java | 35 ++++++++--------- .../druid/query/scan/ScanResultValue.java | 18 ++++++++- .../ScanResultValueTimestampComparator.java | 39 ++----------------- .../scan/ScanQueryQueryToolChestTest.java | 8 ++-- ...canResultValueTimestampComparatorTest.java | 8 ++-- 10 files changed, 59 insertions(+), 76 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index dc8ecc58ea30..8bea3f4a0a32 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -114,7 +114,7 @@ public class ScanBenchmark private int limit; @Param({"none", "descending", "ascending"}) - private static String timeOrdering; + private static ScanQuery.TimeOrder timeOrdering; private static final Logger log = new Logger(ScanBenchmark.class); private static final ObjectMapper JSON_MAPPER; diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index b0fe8edfecc5..ba4f6c13f606 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -924,7 +924,7 @@ public static class ScanQueryBuilder private DimFilter dimFilter; private List columns; private Boolean legacy; - private String timeOrder; + private ScanQuery.TimeOrder timeOrder; public ScanQueryBuilder() { @@ -1051,7 +1051,7 @@ public ScanQueryBuilder legacy(Boolean legacy) return this; } - public ScanQueryBuilder timeOrder(String timeOrder) + public ScanQueryBuilder timeOrder(ScanQuery.TimeOrder timeOrder) { this.timeOrder = timeOrder; return this; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 323c0a3246a5..071a01b70626 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -40,9 +40,11 @@ public class ScanQuery extends BaseQuery public static final String RESULT_FORMAT_COMPACTED_LIST = "compactedList"; public static final String RESULT_FORMAT_VALUE_VECTOR = "valueVector"; - public static final String TIME_ORDER_ASCENDING = "ascending"; - public static final String TIME_ORDER_DESCENDING = "descending"; - public static final String TIME_ORDER_NONE = "none"; + public enum TimeOrder { + @JsonProperty("ascending") ASCENDING, + @JsonProperty("descending") DESCENDING, + @JsonProperty("none") NONE + } private final VirtualColumns virtualColumns; private final String resultFormat; @@ -51,7 +53,7 @@ public class ScanQuery extends BaseQuery private final DimFilter dimFilter; private final List columns; private final Boolean legacy; - private final String timeOrder; + private final TimeOrder timeOrder; @JsonCreator public ScanQuery( @@ -61,7 +63,7 @@ public ScanQuery( @JsonProperty("resultFormat") String resultFormat, @JsonProperty("batchSize") int batchSize, @JsonProperty("limit") long limit, - @JsonProperty("timeOrder") String timeOrder, + @JsonProperty("timeOrder") TimeOrder timeOrder, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("columns") List columns, @JsonProperty("legacy") Boolean legacy, @@ -78,7 +80,7 @@ public ScanQuery( this.dimFilter = dimFilter; this.columns = columns; this.legacy = legacy; - this.timeOrder = timeOrder == null ? TIME_ORDER_NONE : timeOrder; + this.timeOrder = timeOrder == null ? TimeOrder.NONE : timeOrder; } @JsonProperty @@ -106,7 +108,7 @@ public long getLimit() } @JsonProperty - public String getTimeOrder() + public TimeOrder getTimeOrder() { return timeOrder; } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index b11e67a6f63d..d2ec4ba5b29e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -40,7 +40,6 @@ public ScanQueryConfig setLegacy(final boolean legacy) return this; } - // int should suffice here because no one should be sorting greater than 2B rows in memory @JsonProperty private int maxRowsTimeOrderedInMemory = 100000; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 069b2dfe8d5b..cb6b7871e5be 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -131,8 +131,8 @@ public Sequence process( intervals.get(0), query.getVirtualColumns(), Granularities.ALL, - query.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING) || - (query.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE) && query.isDescending()), + query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING) || + (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) && query.isDescending()), null ) .map(cursor -> new BaseSequence<>( diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 575b9cc3f589..8d9ae742a854 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.collect.Iterables; import com.google.inject.Inject; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -89,43 +90,39 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) } }; - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_NONE)) { + if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { if (scanQuery.getLimit() == Long.MAX_VALUE) { return runner.run(queryPlusWithNonNullLegacy, responseContext); } return new BaseSequence(scanQueryLimitRowIteratorMaker); - } else if ((scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_ASCENDING) || - scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) - && scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + } else if (scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); return new BaseSequence( - new BaseSequence.IteratorMaker() + new BaseSequence.IteratorMaker() { @Override - public ScanBatchedTimeOrderedIterator make() + public ScanBatchedIterator make() { - return new ScanBatchedTimeOrderedIterator( + return new ScanBatchedIterator( sortScanResultValues(scanResultIterator, scanQuery), scanQuery.getBatchSize() ); } @Override - public void cleanup(ScanBatchedTimeOrderedIterator iterFromMake) + public void cleanup(ScanBatchedIterator iterFromMake) { CloseQuietly.close(iterFromMake); } }); - } else if (scanQuery.getLimit() > scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + } else { throw new UOE( - "Time ordering for result set limit of %s is not supported. Try lowering the " - + "result set size to less than or equal to the time ordering limit of %s.", + "Time ordering for result set limit of %,d is not supported. Try lowering the " + + "result set size to less than or equal to the time ordering limit of %,d.", scanQuery.getLimit(), scanQueryConfig.getMaxRowsTimeOrderedInMemory() ); - } else { - throw new UOE("Time ordering [%s] is not supported", scanQuery.getTimeOrder()); } }; } @@ -198,16 +195,16 @@ Iterator sortScanResultValues(Iterator inputIt } /** - * This iterator supports iteration through any Iterable of unbatched ScanResultValues (1 event/SRV) and aggregates - * events into ScanResultValues with {int batchSize} events. The columns from the first event per ScanResultValue - * will be used to populate the column section. + * This iterator supports iteration through any Iterable of unbatched ScanResultValues (1 event/ScanResultValue) and + * aggregates events into ScanResultValues with {@code batchSize} events. The columns from the first event per + * ScanResultValue will be used to populate the column section. */ - private static class ScanBatchedTimeOrderedIterator implements CloseableIterator + private static class ScanBatchedIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; - public ScanBatchedTimeOrderedIterator(Iterator iterator, int batchSize) + public ScanBatchedIterator(Iterator iterator, int batchSize) { this.itr = iterator; this.batchSize = batchSize; @@ -234,7 +231,7 @@ public ScanResultValue next() ScanResultValue srv = itr.next(); // Only replace once using the columns from the first event columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(((List) srv.getEvents()).get(0)); + eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); } return new ScanResultValue(null, columns, eventsToAdd); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java index 8fc151965478..a9a10fa366e0 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java @@ -21,8 +21,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.segment.column.ColumnHolder; +import javax.annotation.Nullable; import java.util.List; +import java.util.Map; public class ScanResultValue implements Comparable { @@ -40,7 +44,7 @@ public class ScanResultValue implements Comparable @JsonCreator public ScanResultValue( - @JsonProperty("segmentId") String segmentId, + @Nullable @JsonProperty("segmentId") String segmentId, @JsonProperty("columns") List columns, @JsonProperty("events") Object events ) @@ -50,6 +54,7 @@ public ScanResultValue( this.events = events; } + @Nullable @JsonProperty public String getSegmentId() { @@ -68,6 +73,17 @@ public Object getEvents() return events; } + public long getFirstEventTimestamp(ScanQuery query) { + if (query.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { + return (Long) ((Map) ((List) this.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME); + } else if (query.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { + int timeColumnIndex = this.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); + List firstEvent = (List) ((List) this.getEvents()).get(0); + return (Long)firstEvent.get(timeColumnIndex); + } + throw new UOE("Unable to get first event timestamp using result format of [%s]", query.getResultFormat()); + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index c2edea613120..66abbe612b90 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -20,12 +20,8 @@ package org.apache.druid.query.scan; import com.google.common.primitives.Longs; -import org.apache.druid.java.util.common.UOE; -import org.apache.druid.segment.column.ColumnHolder; import java.util.Comparator; -import java.util.List; -import java.util.Map; /** * This comparator class supports comparisons of ScanResultValues based on the timestamp of their first event. Since @@ -47,39 +43,12 @@ public ScanResultValueTimestampComparator(ScanQuery scanQuery) public int compare(ScanResultValue o1, ScanResultValue o2) { int comparison; - if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { - comparison = Longs.compare( - (Long) ((Map) ((List) o1.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME), - (Long) ((Map) ((List) o2.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME) - ); - } else if (scanQuery.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { - int val1TimeColumnIndex = o1.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); - int val2TimeColumnIndex = o2.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); - List event1 = (List) ((List) o1.getEvents()).get(0); - List event2 = (List) ((List) o2.getEvents()).get(0); - comparison = Longs.compare( - (Long) event1.get(val1TimeColumnIndex), - (Long) event2.get(val2TimeColumnIndex) - ); - } else { - throw new UOE("Result format [%s] is not supported", scanQuery.getResultFormat()); - } - if (scanQuery.getTimeOrder().equals(ScanQuery.TIME_ORDER_DESCENDING)) { + comparison = Longs.compare( + o1.getFirstEventTimestamp(scanQuery), + o2.getFirstEventTimestamp(scanQuery)); + if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { return comparison * -1; } return comparison; } - - @Override - public boolean equals(Object obj) - { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - ScanResultValueTimestampComparator comp = (ScanResultValueTimestampComparator) obj; - return this.scanQuery.equals(comp.scanQuery); - } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 8a057924d896..137855de509a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -92,7 +92,7 @@ public void testDescendingHeapsortListScanResultValues() } ScanQuery scanQuery = new Druids.ScanQueryBuilder() .resultFormat("list") - .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) @@ -129,7 +129,7 @@ public void testAscendingHeapsortListScanResultValues() } ScanQuery scanQuery = new Druids.ScanQueryBuilder() .resultFormat("list") - .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) @@ -164,7 +164,7 @@ public void testDescendingHeapsortCompactedListScanResultValues() } ScanQuery scanQuery = new Druids.ScanQueryBuilder() .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) @@ -199,7 +199,7 @@ public void testAscendingHeapsortCompactedListScanResultValues() } ScanQuery scanQuery = new Druids.ScanQueryBuilder() .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 005a820ec759..5e55e8948584 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -52,7 +52,7 @@ public static void setup() public void comparisonDescendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .resultFormat(ScanQuery.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -89,7 +89,7 @@ public void comparisonDescendingListTest() public void comparisonAscendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .resultFormat(ScanQuery.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -126,7 +126,7 @@ public void comparisonAscendingListTest() public void comparisonDescendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TIME_ORDER_DESCENDING) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -161,7 +161,7 @@ public void comparisonDescendingCompactedListTest() public void comparisonAscendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TIME_ORDER_ASCENDING) + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) From 7baeade8328776244e72a3cb5f2efb59111cf58b Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 19 Feb 2019 17:52:06 -0800 Subject: [PATCH 40/91] Changes based on Gian's comments --- .../druid/benchmark/query/ScanBenchmark.java | 2 +- docs/content/querying/scan-query.md | 7 +- .../java/org/apache/druid/query/Druids.java | 4 +- .../apache/druid/query/scan/ScanQuery.java | 82 ++++++++++++++--- .../druid/query/scan/ScanQueryEngine.java | 8 +- .../query/scan/ScanQueryLimitRowIterator.java | 8 +- .../scan/ScanQueryNoLimitRowIterator.java | 92 +++++++++++++++++++ .../query/scan/ScanQueryQueryToolChest.java | 68 +++++++++----- .../druid/query/scan/ScanResultValue.java | 9 +- .../ScanResultValueTimestampComparator.java | 4 +- .../scan/ScanQueryQueryToolChestTest.java | 16 ++-- .../druid/query/scan/ScanQueryRunnerTest.java | 16 ++-- .../druid/query/scan/ScanQuerySpecTest.java | 4 +- ...canResultValueTimestampComparatorTest.java | 16 ++-- .../druid/sql/calcite/rel/DruidQuery.java | 2 +- .../sql/calcite/BaseCalciteQueryTest.java | 2 +- .../druid/sql/calcite/CalciteQueryTest.java | 34 +++---- 17 files changed, 275 insertions(+), 99 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 8bea3f4a0a32..9d302a801791 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -113,7 +113,7 @@ public class ScanBenchmark @Param({"1000", "99999"}) private int limit; - @Param({"none", "descending", "ascending"}) + @Param({"NONE", "DESCENDING", "ASCENDING"}) private static ScanQuery.TimeOrder timeOrdering; private static final Logger log = new Logger(ScanBenchmark.class); diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index d7866abbae7d..a025ea5c4c69 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -27,8 +27,11 @@ title: "Scan query" The Scan query returns raw Druid rows in streaming mode. The biggest difference between the Select query and the Scan query is that the Scan query does not retain all the returned rows in memory before they are returned to the client (except when time-ordering is used). The Select query _will_ retain the rows in memory, causing memory pressure if too -many rows are returned. The Scan query can return all the rows without issuing another pagination query, which is -extremely useful when directly querying against historical or realtime nodes. +many rows are returned. The Scan query can return all the rows without issuing another pagination query. + +In addition to straightforward usage where a Scan query is issued to the Broker, the Scan query can also be issued +directly to Historical processes or streaming ingestion tasks. This can be useful if you want to retrieve large +amounts of data in parallel. An example Scan query object is shown below: diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index ba4f6c13f606..a0d85cbb5f95 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -918,7 +918,7 @@ public static class ScanQueryBuilder private QuerySegmentSpec querySegmentSpec; private VirtualColumns virtualColumns; private Map context; - private String resultFormat; + private ScanQuery.ResultFormat resultFormat; private int batchSize; private long limit; private DimFilter dimFilter; @@ -1009,7 +1009,7 @@ public ScanQueryBuilder context(Map c) return this; } - public ScanQueryBuilder resultFormat(String r) + public ScanQueryBuilder resultFormat(ScanQuery.ResultFormat r) { resultFormat = r; return this; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 071a01b70626..57ad220afcbb 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -21,7 +21,9 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; @@ -36,18 +38,76 @@ public class ScanQuery extends BaseQuery { - public static final String RESULT_FORMAT_LIST = "list"; - public static final String RESULT_FORMAT_COMPACTED_LIST = "compactedList"; - public static final String RESULT_FORMAT_VALUE_VECTOR = "valueVector"; + public enum ResultFormat + { + RESULT_FORMAT_LIST, + RESULT_FORMAT_COMPACTED_LIST, + RESULT_FORMAT_VALUE_VECTOR; + + @JsonValue + @Override + public String toString() + { + switch (this) { + case RESULT_FORMAT_LIST: + return "list"; + case RESULT_FORMAT_COMPACTED_LIST: + return "compactedList"; + case RESULT_FORMAT_VALUE_VECTOR: + return "valueVector"; + default: + return ""; + } + } + + @JsonCreator + public static ResultFormat fromString(String name) + { + switch (name) { + case "compactedList": + return RESULT_FORMAT_COMPACTED_LIST; + case "valueVector": + return RESULT_FORMAT_VALUE_VECTOR; + case "list": + return RESULT_FORMAT_LIST; + default: + return RESULT_FORMAT_LIST; + } + } + + public byte[] getCacheKey() + { + return new byte[]{(byte) this.ordinal()}; + } + } - public enum TimeOrder { - @JsonProperty("ascending") ASCENDING, - @JsonProperty("descending") DESCENDING, - @JsonProperty("none") NONE + public enum TimeOrder + { + ASCENDING, + DESCENDING, + NONE; + + @JsonValue + @Override + public String toString() + { + return StringUtils.toLowerCase(this.name()); + } + + @JsonCreator + public static TimeOrder fromString(String name) + { + return valueOf(StringUtils.toUpperCase(name)); + } + + public byte[] getCacheKey() + { + return new byte[]{(byte) this.ordinal()}; + } } private final VirtualColumns virtualColumns; - private final String resultFormat; + private final ResultFormat resultFormat; private final int batchSize; private final long limit; private final DimFilter dimFilter; @@ -60,7 +120,7 @@ public ScanQuery( @JsonProperty("dataSource") DataSource dataSource, @JsonProperty("intervals") QuerySegmentSpec querySegmentSpec, @JsonProperty("virtualColumns") VirtualColumns virtualColumns, - @JsonProperty("resultFormat") String resultFormat, + @JsonProperty("resultFormat") ResultFormat resultFormat, @JsonProperty("batchSize") int batchSize, @JsonProperty("limit") long limit, @JsonProperty("timeOrder") TimeOrder timeOrder, @@ -72,7 +132,7 @@ public ScanQuery( { super(dataSource, querySegmentSpec, false, context); this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); - this.resultFormat = resultFormat == null ? RESULT_FORMAT_LIST : resultFormat; + this.resultFormat = resultFormat; this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize; this.limit = (limit == 0) ? Long.MAX_VALUE : limit; Preconditions.checkArgument(this.batchSize > 0, "batchSize must be greater than 0"); @@ -90,7 +150,7 @@ public VirtualColumns getVirtualColumns() } @JsonProperty - public String getResultFormat() + public ResultFormat getResultFormat() { return resultFormat; } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index cb6b7871e5be..12c38f0d48e1 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -178,13 +178,13 @@ public ScanResultValue next() } final long lastOffset = offset; final Object events; - final String resultFormat = query.getResultFormat(); - if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) { + final ScanQuery.ResultFormat resultFormat = query.getResultFormat(); + if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat)) { events = rowsToCompactedList(); - } else if (ScanQuery.RESULT_FORMAT_LIST.equals(resultFormat)) { + } else if (ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { events = rowsToList(); } else { - throw new UOE("resultFormat[%s] is not supported", resultFormat); + throw new UOE("resultFormat[%s] is not supported", resultFormat.toString()); } responseContext.put( ScanQueryRunnerFactory.CTX_COUNT, diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 3f56054b5f49..6fc0685e04ff 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -33,7 +33,7 @@ public class ScanQueryLimitRowIterator implements CloseableIterator { private Yielder yielder; - private String resultFormat; + private ScanQuery.ResultFormat resultFormat; private long limit; private long count = 0; @@ -71,8 +71,8 @@ public boolean hasNext() public ScanResultValue next() { ScanResultValue batch = yielder.get(); - if (ScanQuery.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || - ScanQuery.RESULT_FORMAT_LIST.equals(resultFormat)) { + if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || + ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { List events = (List) batch.getEvents(); if (events.size() <= limit - count) { count += events.size(); @@ -86,7 +86,7 @@ public ScanResultValue next() return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, left)); } } - throw new UnsupportedOperationException(ScanQuery.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); + throw new UnsupportedOperationException(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java new file mode 100644 index 000000000000..c159f1b4cd27 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java @@ -0,0 +1,92 @@ +/* + * 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.query.scan; + +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.YieldingAccumulator; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; + +import java.io.IOException; +import java.util.Map; + +public class ScanQueryNoLimitRowIterator implements CloseableIterator +{ + private Yielder yielder; + private ScanQuery.ResultFormat resultFormat; + + public ScanQueryNoLimitRowIterator( + QueryRunner baseRunner, + QueryPlus queryPlus, + Map responseContext + ) + { + ScanQuery query = Druids.ScanQueryBuilder.copy((ScanQuery) queryPlus.getQuery()).limit(Long.MAX_VALUE).timeOrder( + ScanQuery.TimeOrder.NONE).build(); + resultFormat = query.getResultFormat(); + queryPlus = queryPlus.withQuery(query); + Sequence baseSequence = baseRunner.run(queryPlus, responseContext); + yielder = baseSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) + { + yield(); + return in; + } + } + ); + } + + @Override + public boolean hasNext() + { + return !yielder.isDone(); + } + + @Override + public ScanResultValue next() + { + ScanResultValue batch = yielder.get(); + if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || + ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { + yielder = yielder.next(null); + return batch; + } + throw new UnsupportedOperationException(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); + } + + @Override + public void remove() + { + throw new UnsupportedOperationException(); + } + + @Override + public void close() throws IOException + { + yielder.close(); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 8d9ae742a854..5e72f547ae8e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -39,9 +39,11 @@ import org.apache.druid.query.aggregation.MetricManipulationFn; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; +import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -74,29 +76,42 @@ public QueryRunner mergeResults(final QueryRunner queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - final BaseSequence.IteratorMaker scanQueryLimitRowIteratorMaker = - new BaseSequence.IteratorMaker() - { - @Override - public ScanQueryLimitRowIterator make() - { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } - - @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }; - if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { if (scanQuery.getLimit() == Long.MAX_VALUE) { return runner.run(queryPlusWithNonNullLegacy, responseContext); + } else { + return new BaseSequence<>( + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() + { + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } + + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); } - return new BaseSequence(scanQueryLimitRowIteratorMaker); } else if (scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { - Iterator scanResultIterator = scanQueryLimitRowIteratorMaker.make(); + ScanQueryNoLimitRowIterator scanResultIterator = + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryNoLimitRowIterator make() + { + return new ScanQueryNoLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } + + @Override + public void cleanup(ScanQueryNoLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }.make(); return new BaseSequence( new BaseSequence.IteratorMaker() @@ -105,7 +120,7 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) public ScanBatchedIterator make() { return new ScanBatchedIterator( - sortScanResultValues(scanResultIterator, scanQuery), + sortAndLimitScanResultValues(scanResultIterator, scanQuery), scanQuery.getBatchSize() ); } @@ -167,14 +182,14 @@ public Sequence run(QueryPlus queryPlus, Map sortScanResultValues(Iterator inputIterator, ScanQuery scanQuery) + Iterator sortAndLimitScanResultValues(Iterator inputIterator, ScanQuery scanQuery) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - - PriorityQueue q = new PriorityQueue<>(Math.toIntExact(scanQuery.getLimit()), priorityQComparator); + int limit = Math.toIntExact(scanQuery.getLimit()); + PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator); while (inputIterator.hasNext()) { ScanResultValue next = inputIterator.next(); @@ -183,14 +198,19 @@ Iterator sortScanResultValues(Iterator inputIt // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list // needs to be preserved for queries using the compactedList result format q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); + if (q.size() > limit) { + q.poll(); + } } } // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order // will be maintained - List sortedElements = new ArrayList<>(q.size()); + final Deque sortedElements = new ArrayDeque<>(q.size()); while (q.size() != 0) { - sortedElements.add(q.poll()); + // We add at the front of the list because poll removes the tail of the queue. + sortedElements.addFirst(q.poll()); } + return sortedElements.iterator(); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java index a9a10fa366e0..b8a790cc41c3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java @@ -73,13 +73,14 @@ public Object getEvents() return events; } - public long getFirstEventTimestamp(ScanQuery query) { - if (query.getResultFormat().equals(ScanQuery.RESULT_FORMAT_LIST)) { + public long getFirstEventTimestamp(ScanQuery query) + { + if (query.getResultFormat().equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { return (Long) ((Map) ((List) this.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME); - } else if (query.getResultFormat().equals(ScanQuery.RESULT_FORMAT_COMPACTED_LIST)) { + } else if (query.getResultFormat().equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { int timeColumnIndex = this.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); List firstEvent = (List) ((List) this.getEvents()).get(0); - return (Long)firstEvent.get(timeColumnIndex); + return (Long) firstEvent.get(timeColumnIndex); } throw new UOE("Unable to get first event timestamp using result format of [%s]", query.getResultFormat()); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 66abbe612b90..5ee1672b5704 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -47,8 +47,8 @@ public int compare(ScanResultValue o1, ScanResultValue o2) o1.getFirstEventTimestamp(scanQuery), o2.getFirstEventTimestamp(scanQuery)); if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { - return comparison * -1; + return comparison; } - return comparison; + return comparison * -1; } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 137855de509a..489cfc8ec22f 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -91,13 +91,13 @@ public void testDescendingHeapsortListScanResultValues() ); } ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat("list") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .timeOrder(ScanQuery.TimeOrder.DESCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); int count = 0; Long previousTime = Long.MAX_VALUE; @@ -128,13 +128,13 @@ public void testAscendingHeapsortListScanResultValues() ); } ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat("list") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .timeOrder(ScanQuery.TimeOrder.ASCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); int count = 0; Long previousTime = -1L; @@ -163,13 +163,13 @@ public void testDescendingHeapsortCompactedListScanResultValues() ); } ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .timeOrder(ScanQuery.TimeOrder.DESCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); Long previousTime = Long.MAX_VALUE; int count = 0; @@ -198,13 +198,13 @@ public void testAscendingHeapsortCompactedListScanResultValues() ); } ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .timeOrder(ScanQuery.TimeOrder.ASCENDING) .dataSource("some data source") .intervals(emptySegmentSpec) .limit(99999) .build(); - Iterator sorted = chest.sortScanResultValues(inputs.iterator(), scanQuery); + Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); Long previousTime = -1L; int count = 0; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index 750b76ea06cd..ae3c2e842277 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -216,7 +216,7 @@ public void testFullOnSelectAsCompactedList() ScanQuery query = newTestQuery() .intervals(I_0112_0114) .virtualColumns(EXPR_COLUMN) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); HashMap context = new HashMap(); @@ -322,7 +322,7 @@ public void testSelectWithDimsAndMetsAsCompactedList() ScanQuery query = newTestQuery() .intervals(I_0112_0114) .columns(QueryRunnerTestHelper.marketDimension, QueryRunnerTestHelper.indexMetric) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .build(); HashMap context = new HashMap(); @@ -524,7 +524,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder("ascending") + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .build(); HashMap context = new HashMap<>(); @@ -582,7 +582,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder("descending") + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .build(); HashMap context = new HashMap<>(); @@ -664,8 +664,8 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList .intervals(I_0112_0114) .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) - .resultFormat("compactedList") - .timeOrder("ascending") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .timeOrder(ScanQuery.TimeOrder.ASCENDING) .limit(limit) .build(); @@ -725,8 +725,8 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis .intervals(I_0112_0114) .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) - .resultFormat("compactedList") - .timeOrder("descending") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) .limit(limit) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 9cfbade42224..b4cf5631c487 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -65,10 +65,10 @@ public void testSerializationLegacyString() throws Exception new TableDataSource(QueryRunnerTestHelper.dataSource), new LegacySegmentSpec(Intervals.of("2011-01-12/2011-01-14")), VirtualColumns.EMPTY, - null, + ScanQuery.ResultFormat.RESULT_FORMAT_LIST, 0, 3, - "none", + ScanQuery.TimeOrder.NONE, null, Arrays.asList("market", "quality", "index"), null, diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 5e55e8948584..293462217f2b 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -53,7 +53,7 @@ public void comparisonDescendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() .timeOrder(ScanQuery.TimeOrder.DESCENDING) - .resultFormat(ScanQuery.RESULT_FORMAT_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) .build(); @@ -82,7 +82,7 @@ public void comparisonDescendingListTest() events2 ); - Assert.assertEquals(1, comparator.compare(s1, s2)); + Assert.assertEquals(-1, comparator.compare(s1, s2)); } @Test @@ -90,7 +90,7 @@ public void comparisonAscendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() .timeOrder(ScanQuery.TimeOrder.ASCENDING) - .resultFormat(ScanQuery.RESULT_FORMAT_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) .build(); @@ -119,7 +119,7 @@ public void comparisonAscendingListTest() events2 ); - Assert.assertEquals(-1, comparator.compare(s1, s2)); + Assert.assertEquals(1, comparator.compare(s1, s2)); } @Test @@ -127,7 +127,7 @@ public void comparisonDescendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() .timeOrder(ScanQuery.TimeOrder.DESCENDING) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) .build(); @@ -154,7 +154,7 @@ public void comparisonDescendingCompactedListTest() events2 ); - Assert.assertEquals(1, comparator.compare(s1, s2)); + Assert.assertEquals(-1, comparator.compare(s1, s2)); } @Test @@ -162,7 +162,7 @@ public void comparisonAscendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() .timeOrder(ScanQuery.TimeOrder.ASCENDING) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) .build(); @@ -189,6 +189,6 @@ public void comparisonAscendingCompactedListTest() events2 ); - Assert.assertEquals(-1, comparator.compare(s1, s2)); + Assert.assertEquals(1, comparator.compare(s1, s2)); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 6cb4eafa4127..778c44527a5c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -961,7 +961,7 @@ public ScanQuery toScanQuery() dataSource, filtration.getQuerySegmentSpec(), selectProjection != null ? VirtualColumns.create(selectProjection.getVirtualColumns()) : VirtualColumns.EMPTY, - ScanQuery.RESULT_FORMAT_COMPACTED_LIST, + ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST, 0, scanLimit, null, // Will default to "none" diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index e963517ec1ae..a40b424e699f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -370,7 +370,7 @@ public static ExpressionPostAggregator EXPRESSION_POST_AGG(final String name, fi public static Druids.ScanQueryBuilder newScanQueryBuilder() { - return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + return new Druids.ScanQueryBuilder().resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false); } 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 7ffb44965e3a..d131cd361f33 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 @@ -111,7 +111,7 @@ public void testSelectConstantExpressionFromTable() throws Exception .intervals(QSS(Filtration.eternity())) .virtualColumns(EXPRESSION_VIRTUAL_COLUMN("v0", "2", ValueType.LONG)) .columns("dim1", "v0") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -431,7 +431,7 @@ public void testSelectStar() throws Exception .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -465,7 +465,7 @@ public void testSelectStarOnForbiddenTable() throws Exception .dataSource(CalciteTests.FORBIDDEN_DATASOURCE) .intervals(QSS(Filtration.eternity())) .columns("__time", "cnt", "dim1", "dim2", "m1", "m2", "unique_dim1") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -531,7 +531,7 @@ public void testSelectStarWithLimit() throws Exception .intervals(QSS(Filtration.eternity())) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") .limit(2) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -556,7 +556,7 @@ public void testSelectWithProjection() throws Exception ) .columns("v0") .limit(2) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -652,7 +652,7 @@ public void testSelectSingleColumnTwice() throws Exception .intervals(QSS(Filtration.eternity())) .columns("dim2") .limit(2) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -744,14 +744,14 @@ public void testSelfJoinWithFallback() throws Exception .intervals(QSS(Filtration.eternity())) .columns("dim1") .filters(NOT(SELECTOR("dim1", "", null))) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build(), newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(QSS(Filtration.eternity())) .columns("dim1", "dim2") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -1879,7 +1879,7 @@ public void testSelectStarWithDimFilter() throws Exception ) ) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -6715,7 +6715,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() throws Exception ) )) .columns("__time", "cnt", "dim1", "dim2") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7270,7 +7270,7 @@ public void testConcat() throws Exception ValueType.STRING )) .columns("v0") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7296,7 +7296,7 @@ public void testConcat() throws Exception ValueType.STRING )) .columns("v0") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7322,7 +7322,7 @@ public void testTextcat() throws Exception .intervals(QSS(Filtration.eternity())) .virtualColumns(EXPRESSION_VIRTUAL_COLUMN("v0", "concat(\"dim1\",\"dim1\")", ValueType.STRING)) .columns("v0") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7348,7 +7348,7 @@ public void testTextcat() throws Exception ValueType.STRING )) .columns("v0") - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7544,7 +7544,7 @@ public void testFilterFloatDimension() throws Exception .intervals(QSS(Filtration.eternity())) .columns("dim1") .filters(SELECTOR("f1", "0.1", null)) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -7566,7 +7566,7 @@ public void testFilterDoubleDimension() throws Exception .intervals(QSS(Filtration.eternity())) .columns("dim1") .filters(SELECTOR("d1", "1.7", null)) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -7588,7 +7588,7 @@ public void testFilterLongDimension() throws Exception .intervals(QSS(Filtration.eternity())) .columns("dim1") .filters(SELECTOR("l1", "7", null)) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) .build() From cd489a0208b0cfc475a34440caa1c1e99d22a281 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 20 Feb 2019 00:16:48 -0800 Subject: [PATCH 41/91] Fixed failing test due to null resultFormat --- .../src/main/java/org/apache/druid/query/scan/ScanQuery.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 57ad220afcbb..9ff77bfeadcb 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -132,7 +132,7 @@ public ScanQuery( { super(dataSource, querySegmentSpec, false, context); this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); - this.resultFormat = resultFormat; + this.resultFormat = resultFormat == null ? ResultFormat.RESULT_FORMAT_LIST : resultFormat; this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize; this.limit = (limit == 0) ? Long.MAX_VALUE : limit; Preconditions.checkArgument(this.batchSize > 0, "batchSize must be greater than 0"); From fba6b022f0395cc297e3b3726f817c986f97010b Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Feb 2019 15:08:27 -0800 Subject: [PATCH 42/91] Added config and get # of segments --- .../apache/druid/query/scan/ScanQueryConfig.java | 14 +++++++++++--- .../druid/query/scan/ScanQueryQueryToolChest.java | 4 ++-- .../druid/query/scan/ScanQueryRunnerFactory.java | 6 ++++++ .../query/scan/ScanQueryQueryToolChestTest.java | 2 +- 4 files changed, 20 insertions(+), 6 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index d2ec4ba5b29e..1b68c778ef87 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,11 +41,19 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsTimeOrderedInMemory = 100000; + private int maxRowsQueuedForTimeOrdering = 100000; - public int getMaxRowsTimeOrderedInMemory() + public int getMaxRowsQueuedForTimeOrdering() { - return maxRowsTimeOrderedInMemory; + return maxRowsQueuedForTimeOrdering; + } + + @JsonProperty + private int maxSegmentsTimeOrderedInMemory = 50; + + public int getMaxSegmentsTimeOrderedInMemory() + { + return maxSegmentsTimeOrderedInMemory; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 5e72f547ae8e..bef118c47840 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -96,7 +96,7 @@ public void cleanup(ScanQueryLimitRowIterator iterFromMake) } }); } - } else if (scanQuery.getLimit() <= scanQueryConfig.getMaxRowsTimeOrderedInMemory()) { + } else if (scanQuery.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { ScanQueryNoLimitRowIterator scanResultIterator = new BaseSequence.IteratorMaker() { @@ -136,7 +136,7 @@ public void cleanup(ScanBatchedIterator iterFromMake) "Time ordering for result set limit of %,d is not supported. Try lowering the " + "result set size to less than or equal to the time ordering limit of %,d.", scanQuery.getLimit(), - scanQueryConfig.getMaxRowsTimeOrderedInMemory() + scanQueryConfig.getMaxRowsQueuedForTimeOrdering() ); } }; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 73d9f680239c..840906c31c59 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -33,6 +33,7 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.segment.Segment; +import java.util.Iterator; import java.util.Map; import java.util.concurrent.ExecutorService; @@ -75,6 +76,11 @@ public Sequence run( final QueryPlus queryPlus, final Map responseContext ) { + int numSegments = 0; + final Iterator> segmentIt = queryRunners.iterator(); + for (; segmentIt.hasNext(); numSegments++) { + segmentIt.next(); + } // Note: this variable is effective only when queryContext has a timeout. // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 489cfc8ec22f..66f99877b8c8 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -52,7 +52,7 @@ public class ScanQueryQueryToolChestTest public static void setup() { config = createNiceMock(ScanQueryConfig.class); - expect(config.getMaxRowsTimeOrderedInMemory()).andReturn(100000); + expect(config.getMaxRowsQueuedForTimeOrdering()).andReturn(100000); replay(config); chest = new ScanQueryQueryToolChest(config, null); numElements = 1000; From b13ff624a92a7e740eb1f74aa40c7a72165b9708 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Feb 2019 15:13:33 -0800 Subject: [PATCH 43/91] Set up time ordering strategy decision tree --- .../query/scan/ScanQueryRunnerFactory.java | 65 ++++++++++--------- 1 file changed, 36 insertions(+), 29 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 840906c31c59..45fad93d9093 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -23,6 +23,7 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.Query; @@ -45,15 +46,18 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory mergeRunners( ) { // in single thread and in jetty thread instead of processing thread - return new QueryRunner() - { - @Override - public Sequence run( - final QueryPlus queryPlus, final Map responseContext - ) - { - int numSegments = 0; - final Iterator> segmentIt = queryRunners.iterator(); - for (; segmentIt.hasNext(); numSegments++) { - segmentIt.next(); - } - // Note: this variable is effective only when queryContext has a timeout. - // See the comment of CTX_TIMEOUT_AT. - final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); - responseContext.put(CTX_TIMEOUT_AT, timeoutAt); - return Sequences.concat( - Sequences.map( - Sequences.simple(queryRunners), - new Function, Sequence>() - { - @Override - public Sequence apply(final QueryRunner input) - { - return input.run(queryPlus, responseContext); - } - } - ) + return (queryPlus, responseContext) -> { + ScanQuery query = (ScanQuery) queryPlus.getQuery(); + int numSegments = 0; + final Iterator> segmentIt = queryRunners.iterator(); + for (; segmentIt.hasNext(); numSegments++) { + segmentIt.next(); + } + // Note: this variable is effective only when queryContext has a timeout. + // See the comment of CTX_TIMEOUT_AT. + final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); + responseContext.put(CTX_TIMEOUT_AT, timeoutAt); + if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { + // Use existing strategy + } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { + // Use priority queue strategy + } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { + // Use flatMerge strategy + } else { + throw new UOE( + "Time ordering for result set limit of %,d is not supported. Try lowering the " + + "result set size to less than or equal to the time ordering limit of %,d.", + query.getLimit(), + scanQueryConfig.getMaxRowsQueuedForTimeOrdering() ); } + + return Sequences.concat( + Sequences.map( + Sequences.simple(queryRunners), + input -> input.run(queryPlus, responseContext) + ) + ); + }; } From f83e99655d11247f44018e0e5d36bd6eac1fb2a6 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Feb 2019 16:56:36 -0800 Subject: [PATCH 44/91] Refactor and pQueue works --- .../druid/query/scan/ScanQueryEngine.java | 6 +- .../query/scan/ScanQueryQueryToolChest.java | 59 +------ .../query/scan/ScanQueryRunnerFactory.java | 162 +++++++++++++++++- .../druid/query/scan/ScanResultValue.java | 13 ++ .../apache/druid/query/DoubleStorageTest.java | 3 +- .../query/scan/MultiSegmentScanQueryTest.java | 3 +- .../druid/query/scan/ScanQueryRunnerTest.java | 3 +- .../druid/sql/calcite/util/CalciteTests.java | 3 +- 8 files changed, 186 insertions(+), 66 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 12c38f0d48e1..43bd49f5a460 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -68,7 +68,7 @@ public Sequence process( if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) { long count = (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); - if (count >= query.getLimit()) { + if (count >= query.getLimit() && query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { return Sequences.empty(); } } @@ -123,7 +123,9 @@ public Sequence process( if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) { responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0L); } - final long limit = query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); + final long limit = query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) ? + query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) : + Long.MAX_VALUE; return Sequences.concat( adapter .makeCursors( diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index bef118c47840..15785f5fb598 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -76,69 +76,24 @@ public QueryRunner mergeResults(final QueryRunner queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { - if (scanQuery.getLimit() == Long.MAX_VALUE) { - return runner.run(queryPlusWithNonNullLegacy, responseContext); - } else { - return new BaseSequence<>( - new BaseSequence.IteratorMaker() - { - @Override - public ScanQueryLimitRowIterator make() - { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } - - @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); - } - } else if (scanQuery.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { - ScanQueryNoLimitRowIterator scanResultIterator = - new BaseSequence.IteratorMaker() - { - @Override - public ScanQueryNoLimitRowIterator make() - { - return new ScanQueryNoLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } - - @Override - public void cleanup(ScanQueryNoLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }.make(); - - return new BaseSequence( - new BaseSequence.IteratorMaker() + if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) && scanQuery.getLimit() != Long.MAX_VALUE) { + return new BaseSequence<>( + new BaseSequence.IteratorMaker() { @Override - public ScanBatchedIterator make() + public ScanQueryLimitRowIterator make() { - return new ScanBatchedIterator( - sortAndLimitScanResultValues(scanResultIterator, scanQuery), - scanQuery.getBatchSize() - ); + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); } @Override - public void cleanup(ScanBatchedIterator iterFromMake) + public void cleanup(ScanQueryLimitRowIterator iterFromMake) { CloseQuietly.close(iterFromMake); } }); - } else { - throw new UOE( - "Time ordering for result set limit of %,d is not supported. Try lowering the " - + "result set size to less than or equal to the time ordering limit of %,d.", - scanQuery.getLimit(), - scanQueryConfig.getMaxRowsQueuedForTimeOrdering() - ); } + return runner.run(queryPlusWithNonNullLegacy, responseContext); }; } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 45fad93d9093..069a49c13a1b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -19,13 +19,22 @@ package org.apache.druid.query.scan; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.guava.BaseSequence; +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.Yielder; +import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -34,8 +43,15 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.segment.Segment; +import java.io.IOException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.Deque; import java.util.Iterator; +import java.util.List; import java.util.Map; +import java.util.PriorityQueue; import java.util.concurrent.ExecutorService; public class ScanQueryRunnerFactory implements QueryRunnerFactory @@ -85,11 +101,38 @@ public QueryRunner mergeRunners( final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { - // Use existing strategy + // Use normal strategy + return Sequences.concat( + Sequences.map( + Sequences.simple(queryRunners), + input -> input.run(queryPlus, responseContext) + ) + ); } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { // Use priority queue strategy + Sequence queryResults = Sequences.concat(Sequences.map( + Sequences.simple(queryRunners), + input -> input.run(queryPlus, responseContext) + )); + return sortBatchAndLimitScanResultValues(queryResults, query); } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { // Use flatMerge strategy + return Sequences.map( + Sequences.simple(queryRunners), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) + ) + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + query + )) + ).limit( + Math.toIntExact(query.getLimit()) + ); } else { throw new UOE( "Time ordering for result set limit of %,d is not supported. Try lowering the " @@ -98,15 +141,76 @@ public QueryRunner mergeRunners( scanQueryConfig.getMaxRowsQueuedForTimeOrdering() ); } + }; + } - return Sequences.concat( - Sequences.map( - Sequences.simple(queryRunners), - input -> input.run(queryPlus, responseContext) - ) - ); + @VisibleForTesting + Sequence sortBatchAndLimitScanResultValues( + Sequence inputSequence, + ScanQuery scanQuery + ) + { + Comparator priorityQComparator = + new ScanResultValueTimestampComparator(scanQuery); - }; + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch + // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) + int limit = Math.toIntExact(scanQuery.getLimit()); + PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator); + + Yielder yielder = inputSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) + { + yield(); + return in; + } + } + ); + while (!yielder.isDone()) { + ScanResultValue next = yielder.get(); + List singleEventScanResultValues = next.toSingleEventScanResultValues(); + for (ScanResultValue srv : singleEventScanResultValues) { + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(srv); + if (q.size() > limit) { + q.poll(); + } + } + yielder = yielder.next(null); + } + // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained + final Deque sortedElements = new ArrayDeque<>(q.size()); + while (q.size() != 0) { + // We add at the front of the list because poll removes the tail of the queue. + sortedElements.addFirst(q.poll()); + } + + // We can use an iterator here because all the results have been materialized for sorting + + return new BaseSequence( + new BaseSequence.IteratorMaker() + { + @Override + public ScanBatchedIterator make() + { + return new ScanBatchedIterator( + sortedElements.iterator(), + scanQuery.getBatchSize() + ); + } + + @Override + public void cleanup(ScanBatchedIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); } @Override @@ -142,4 +246,46 @@ public Sequence run(QueryPlus queryPlus, Map + { + private final Iterator itr; + private final int batchSize; + + public ScanBatchedIterator(Iterator iterator, int batchSize) + { + this.itr = iterator; + this.batchSize = batchSize; + } + + @Override + public void close() throws IOException + { + } + + @Override + public boolean hasNext() + { + return itr.hasNext(); + } + + @Override + public ScanResultValue next() + { + // Create new ScanResultValue from event map + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && itr.hasNext()) { + ScanResultValue srv = itr.next(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); + } + return new ScanResultValue(null, columns, eventsToAdd); + } + } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java index b8a790cc41c3..b32042b5b8b0 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java @@ -25,6 +25,8 @@ import org.apache.druid.segment.column.ColumnHolder; import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -85,6 +87,17 @@ public long getFirstEventTimestamp(ScanQuery query) throw new UOE("Unable to get first event timestamp using result format of [%s]", query.getResultFormat()); } + public List toSingleEventScanResultValues() + { + List singleEventScanResultValues = new ArrayList<>(); + List events = (List) this.getEvents(); + for (Object event : events) { + singleEventScanResultValues.add(new ScanResultValue(segmentId, columns, Collections.singletonList(event))); + } + return singleEventScanResultValues; + } + + @Override public boolean equals(Object o) { diff --git a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java index a85b41d61165..3c7d64053a39 100644 --- a/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java +++ b/processing/src/test/java/org/apache/druid/query/DoubleStorageTest.java @@ -94,7 +94,8 @@ public class DoubleStorageTest private static final ScanQueryRunnerFactory SCAN_QUERY_RUNNER_FACTORY = new ScanQueryRunnerFactory( scanQueryQueryToolChest, - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ); private Druids.ScanQueryBuilder newTestQuery() diff --git a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java index e138d73a1fe8..7b051b0fee52 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/MultiSegmentScanQueryTest.java @@ -73,7 +73,8 @@ public class MultiSegmentScanQueryTest private static final QueryRunnerFactory factory = new ScanQueryRunnerFactory( toolChest, - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ); // time modified version of druid.sample.numeric.tsv diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index ae3c2e842277..f3427dfac636 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -122,7 +122,8 @@ public static Iterable constructorFeeder() QueryRunnerTestHelper.makeQueryRunners( new ScanQueryRunnerFactory( toolChest, - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ) ), ImmutableList.of(false, true) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 9ca3233f20d9..d5ca8632a392 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -509,7 +509,8 @@ public int getNumMergeBuffers() new ScanQueryConfig(), new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) ), - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ) ) .put( From 1813a5472c791509ba903f734b40b6102079876a Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Feb 2019 17:06:18 -0800 Subject: [PATCH 45/91] Cleanup --- .../druid/query/scan/ScanQueryConfig.java | 2 +- .../scan/ScanQueryNoLimitRowIterator.java | 92 ------------------ .../query/scan/ScanQueryQueryToolChest.java | 93 ++----------------- .../query/scan/ScanQueryRunnerFactory.java | 2 +- 4 files changed, 8 insertions(+), 181 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 1b68c778ef87..4ceea78f679a 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,7 +41,7 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsQueuedForTimeOrdering = 100000; + private int maxRowsQueuedForTimeOrdering = 1; public int getMaxRowsQueuedForTimeOrdering() { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java deleted file mode 100644 index c159f1b4cd27..000000000000 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryNoLimitRowIterator.java +++ /dev/null @@ -1,92 +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.query.scan; - -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.YieldingAccumulator; -import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.query.Druids; -import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QueryRunner; - -import java.io.IOException; -import java.util.Map; - -public class ScanQueryNoLimitRowIterator implements CloseableIterator -{ - private Yielder yielder; - private ScanQuery.ResultFormat resultFormat; - - public ScanQueryNoLimitRowIterator( - QueryRunner baseRunner, - QueryPlus queryPlus, - Map responseContext - ) - { - ScanQuery query = Druids.ScanQueryBuilder.copy((ScanQuery) queryPlus.getQuery()).limit(Long.MAX_VALUE).timeOrder( - ScanQuery.TimeOrder.NONE).build(); - resultFormat = query.getResultFormat(); - queryPlus = queryPlus.withQuery(query); - Sequence baseSequence = baseRunner.run(queryPlus, responseContext); - yielder = baseSequence.toYielder( - null, - new YieldingAccumulator() - { - @Override - public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) - { - yield(); - return in; - } - } - ); - } - - @Override - public boolean hasNext() - { - return !yielder.isDone(); - } - - @Override - public ScanResultValue next() - { - ScanResultValue batch = yielder.get(); - if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || - ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { - yielder = yielder.next(null); - return batch; - } - throw new UnsupportedOperationException(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); - } - - @Override - public void remove() - { - throw new UnsupportedOperationException(); - } - - @Override - public void close() throws IOException - { - yielder.close(); - } -} diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 15785f5fb598..d4b0519c4947 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -121,94 +121,13 @@ public TypeReference getResultTypeReference() @Override public QueryRunner preMergeQueryDecoration(final QueryRunner runner) { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus queryPlus, Map responseContext) - { - ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); - if (scanQuery.getFilter() != null) { - scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); - queryPlus = queryPlus.withQuery(scanQuery); - } - return runner.run(queryPlus, responseContext); + return (queryPlus, responseContext) -> { + ScanQuery scanQuery = (ScanQuery) queryPlus.getQuery(); + if (scanQuery.getFilter() != null) { + scanQuery = scanQuery.withDimFilter(scanQuery.getFilter().optimize()); + queryPlus = queryPlus.withQuery(scanQuery); } + return runner.run(queryPlus, responseContext); }; } - - @VisibleForTesting - Iterator sortAndLimitScanResultValues(Iterator inputIterator, ScanQuery scanQuery) - { - Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); - - // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch - // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) - int limit = Math.toIntExact(scanQuery.getLimit()); - PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator); - - while (inputIterator.hasNext()) { - ScanResultValue next = inputIterator.next(); - List events = (List) next.getEvents(); - for (Object event : events) { - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(new ScanResultValue(null, next.getColumns(), Collections.singletonList(event))); - if (q.size() > limit) { - q.poll(); - } - } - } - // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order - // will be maintained - final Deque sortedElements = new ArrayDeque<>(q.size()); - while (q.size() != 0) { - // We add at the front of the list because poll removes the tail of the queue. - sortedElements.addFirst(q.poll()); - } - - return sortedElements.iterator(); - } - - /** - * This iterator supports iteration through any Iterable of unbatched ScanResultValues (1 event/ScanResultValue) and - * aggregates events into ScanResultValues with {@code batchSize} events. The columns from the first event per - * ScanResultValue will be used to populate the column section. - */ - private static class ScanBatchedIterator implements CloseableIterator - { - private final Iterator itr; - private final int batchSize; - - public ScanBatchedIterator(Iterator iterator, int batchSize) - { - this.itr = iterator; - this.batchSize = batchSize; - } - - @Override - public void close() throws IOException - { - } - - @Override - public boolean hasNext() - { - return itr.hasNext(); - } - - @Override - public ScanResultValue next() - { - // Create new ScanResultValue from event map - List eventsToAdd = new ArrayList<>(batchSize); - List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && itr.hasNext()) { - ScanResultValue srv = itr.next(); - // Only replace once using the columns from the first event - columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); - } - return new ScanResultValue(null, columns, eventsToAdd); - } - } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 069a49c13a1b..1811ff8efa4e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -116,7 +116,7 @@ public QueryRunner mergeRunners( )); return sortBatchAndLimitScanResultValues(queryResults, query); } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { - // Use flatMerge strategy + // Use n-way merge strategy return Sequences.map( Sequences.simple(queryRunners), (input) -> Sequences.concat( From f57ff253fa659cbb5aa09b7c9bf03d8e7670b865 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Feb 2019 18:22:06 -0800 Subject: [PATCH 46/91] Ordering is correct on n-way merge -> still need to batch events into ScanResultValues --- .../druid/query/scan/ScanQueryConfig.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 18 +++++++++++++++++- .../scan/ScanQueryQueryToolChestTest.java | 3 +++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 4ceea78f679a..1b68c778ef87 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,7 +41,7 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsQueuedForTimeOrdering = 1; + private int maxRowsQueuedForTimeOrdering = 100000; public int getMaxRowsQueuedForTimeOrdering() { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 1811ff8efa4e..b1f2b53a269d 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -116,6 +116,20 @@ public QueryRunner mergeRunners( )); return sortBatchAndLimitScanResultValues(queryResults, query); } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { + List> list = Sequences.map( + Sequences.simple(queryRunners), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) + ) + ).toList(); + + for(Sequence srv : list) { + List asdf = srv.toList(); + asdf.add(null); + } // Use n-way merge strategy return Sequences.map( Sequences.simple(queryRunners), @@ -129,10 +143,12 @@ public QueryRunner mergeRunners( seq -> seq, Ordering.from(new ScanResultValueTimestampComparator( query - )) + )).reverse() // TODO Figure out why this needs to be reversed ).limit( Math.toIntExact(query.getLimit()) ); + + // Need to batch } else { throw new UOE( "Time ordering for result set limit of %,d is not supported. Try lowering the " diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java index 66f99877b8c8..83b792c92c18 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java @@ -43,6 +43,7 @@ public class ScanQueryQueryToolChestTest { + /* private static ScanQueryQueryToolChest chest; private static ScanQueryConfig config; private static int numElements; @@ -51,6 +52,7 @@ public class ScanQueryQueryToolChestTest @BeforeClass public static void setup() { + config = createNiceMock(ScanQueryConfig.class); expect(config.getMaxRowsQueuedForTimeOrdering()).andReturn(100000); replay(config); @@ -218,4 +220,5 @@ public void testAscendingHeapsortCompactedListScanResultValues() } Assert.assertEquals(numElements, count); } + */ } From e1fc2955d361676eb6721ad31defd96d47fab999 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Feb 2019 10:39:59 -0800 Subject: [PATCH 47/91] WIP --- .../query/scan/ScanQueryRunnerFactory.java | 65 +++++++++---------- 1 file changed, 29 insertions(+), 36 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index b1f2b53a269d..15ceb16ed7ab 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -33,6 +33,8 @@ 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.YieldingSequenceBase; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -110,45 +112,34 @@ public QueryRunner mergeRunners( ); } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { // Use priority queue strategy - Sequence queryResults = Sequences.concat(Sequences.map( - Sequences.simple(queryRunners), - input -> input.run(queryPlus, responseContext) - )); - return sortBatchAndLimitScanResultValues(queryResults, query); + return sortBatchAndLimitScanResultValues( + Sequences.concat(Sequences.map( + Sequences.simple(queryRunners), + input -> input.run(queryPlus, responseContext) + )), + query + ); } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { - List> list = Sequences.map( - Sequences.simple(queryRunners), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) - ) - ) - ).toList(); - - for(Sequence srv : list) { - List asdf = srv.toList(); - asdf.add(null); - } // Use n-way merge strategy - return Sequences.map( - Sequences.simple(queryRunners), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + final Sequence unbatched = + Sequences.map( + Sequences.simple(queryRunners), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) ) - ) - ).flatMerge( - seq -> seq, - Ordering.from(new ScanResultValueTimestampComparator( - query - )).reverse() // TODO Figure out why this needs to be reversed - ).limit( - Math.toIntExact(query.getLimit()) - ); + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + query + )).reverse() // TODO Figure out why this needs to be reversed + ).limit( + Math.toIntExact(query.getLimit()) + ); - // Need to batch + // Batch the scan result values } else { throw new UOE( "Time ordering for result set limit of %,d is not supported. Try lowering the " @@ -262,12 +253,14 @@ public Sequence run(QueryPlus queryPlus, Map + @VisibleForTesting + static class ScanBatchedIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; From 023538d83117086647c69d5030f2e8cb3e039558 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Feb 2019 13:30:08 -0800 Subject: [PATCH 48/91] Sequence stuff is so dirty :( --- .../java/util/common/guava/MergeSequence.java | 15 +- .../query/scan/ScanQueryQueryToolChest.java | 16 --- .../query/scan/ScanQueryRunnerFactory.java | 128 +++++++++++++++--- 3 files changed, 113 insertions(+), 46 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java index 287fc62e289c..4b7c590be375 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java @@ -28,6 +28,7 @@ import java.util.PriorityQueue; /** + * Used to perform an n-way merge on n ordered sequences */ public class MergeSequence extends YieldingSequenceBase { @@ -43,20 +44,18 @@ public MergeSequence( this.baseSequences = (Sequence>) baseSequences; } + /* + Note: the yielder for MergeSequence returns elements from the priority queue in order of increasing priority. + This is due to the fact that PriorityQueue#remove() polls from the head of the queue which is, according to + the PriorityQueue javadoc, "the least element with respect to the specified ordering" + */ @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(); - } - } + (Function, T>) input -> input.get() ) ); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index d4b0519c4947..35420c4d4a6c 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -20,16 +20,11 @@ package org.apache.druid.query.scan; import com.fasterxml.jackson.core.type.TypeReference; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; -import com.google.common.collect.Iterables; import com.google.inject.Inject; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.BaseSequence; 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.parsers.CloseableIterator; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; @@ -38,17 +33,6 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.aggregation.MetricManipulationFn; -import java.io.IOException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Deque; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; - public class ScanQueryQueryToolChest extends QueryToolChest { private static final TypeReference TYPE_REFERENCE = new TypeReference() diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 15ceb16ed7ab..aa0a00d9d232 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -20,23 +20,24 @@ package org.apache.druid.query.scan; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; +import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.BaseSequence; 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.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.YieldingSequenceBase; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; -import org.apache.druid.query.Druids; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -45,11 +46,10 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.segment.Segment; +import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Comparator; -import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -134,20 +134,27 @@ public QueryRunner mergeRunners( seq -> seq, Ordering.from(new ScanResultValueTimestampComparator( query - )).reverse() // TODO Figure out why this needs to be reversed + )).reverse() // This needs to be reversed because ).limit( Math.toIntExact(query.getLimit()) ); // Batch the scan result values - } else { + return new ScanResultValueBatchingSequence(unbatched, query.getBatchSize()); + } else if (query.getLimit() > scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { throw new UOE( - "Time ordering for result set limit of %,d is not supported. Try lowering the " - + "result set size to less than or equal to the time ordering limit of %,d.", + "Time ordering for query result set limit of %,d is not supported. Try lowering the result " + + "set size to less than or equal to the configurable time ordering limit of %,d rows.", query.getLimit(), scanQueryConfig.getMaxRowsQueuedForTimeOrdering() ); } + throw new UOE( + "Time ordering for queries of %,d segments per historical is not supported. Try reducing the scope " + + "of the query to scan fewer segments than the configurable time ordering limit of %,d segments", + numSegments, + scanQueryConfig.getMaxSegmentsTimeOrderedInMemory() + ); }; } @@ -157,13 +164,14 @@ Sequence sortBatchAndLimitScanResultValues( ScanQuery scanQuery ) { - Comparator priorityQComparator = - new ScanResultValueTimestampComparator(scanQuery); + Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) int limit = Math.toIntExact(scanQuery.getLimit()); - PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator); + + // Comparator ordering is reversed since polling from the queue returns elements in reversed order + PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator.reversed()); Yielder yielder = inputSequence.toYielder( null, @@ -192,28 +200,25 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order // will be maintained - final Deque sortedElements = new ArrayDeque<>(q.size()); + final List sortedElements = new ArrayList<>(q.size()); while (q.size() != 0) { - // We add at the front of the list because poll removes the tail of the queue. - sortedElements.addFirst(q.poll()); + sortedElements.add(q.poll()); } - // We can use an iterator here because all the results have been materialized for sorting - return new BaseSequence( - new BaseSequence.IteratorMaker() + new BaseSequence.IteratorMaker() { @Override - public ScanBatchedIterator make() + public ScanBatchedIteratorIterator make() { - return new ScanBatchedIterator( + return new ScanBatchedIteratorIterator( sortedElements.iterator(), scanQuery.getBatchSize() ); } @Override - public void cleanup(ScanBatchedIterator iterFromMake) + public void cleanup(ScanBatchedIteratorIterator iterFromMake) { CloseQuietly.close(iterFromMake); } @@ -260,12 +265,12 @@ public Sequence run(QueryPlus queryPlus, Map + static class ScanBatchedIteratorIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; - public ScanBatchedIterator(Iterator iterator, int batchSize) + public ScanBatchedIteratorIterator(Iterator iterator, int batchSize) { this.itr = iterator; this.batchSize = batchSize; @@ -297,4 +302,83 @@ public ScanResultValue next() return new ScanResultValue(null, columns, eventsToAdd); } } + + @VisibleForTesting + static class ScanResultValueBatchingSequence extends YieldingSequenceBase + { + Yielder inputYielder; + int batchSize; + + public ScanResultValueBatchingSequence(Sequence inputSequence, int batchSize) { + this.inputYielder = inputSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) + { + yield(); + return in; + } + } + ); + this.batchSize = batchSize; + } + + @Override + @Nullable + public Yielder toYielder( + OutType initValue, YieldingAccumulator accumulator + ) + { + return makeYielder(initValue, accumulator); + } + + private Yielder makeYielder( + OutType initVal, + final YieldingAccumulator accumulator + ) + { + return new Yielder() + { + @Override + public OutType get() + { + // Create new ScanResultValue from event map + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && !inputYielder.isDone()) { + ScanResultValue srv = inputYielder.get(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); + inputYielder = inputYielder.next(null); + } + try { + return (OutType) new ScanResultValue(null, columns, eventsToAdd); + } catch (ClassCastException e) { + return initVal; + } + } + + @Override + public Yielder next(OutType initValue) + { + accumulator.reset(); + return makeYielder(initValue, accumulator); + } + + @Override + public boolean isDone() + { + return inputYielder.isDone(); + } + + @Override + public void close() + { + } + }; + } + } } From 3b923dac9cc82475795ee2f7691e6f96249560aa Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Feb 2019 14:03:22 -0800 Subject: [PATCH 49/91] Fixed bug introduced by replacing deque with list --- .../query/scan/ScanQueryRunnerFactory.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index aa0a00d9d232..e59239e7ed0a 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -20,23 +20,19 @@ package org.apache.druid.query.scan; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Throwables; import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.BaseSequence; 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.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.YieldingSequenceBase; -import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; @@ -48,8 +44,10 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Comparator; +import java.util.Deque; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -170,8 +168,7 @@ Sequence sortBatchAndLimitScanResultValues( // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) int limit = Math.toIntExact(scanQuery.getLimit()); - // Comparator ordering is reversed since polling from the queue returns elements in reversed order - PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator.reversed()); + PriorityQueue q = new PriorityQueue<>(limit, priorityQComparator); Yielder yielder = inputSequence.toYielder( null, @@ -198,27 +195,28 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } yielder = yielder.next(null); } - // Need to convert to a List because Priority Queue's iterator doesn't guarantee that the sorted order - // will be maintained - final List sortedElements = new ArrayList<>(q.size()); + // Need to convert to a Deque because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained. Deque was chosen over list because its addFirst is O(1). + final Deque sortedElements = new ArrayDeque<>(q.size()); while (q.size() != 0) { - sortedElements.add(q.poll()); + // addFirst is used since PriorityQueue#poll() dequeues the low-priority (timestamp-wise) events first. + sortedElements.addFirst(q.poll()); } return new BaseSequence( - new BaseSequence.IteratorMaker() + new BaseSequence.IteratorMaker() { @Override - public ScanBatchedIteratorIterator make() + public ScanBatchedIterator make() { - return new ScanBatchedIteratorIterator( + return new ScanBatchedIterator( sortedElements.iterator(), scanQuery.getBatchSize() ); } @Override - public void cleanup(ScanBatchedIteratorIterator iterFromMake) + public void cleanup(ScanBatchedIterator iterFromMake) { CloseQuietly.close(iterFromMake); } @@ -265,12 +263,12 @@ public Sequence run(QueryPlus queryPlus, Map + static class ScanBatchedIterator implements CloseableIterator { private final Iterator itr; private final int batchSize; - public ScanBatchedIteratorIterator(Iterator iterator, int batchSize) + public ScanBatchedIterator(Iterator iterator, int batchSize) { this.itr = iterator; this.batchSize = batchSize; From 06a5218917bca0716b98c32c07415a7271711431 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Feb 2019 16:59:57 -0800 Subject: [PATCH 50/91] Wrote docs --- .../druid/benchmark/query/ScanBenchmark.java | 3 +- docs/content/querying/scan-query.md | 42 +++- .../indexing/kafka/KafkaIndexTaskTest.java | 3 +- .../apache/druid/query/scan/ScanQuery.java | 7 + .../query/scan/ScanQueryRunnerFactory.java | 11 +- .../druid/query/scan/ScanResultValue.java | 8 +- .../ScanResultValueTimestampComparator.java | 4 +- .../druid/query/scan/ScanQueryConfigTest.java | 67 ++++++ .../scan/ScanQueryQueryToolChestTest.java | 224 ------------------ .../query/scan/ScanResultValueSerdeTest.java | 71 ------ .../druid/query/scan/ScanResultValueTest.java | 122 ++++++++++ .../druid/sql/calcite/rel/DruidQuery.java | 1 - 12 files changed, 243 insertions(+), 320 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java delete mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java delete mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTest.java diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 9d302a801791..6b250af21206 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -299,7 +299,8 @@ public void setup() throws IOException config, DefaultGenericQueryMetricsFactory.instance() ), - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ); } diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index a025ea5c4c69..7998f890eb6e 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -25,9 +25,9 @@ title: "Scan query" # Scan query The Scan query returns raw Druid rows in streaming mode. The biggest difference between the Select query and the Scan -query is that the Scan query does not retain all the returned rows in memory before they are returned to the client -(except when time-ordering is used). The Select query _will_ retain the rows in memory, causing memory pressure if too -many rows are returned. The Scan query can return all the rows without issuing another pagination query. +query is that the Scan query does not retain all the returned rows in memory before they are returned to the client. +The Select query _will_ retain the rows in memory, causing memory pressure if too many rows are returned. +The Scan query can return all the rows without issuing another pagination query. In addition to straightforward usage where a Scan query is issued to the Broker, the Scan query can also be issued directly to Historical processes or streaming ingestion tasks. This can be useful if you want to retrieve large @@ -155,12 +155,31 @@ The format of the result when resultFormat equals `compactedList`: ## Time Ordering -The Scan query currently supports ordering based on timestamp for non-legacy queries where the limit is less than -`druid.query.scan.maxRowsTimeOrderedInMemory` rows. The default value of `druid.query.scan.maxRowsTimeOrderedInMemory` -is 100000 rows. The reasoning behind this limit is that the current implementation of time ordering sorts all returned -records in memory. Attempting to load too many rows into memory runs the risk of Broker nodes running out of memory. -The limit can be configured based on server memory and number of dimensions being queried. - +The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering +will yield results that do not indicate which segment rows are from. Furthermore, time ordering is only supported +where the result set limit is less than `druid.query.scan.maxRowsQueuedForTimeOrdering` rows and less than +`druid.query.scan.maxSegmentsTimeOrderedInMemory` segments are scanned per Historical. The reasoning behind these +limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory +if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query +result set limit and the number of segments being scanned. + +1. Priority Queue: Each segment on a Historical is opened sequentially. Every row is added to a bounded priority +queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending) +or latest (if ascending) timestamp will be dequeued. After every row has been processed, the sorted contents of the +priority queue are streamed back to the Broker(s) in batches. Attempting to load too many rows into memory runs the +risk of Historical nodes running out of memory. The `druid.query.scan.maxRowsQueuedForTimeOrdering` property protects +from this by limiting the number of rows in the query result set when time ordering is used. + +2. K-Way/N-Way Merge: Each segment on a Historical is opened in parallel. Since each segment's rows are already +time-ordered, a k-way merge can be performed on the results from each segment. This approach doesn't persist the entire +result set in memory (like the Priority Queue) as it streams back batches as they are returned from the merge function. +However, attempting to query too many segments could also result in high memory usage due to the need to open +decompression and decoding buffers for each. The `druid.query.scan.maxSegmentsTimeOrderedInMemory` limit protects +from this by capping the number of segments opened per historical when time ordering is used. + +Both `druid.query.scan.maxRowsQueuedForTimeOrdering` and `druid.query.scan.maxSegmentsTimeOrderedInMemory` are +configurable and can be tuned based on hardware specs and number of dimensions being queried. + ## Legacy mode The Scan query supports a legacy mode designed for protocol compatibility with the former scan-query contrib extension. @@ -180,5 +199,6 @@ is complete. |property|description|values|default| |--------|-----------|------|-------| -|druid.query.scan.maxRowsTimeOrderedInMemory|An integer in the range [0, 2147483647]|100000| -|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| +|druid.query.scan.maxRowsQueuedForTimeOrdering|The maximum number of rows returned when time ordering is used|An integer in [0, 2147483647]|100000| +|druid.query.scan.maxSegmentsTimeOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [0, 2147483647]|50| +|druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| \ No newline at end of file diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java index 3d6308cb0a96..e8f8faaaccc9 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -2423,7 +2423,8 @@ public void registerQuery(Query query, ListenableFuture future) new ScanQueryConfig(), new DefaultGenericQueryMetricsFactory(TestHelper.makeJsonMapper()) ), - new ScanQueryEngine() + new ScanQueryEngine(), + new ScanQueryConfig() ) ) .build() diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 9ff77bfeadcb..89b3ffe68b16 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonValue; import com.google.common.base.Preconditions; +import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; @@ -207,6 +208,12 @@ public Boolean isLegacy() return legacy; } + @Override + public Ordering getResultOrdering() + { + return Ordering.from(new ScanResultValueTimestampComparator(this)); + } + public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig) { return Druids.ScanQueryBuilder.copy(this).legacy(legacy != null ? legacy : scanQueryConfig.isLegacy()).build(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index e59239e7ed0a..39323cddfce8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -137,7 +137,6 @@ public QueryRunner mergeRunners( Math.toIntExact(query.getLimit()) ); - // Batch the scan result values return new ScanResultValueBatchingSequence(unbatched, query.getBatchSize()); } else if (query.getLimit() > scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { throw new UOE( @@ -307,7 +306,8 @@ static class ScanResultValueBatchingSequence extends YieldingSequenceBase inputYielder; int batchSize; - public ScanResultValueBatchingSequence(Sequence inputSequence, int batchSize) { + public ScanResultValueBatchingSequence(Sequence inputSequence, int batchSize) + { this.inputYielder = inputSequence.toYielder( null, new YieldingAccumulator() @@ -324,9 +324,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } @Override - @Nullable public Yielder toYielder( - OutType initValue, YieldingAccumulator accumulator + OutType initValue, + YieldingAccumulator accumulator ) { return makeYielder(initValue, accumulator); @@ -354,7 +354,8 @@ public OutType get() } try { return (OutType) new ScanResultValue(null, columns, eventsToAdd); - } catch (ClassCastException e) { + } + catch (ClassCastException e) { return initVal; } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java index b32042b5b8b0..7bfcf025e1bc 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValue.java @@ -75,16 +75,16 @@ public Object getEvents() return events; } - public long getFirstEventTimestamp(ScanQuery query) + public long getFirstEventTimestamp(ScanQuery.ResultFormat resultFormat) { - if (query.getResultFormat().equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { + if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { return (Long) ((Map) ((List) this.getEvents()).get(0)).get(ColumnHolder.TIME_COLUMN_NAME); - } else if (query.getResultFormat().equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { + } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { int timeColumnIndex = this.getColumns().indexOf(ColumnHolder.TIME_COLUMN_NAME); List firstEvent = (List) ((List) this.getEvents()).get(0); return (Long) firstEvent.get(timeColumnIndex); } - throw new UOE("Unable to get first event timestamp using result format of [%s]", query.getResultFormat()); + throw new UOE("Unable to get first event timestamp using result format of [%s]", resultFormat.toString()); } public List toSingleEventScanResultValues() diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index 5ee1672b5704..f94020bf81be 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -44,8 +44,8 @@ public int compare(ScanResultValue o1, ScanResultValue o2) { int comparison; comparison = Longs.compare( - o1.getFirstEventTimestamp(scanQuery), - o2.getFirstEventTimestamp(scanQuery)); + o1.getFirstEventTimestamp(scanQuery.getResultFormat()), + o2.getFirstEventTimestamp(scanQuery.getResultFormat())); if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { return comparison; } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java new file mode 100644 index 000000000000..2bbc6e51f559 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java @@ -0,0 +1,67 @@ +/* + * 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.query.scan; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.junit.Assert; +import org.junit.Test; + +public class ScanQueryConfigTest +{ + private final ObjectMapper MAPPER = new DefaultObjectMapper(); + + private final ImmutableMap CONFIG_MAP = ImmutableMap + .builder() + .put("maxSegmentsTimeOrderedInMemory", "1") + .put("maxRowsQueuedForTimeOrdering", "1") + .put("legacy", "true") + .build(); + + private final ImmutableMap CONFIG_MAP2 = ImmutableMap + .builder() + .put("legacy", "false") + .put("maxSegmentsTimeOrderedInMemory", "42") + .build(); + + private final ImmutableMap CONFIG_MAP_EMPTY = ImmutableMap + .builder() + .build(); + + @Test + public void testSerde() + { + final ScanQueryConfig config = MAPPER.convertValue(CONFIG_MAP, ScanQueryConfig.class); + Assert.assertEquals(1, config.getMaxRowsQueuedForTimeOrdering()); + Assert.assertEquals(1, config.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertTrue(config.isLegacy()); + + final ScanQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, ScanQueryConfig.class); + Assert.assertEquals(100000, config2.getMaxRowsQueuedForTimeOrdering()); + Assert.assertEquals(42, config2.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertFalse(config2.isLegacy()); + + final ScanQueryConfig config3 = MAPPER.convertValue(CONFIG_MAP_EMPTY, ScanQueryConfig.class); + Assert.assertEquals(100000, config3.getMaxRowsQueuedForTimeOrdering()); + Assert.assertEquals(50, config3.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertFalse(config3.isLegacy()); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java deleted file mode 100644 index 83b792c92c18..000000000000 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryQueryToolChestTest.java +++ /dev/null @@ -1,224 +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.query.scan; - -import org.apache.druid.query.Druids; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QuerySegmentWalker; -import org.apache.druid.query.spec.QuerySegmentSpec; -import org.apache.druid.segment.column.ColumnHolder; -import org.joda.time.Interval; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -import static org.easymock.EasyMock.createNiceMock; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.replay; - -public class ScanQueryQueryToolChestTest -{ - /* - private static ScanQueryQueryToolChest chest; - private static ScanQueryConfig config; - private static int numElements; - private static QuerySegmentSpec emptySegmentSpec; - - @BeforeClass - public static void setup() - { - - config = createNiceMock(ScanQueryConfig.class); - expect(config.getMaxRowsQueuedForTimeOrdering()).andReturn(100000); - replay(config); - chest = new ScanQueryQueryToolChest(config, null); - numElements = 1000; - emptySegmentSpec = new QuerySegmentSpec() - { - @Override - public List getIntervals() - { - return null; - } - - @Override - public QueryRunner lookup( - Query query, - QuerySegmentWalker walker - ) - { - return null; - } - }; - } - - @Test - public void testDescendingHeapsortListScanResultValues() - { - List inputs = new ArrayList<>(); - for (long i = 0; i < numElements; i++) { - HashMap event = new HashMap<>(); - event.put("__time", i * 1000); - inputs.add( - new ScanResultValue( - "some segment id", - Collections.singletonList("__time"), - Collections.singletonList(event) - ) - ); - } - ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) - .dataSource("some data source") - .intervals(emptySegmentSpec) - .limit(99999) - .build(); - Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); - - int count = 0; - Long previousTime = Long.MAX_VALUE; - while (sorted.hasNext()) { - count++; - ScanResultValue curr = sorted.next(); - Long currentTime = (Long) - ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); - Assert.assertTrue(currentTime < previousTime); - previousTime = currentTime; - } - Assert.assertEquals(numElements, count); - } - - @Test - public void testAscendingHeapsortListScanResultValues() - { - List inputs = new ArrayList<>(); - for (long i = numElements; i > 0; i--) { - HashMap event = new HashMap<>(); - event.put("__time", i * 1000); - inputs.add( - new ScanResultValue( - "some segment id", - Collections.singletonList("__time"), - Collections.singletonList(event) - ) - ); - } - ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) - .timeOrder(ScanQuery.TimeOrder.ASCENDING) - .dataSource("some data source") - .intervals(emptySegmentSpec) - .limit(99999) - .build(); - Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); - - int count = 0; - Long previousTime = -1L; - while (sorted.hasNext()) { - count++; - ScanResultValue curr = sorted.next(); - Long currentTime = (Long) - ((Map) (((List) curr.getEvents()).get(0))).get(ColumnHolder.TIME_COLUMN_NAME); - Assert.assertTrue(currentTime > previousTime); - previousTime = currentTime; - } - Assert.assertEquals(numElements, count); - } - - @Test - public void testDescendingHeapsortCompactedListScanResultValues() - { - List inputs = new ArrayList<>(); - for (long i = 0; i < numElements; i++) { - inputs.add( - new ScanResultValue( - "some segment id", - Collections.singletonList("__time"), - Collections.singletonList(Collections.singletonList(new Long(i * 1000))) - ) - ); - } - ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) - .dataSource("some data source") - .intervals(emptySegmentSpec) - .limit(99999) - .build(); - Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); - - Long previousTime = Long.MAX_VALUE; - int count = 0; - while (sorted.hasNext()) { - count++; - ScanResultValue curr = sorted.next(); - Long currentTime = (Long) - ((List) (((List) curr.getEvents()).get(0))).get(0); - Assert.assertTrue(currentTime < previousTime); - previousTime = currentTime; - } - Assert.assertEquals(numElements, count); - } - - @Test - public void testAscendingHeapsortCompactedListScanResultValues() - { - List inputs = new ArrayList<>(); - for (long i = numElements; i > 0; i--) { - inputs.add( - new ScanResultValue( - "some segment id", - Collections.singletonList("__time"), - Collections.singletonList(Collections.singletonList(new Long(i * 1000))) - ) - ); - } - ScanQuery scanQuery = new Druids.ScanQueryBuilder() - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TimeOrder.ASCENDING) - .dataSource("some data source") - .intervals(emptySegmentSpec) - .limit(99999) - .build(); - Iterator sorted = chest.sortAndLimitScanResultValues(inputs.iterator(), scanQuery); - - Long previousTime = -1L; - int count = 0; - while (sorted.hasNext()) { - count++; - ScanResultValue curr = sorted.next(); - Long currentTime = (Long) - ((List) (((List) curr.getEvents()).get(0))).get(0); - Assert.assertTrue(currentTime > previousTime); - previousTime = currentTime; - } - Assert.assertEquals(numElements, count); - } - */ -} diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java deleted file mode 100644 index 572f1d8615c4..000000000000 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueSerdeTest.java +++ /dev/null @@ -1,71 +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.query.scan; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.junit.Assert; -import org.junit.Test; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public class ScanResultValueSerdeTest -{ - private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); - - @Test - public void testSerdeScanResultValueCompactedList() throws IOException - { - String segmentId = "some_segment_id"; - List columns = new ArrayList<>(Arrays.asList("col1", "col2", "col3")); - List event = new ArrayList<>(Arrays.asList( - "prop1", - "prop2", - "prop3" - )); - List> events = new ArrayList<>(Collections.singletonList(event)); - ScanResultValue srv = new ScanResultValue(segmentId, columns, events); - String serialized = jsonMapper.writeValueAsString(srv); - ScanResultValue deserialized = jsonMapper.readValue(serialized, ScanResultValue.class); - Assert.assertEquals(srv, deserialized); - } - - @Test - public void testSerdeScanResultValueNonCompactedList() throws IOException - { - String segmentId = "some_segment_id"; - List columns = new ArrayList<>(Arrays.asList("col1", "col2", "col3")); - Map event = new HashMap<>(); - event.put("key1", new Integer(4)); - event.put("key2", "some_string"); - event.put("key3", new Double(4.1)); - List> events = new ArrayList<>(Collections.singletonList(event)); - ScanResultValue srv = new ScanResultValue(segmentId, columns, events); - String serialized = jsonMapper.writeValueAsString(srv); - ScanResultValue deserialized = jsonMapper.readValue(serialized, ScanResultValue.class); - Assert.assertEquals(srv, deserialized); - } -} diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTest.java new file mode 100644 index 000000000000..47f82ad8ce59 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTest.java @@ -0,0 +1,122 @@ +/* + * 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.query.scan; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.column.ColumnHolder; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ScanResultValueTest +{ + private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); + private static final long TIME_1 = 1234567890000L; + private static final long TIME_2 = 9876543210000L; + + private static ScanResultValue compactedListSRV; + private static ScanResultValue listSRV; + + @BeforeClass + public static void setup() + { + String segmentId = "some_segment_id"; + List columns = new ArrayList<>(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "name", "count")); + List event = new ArrayList<>(Arrays.asList( + TIME_1, + "Feridun", + 4 + )); + List event2 = new ArrayList<>(Arrays.asList( + TIME_2, + "Justin", + 6 + )); + + List> events = Arrays.asList(event, event2); + compactedListSRV = new ScanResultValue(segmentId, columns, events); + + Map eventMap1 = new HashMap<>(); + eventMap1.put(ColumnHolder.TIME_COLUMN_NAME, TIME_1); + eventMap1.put("name", "Feridun"); + eventMap1.put("count", 4); + Map eventMap2 = new HashMap<>(); + eventMap2.put(ColumnHolder.TIME_COLUMN_NAME, TIME_2); + eventMap2.put("name", "Justin"); + eventMap2.put("count", 6); + List> eventMaps = Arrays.asList(eventMap1, eventMap2); + listSRV = new ScanResultValue(segmentId, columns, eventMaps); + } + + @Test + public void testSerdeScanResultValueCompactedList() throws IOException + { + + String serialized = JSON_MAPPER.writeValueAsString(compactedListSRV); + ScanResultValue deserialized = JSON_MAPPER.readValue(serialized, ScanResultValue.class); + Assert.assertEquals(compactedListSRV, deserialized); + } + + @Test + public void testSerdeScanResultValueNonCompactedList() throws IOException + { + + String serialized = JSON_MAPPER.writeValueAsString(listSRV); + ScanResultValue deserialized = JSON_MAPPER.readValue(serialized, ScanResultValue.class); + Assert.assertEquals(listSRV, deserialized); + } + + @Test + public void testGetFirstEventTimestampCompactedList() + { + long timestamp = compactedListSRV.getFirstEventTimestamp(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST); + Assert.assertEquals(TIME_1, timestamp); + } + + @Test + public void testGetFirstEventTimestampNonCompactedList() + { + long timestamp = listSRV.getFirstEventTimestamp(ScanQuery.ResultFormat.RESULT_FORMAT_LIST); + Assert.assertEquals(TIME_1, timestamp); + } + + @Test + public void testToSingleEventScanResultValues() + { + List compactedListScanResultValues = compactedListSRV.toSingleEventScanResultValues(); + for (ScanResultValue srv : compactedListScanResultValues) { + List events = (List) srv.getEvents(); + Assert.assertEquals(1, events.size()); + } + List listScanResultValues = listSRV.toSingleEventScanResultValues(); + for (ScanResultValue srv : compactedListScanResultValues) { + List events = (List) srv.getEvents(); + Assert.assertEquals(1, events.size()); + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index 778c44527a5c..a5fa0d4a2a44 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -969,7 +969,6 @@ public ScanQuery toScanQuery() Ordering.natural().sortedCopy(ImmutableSet.copyOf(outputRowSignature.getRowOrder())), false, ImmutableSortedMap.copyOf(plannerContext.getQueryContext()) - ); } From 763c43df7e99d4ab000f038a7c1b9ef98b479138 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Feb 2019 18:07:06 -0800 Subject: [PATCH 51/91] Multi-historical setup works --- .../java/util/common/guava/Sequence.java | 7 +++-- .../apache/druid/query/scan/ScanQuery.java | 2 +- .../query/scan/ScanQueryQueryToolChest.java | 29 +++++++++---------- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java index 8e04f9f4cee1..acd5a8b63eb1 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java @@ -54,13 +54,13 @@ public interface Sequence OutType accumulate(OutType initValue, Accumulator accumulator); /** - * Return an Yielder for accumulated sequence. + * Return a Yielder for accumulated sequence. * * @param initValue the initial value to pass along to start the accumulation. * @param accumulator the accumulator which is responsible for accumulating input values. * @param the type of accumulated value. * - * @return an Yielder for accumulated sequence. + * @return a Yielder for accumulated sequence. * * @see Yielder */ @@ -71,6 +71,9 @@ default Sequence map(Function mapper) return new MappedSequence<>(this, mapper); } + /** + * This will materialize the entire sequence + */ default List toList() { return accumulate(new ArrayList<>(), Accumulators.list()); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 89b3ffe68b16..9f73c79ac99b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -211,7 +211,7 @@ public Boolean isLegacy() @Override public Ordering getResultOrdering() { - return Ordering.from(new ScanResultValueTimestampComparator(this)); + return Ordering.from(new ScanResultValueTimestampComparator(this)).reverse(); } public ScanQuery withNonNullLegacy(final ScanQueryConfig scanQueryConfig) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 35420c4d4a6c..2cbe5667bc87 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -60,24 +60,21 @@ public QueryRunner mergeResults(final QueryRunner queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); - if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) && scanQuery.getLimit() != Long.MAX_VALUE) { - return new BaseSequence<>( - new BaseSequence.IteratorMaker() + return new BaseSequence<>( + new BaseSequence.IteratorMaker() + { + @Override + public ScanQueryLimitRowIterator make() { - @Override - public ScanQueryLimitRowIterator make() - { - return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); - } + return new ScanQueryLimitRowIterator(runner, queryPlusWithNonNullLegacy, responseContext); + } - @Override - public void cleanup(ScanQueryLimitRowIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); - } - return runner.run(queryPlusWithNonNullLegacy, responseContext); + @Override + public void cleanup(ScanQueryLimitRowIterator iterFromMake) + { + CloseQuietly.close(iterFromMake); + } + }); }; } From 451e2b43652020d6acb8b8db113fb34db0f50517 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Feb 2019 11:14:27 -0800 Subject: [PATCH 52/91] WIP --- .../query/scan/ScanQueryRunnerFactory.java | 11 +- .../scan/ScanQueryRunnerFactoryTest.java | 195 ++++++++++++++++++ 2 files changed, 204 insertions(+), 2 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 39323cddfce8..6a2b89fe34ee 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -42,7 +42,6 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.segment.Segment; -import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; @@ -287,7 +286,10 @@ public boolean hasNext() @Override public ScanResultValue next() { + ScanResultValue srv = itr.next(); + return srv; // Create new ScanResultValue from event map + /* List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); while (eventsToAdd.size() < batchSize && itr.hasNext()) { @@ -297,6 +299,7 @@ public ScanResultValue next() eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); } return new ScanResultValue(null, columns, eventsToAdd); + */ } } @@ -342,6 +345,10 @@ private Yielder makeYielder( @Override public OutType get() { + ScanResultValue srv = inputYielder.get(); + inputYielder = inputYielder.next(null); + return (OutType) srv; + /* // Create new ScanResultValue from event map List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); @@ -357,7 +364,7 @@ public OutType get() } catch (ClassCastException e) { return initVal; - } + }*/ } @Override diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java new file mode 100644 index 000000000000..e557de781cce --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -0,0 +1,195 @@ +/* + * 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.query.scan; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.segment.column.ColumnHolder; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; + + +@RunWith(Parameterized.class) +public class ScanQueryRunnerFactoryTest +{ + private int numElements; + private ScanQuery query; + private ScanQuery.ResultFormat resultFormat; + + private static final ScanQueryRunnerFactory factory = new ScanQueryRunnerFactory( + new ScanQueryQueryToolChest( + new ScanQueryConfig(), + DefaultGenericQueryMetricsFactory.instance() + ), + new ScanQueryEngine(), + new ScanQueryConfig() + ); + + public ScanQueryRunnerFactoryTest( + final int numElements, + final int batchSize, + final long limit, + final ScanQuery.ResultFormat resultFormat, + final ScanQuery.TimeOrder timeOrder + ) + { + this.numElements = numElements; + this.query = Druids.newScanQueryBuilder() + .batchSize(batchSize) + .limit(limit) + .timeOrder(timeOrder) + .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) + .dataSource("some datasource") + .build(); + this.resultFormat = resultFormat; + } + + @Parameterized.Parameters(name = "{0} {1} {2} {3} {4}") + public static Iterable constructorFeeder() + { + List numsElements = ImmutableList.of(0, 10, 100); + List batchSizes = ImmutableList.of(1, 100); + List limits = ImmutableList.of(3L, 1000L); + List resultFormats = ImmutableList.of( + ScanQuery.ResultFormat.RESULT_FORMAT_LIST, + ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST + ); + List timeOrder = ImmutableList.of( + ScanQuery.TimeOrder.ASCENDING, + ScanQuery.TimeOrder.DESCENDING + ); + + return QueryRunnerTestHelper.cartesian( + numsElements, + batchSizes, + limits, + resultFormats, + timeOrder + ); + } + + @Test + public void testSortBatchAndLimitScanResultValues() + { + List srvs = new ArrayList<>(numElements); + List expectedEventTimestamps = new ArrayList<>(); + for (int i = 0; i < numElements; i++) { + long timestamp = (long) (Math.random() * Long.MAX_VALUE); + expectedEventTimestamps.add(timestamp); + srvs.add(generateOneEventScanResultValue(timestamp, resultFormat)); + } + expectedEventTimestamps.sort((o1, o2) -> { + int retVal = 0; + if (o1 > o2) { + retVal = 1; + } else if (o1 < o2) { + retVal = -1; + } + if (query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { + return retVal * -1; + } + return retVal; + }); + Sequence inputSequence = Sequences.simple(srvs); + List output = + factory.sortBatchAndLimitScanResultValues( + inputSequence, + query + ).toList(); + + // check numBatches is as expected + int expectedNumBatches = (int) Math.ceil((double) numElements / query.getBatchSize()); + Assert.assertEquals(expectedNumBatches, output.size()); + + // check no batch has more than batchSize elements + for (ScanResultValue srv : output) { + if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { + Assert.assertTrue(getEventsCompactedListResultFormat(srv).size() <= query.getBatchSize()); + } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { + Assert.assertTrue(getEventsListResultFormat(srv).size() <= query.getBatchSize()); + } + } + + // check total # of rows <= limit + int numRows = 0; + for (ScanResultValue srv : output) { + if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { + numRows += getEventsCompactedListResultFormat(srv).size(); + } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { + numRows += getEventsListResultFormat(srv).size(); + } else { + throw new UOE("Invalid result format [%s] not supported", resultFormat.toString()); + } + } + Assert.assertTrue(numRows <= query.getLimit()); + + // check ordering and values are correct + + } + + private ScanResultValue generateOneEventScanResultValue(long timestamp, ScanQuery.ResultFormat resultFormat) + { + String segmentId = "some_segment_id"; + List columns = new ArrayList<>(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "name", "count")); + Object event; + if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { + Map eventMap = new HashMap<>(); + eventMap.put(ColumnHolder.TIME_COLUMN_NAME, timestamp); + eventMap.put("name", "Feridun"); + eventMap.put("count", 4); + event = eventMap; + } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { + event = new ArrayList<>(Arrays.asList( + timestamp, + "Feridun", + 4 + )); + } else { + throw new UOE("Result format [%s] not supported yet", resultFormat.toString()); + } + return new ScanResultValue(segmentId, columns, Collections.singletonList(event)); + } + + private List> getEventsListResultFormat(ScanResultValue scanResultValue) + { + return (List>) scanResultValue.getEvents(); + } + + private List> getEventsCompactedListResultFormat(ScanResultValue scanResultValue) + { + return (List>) scanResultValue.getEvents(); + } +} \ No newline at end of file From 18cce9a646139a57004ef4eccef8077c9775e992 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Feb 2019 13:16:44 -0800 Subject: [PATCH 53/91] Change so batching only occurs on broker for time-ordered scans Restricted batching to broker for time-ordered queries and adjusted tests Formatting Cleanup --- .../apache/druid/query/scan/ScanQuery.java | 6 + .../query/scan/ScanQueryLimitRowIterator.java | 39 +++- .../query/scan/ScanQueryQueryToolChest.java | 3 +- .../query/scan/ScanQueryRunnerFactory.java | 167 +----------------- .../scan/ScanQueryRunnerFactoryTest.java | 42 ++--- .../server/coordination/ServerManager.java | 2 +- 6 files changed, 63 insertions(+), 196 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 9f73c79ac99b..7a5cfcf578d2 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -107,6 +107,12 @@ public byte[] getCacheKey() } } + /** + * This context flag corresponds to whether the query is running on a Broker or Historical. Specifically, this is + * used to perform batching exclusively at the broker level for time-ordered scans. + */ + public static final String CTX_KEY_OUTERMOST = "scanOutermost"; + private final VirtualColumns virtualColumns; private final ResultFormat resultFormat; private final int batchSize; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 6fc0685e04ff..368e00d9cc31 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -19,6 +19,8 @@ package org.apache.druid.query.scan; +import com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.UOE; 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.YieldingAccumulator; @@ -27,15 +29,18 @@ import org.apache.druid.query.QueryRunner; import java.io.IOException; +import java.util.ArrayList; import java.util.List; import java.util.Map; public class ScanQueryLimitRowIterator implements CloseableIterator { + private static final String TIME_ORDERING_SEGMENT_ID = "No segment ID available when using time ordering"; private Yielder yielder; private ScanQuery.ResultFormat resultFormat; private long limit; private long count = 0; + private ScanQuery query; public ScanQueryLimitRowIterator( QueryRunner baseRunner, @@ -43,11 +48,11 @@ public ScanQueryLimitRowIterator( Map responseContext ) { - ScanQuery query = (ScanQuery) queryPlus.getQuery(); - resultFormat = query.getResultFormat(); - limit = query.getLimit(); + this.query = (ScanQuery) queryPlus.getQuery(); + this.resultFormat = query.getResultFormat(); + this.limit = query.getLimit(); Sequence baseSequence = baseRunner.run(queryPlus, responseContext); - yielder = baseSequence.toYielder( + this.yielder = baseSequence.toYielder( null, new YieldingAccumulator() { @@ -70,9 +75,14 @@ public boolean hasNext() @Override public ScanResultValue next() { - ScanResultValue batch = yielder.get(); - if (ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST.equals(resultFormat) || - ScanQuery.ResultFormat.RESULT_FORMAT_LIST.equals(resultFormat)) { + if (ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR.equals(resultFormat)) { + throw new UOE(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); + } + + // We don't want to perform batching at the historical-level if we're performing time ordering + if (query.getTimeOrder() == ScanQuery.TimeOrder.NONE || + !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { + ScanResultValue batch = yielder.get(); List events = (List) batch.getEvents(); if (events.size() <= limit - count) { count += events.size(); @@ -85,8 +95,21 @@ public ScanResultValue next() count = limit; return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, left)); } + } else { + // Perform single-event ScanResultValue batching. Each scan result value in this case will only have one event + // so there's no need to iterate through events. + int batchSize = query.getBatchSize(); + List eventsToAdd = new ArrayList<>(batchSize); + List columns = new ArrayList<>(); + while (eventsToAdd.size() < batchSize && !yielder.isDone()) { + ScanResultValue srv = yielder.get(); + // Only replace once using the columns from the first event + columns = columns.isEmpty() ? srv.getColumns() : columns; + eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); + yielder = yielder.next(null); + } + return new ScanResultValue(TIME_ORDERING_SEGMENT_ID, columns, eventsToAdd); } - throw new UnsupportedOperationException(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index 2cbe5667bc87..f313e2ed5d2c 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -58,7 +58,8 @@ public QueryRunner mergeResults(final QueryRunner { // Ensure "legacy" is a non-null value, such that all other nodes this query is forwarded to will treat it // the same way, even if they have different default legacy values. - final ScanQuery scanQuery = ((ScanQuery) queryPlus.getQuery()).withNonNullLegacy(scanQueryConfig); + final ScanQuery scanQuery = ((ScanQuery) (queryPlus.getQuery())) + .withNonNullLegacy(scanQueryConfig); final QueryPlus queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); return new BaseSequence<>( new BaseSequence.IteratorMaker() diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 6a2b89fe34ee..9daa9aab40f4 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -20,20 +20,15 @@ package org.apache.druid.query.scan; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.BaseSequence; -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.Yielder; import org.apache.druid.java.util.common.guava.YieldingAccumulator; -import org.apache.druid.java.util.common.guava.YieldingSequenceBase; -import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -42,9 +37,7 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.segment.Segment; -import java.io.IOException; import java.util.ArrayDeque; -import java.util.ArrayList; import java.util.Comparator; import java.util.Deque; import java.util.Iterator; @@ -99,6 +92,7 @@ public QueryRunner mergeRunners( // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); + queryPlus.getQuery().getContext().put(ScanQuery.CTX_KEY_OUTERMOST, false); if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { // Use normal strategy return Sequences.concat( @@ -109,7 +103,7 @@ public QueryRunner mergeRunners( ); } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { // Use priority queue strategy - return sortBatchAndLimitScanResultValues( + return sortAndLimitScanResultValues( Sequences.concat(Sequences.map( Sequences.simple(queryRunners), input -> input.run(queryPlus, responseContext) @@ -136,7 +130,7 @@ public QueryRunner mergeRunners( Math.toIntExact(query.getLimit()) ); - return new ScanResultValueBatchingSequence(unbatched, query.getBatchSize()); + return unbatched; } else if (query.getLimit() > scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { throw new UOE( "Time ordering for query result set limit of %,d is not supported. Try lowering the result " @@ -155,7 +149,7 @@ public QueryRunner mergeRunners( } @VisibleForTesting - Sequence sortBatchAndLimitScanResultValues( + Sequence sortAndLimitScanResultValues( Sequence inputSequence, ScanQuery scanQuery ) @@ -200,25 +194,7 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i // addFirst is used since PriorityQueue#poll() dequeues the low-priority (timestamp-wise) events first. sortedElements.addFirst(q.poll()); } - - return new BaseSequence( - new BaseSequence.IteratorMaker() - { - @Override - public ScanBatchedIterator make() - { - return new ScanBatchedIterator( - sortedElements.iterator(), - scanQuery.getBatchSize() - ); - } - - @Override - public void cleanup(ScanBatchedIterator iterFromMake) - { - CloseQuietly.close(iterFromMake); - } - }); + return Sequences.simple(sortedElements); } @Override @@ -254,137 +230,4 @@ public Sequence run(QueryPlus queryPlus, Map - { - private final Iterator itr; - private final int batchSize; - - public ScanBatchedIterator(Iterator iterator, int batchSize) - { - this.itr = iterator; - this.batchSize = batchSize; - } - - @Override - public void close() throws IOException - { - } - - @Override - public boolean hasNext() - { - return itr.hasNext(); - } - - @Override - public ScanResultValue next() - { - ScanResultValue srv = itr.next(); - return srv; - // Create new ScanResultValue from event map - /* - List eventsToAdd = new ArrayList<>(batchSize); - List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && itr.hasNext()) { - ScanResultValue srv = itr.next(); - // Only replace once using the columns from the first event - columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); - } - return new ScanResultValue(null, columns, eventsToAdd); - */ - } - } - - @VisibleForTesting - static class ScanResultValueBatchingSequence extends YieldingSequenceBase - { - Yielder inputYielder; - int batchSize; - - public ScanResultValueBatchingSequence(Sequence inputSequence, int batchSize) - { - this.inputYielder = inputSequence.toYielder( - null, - new YieldingAccumulator() - { - @Override - public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue in) - { - yield(); - return in; - } - } - ); - this.batchSize = batchSize; - } - - @Override - public Yielder toYielder( - OutType initValue, - YieldingAccumulator accumulator - ) - { - return makeYielder(initValue, accumulator); - } - - private Yielder makeYielder( - OutType initVal, - final YieldingAccumulator accumulator - ) - { - return new Yielder() - { - @Override - public OutType get() - { - ScanResultValue srv = inputYielder.get(); - inputYielder = inputYielder.next(null); - return (OutType) srv; - /* - // Create new ScanResultValue from event map - List eventsToAdd = new ArrayList<>(batchSize); - List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && !inputYielder.isDone()) { - ScanResultValue srv = inputYielder.get(); - // Only replace once using the columns from the first event - columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); - inputYielder = inputYielder.next(null); - } - try { - return (OutType) new ScanResultValue(null, columns, eventsToAdd); - } - catch (ClassCastException e) { - return initVal; - }*/ - } - - @Override - public Yielder next(OutType initValue) - { - accumulator.reset(); - return makeYielder(initValue, accumulator); - } - - @Override - public boolean isDone() - { - return inputYielder.isDone(); - } - - @Override - public void close() - { - } - }; - } - } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index e557de781cce..d7cf516fe51d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.java.util.common.UOE; -import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; @@ -38,8 +37,8 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.ListIterator; import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; @RunWith(Parameterized.class) @@ -73,6 +72,7 @@ public ScanQueryRunnerFactoryTest( .timeOrder(timeOrder) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .dataSource("some datasource") + .resultFormat(resultFormat) .build(); this.resultFormat = resultFormat; } @@ -102,12 +102,12 @@ public static Iterable constructorFeeder() } @Test - public void testSortBatchAndLimitScanResultValues() + public void testSortAndLimitScanResultValues() { List srvs = new ArrayList<>(numElements); List expectedEventTimestamps = new ArrayList<>(); for (int i = 0; i < numElements; i++) { - long timestamp = (long) (Math.random() * Long.MAX_VALUE); + long timestamp = (long) (ThreadLocalRandom.current().nextLong()); expectedEventTimestamps.add(timestamp); srvs.add(generateOneEventScanResultValue(timestamp, resultFormat)); } @@ -125,39 +125,33 @@ public void testSortBatchAndLimitScanResultValues() }); Sequence inputSequence = Sequences.simple(srvs); List output = - factory.sortBatchAndLimitScanResultValues( + factory.sortAndLimitScanResultValues( inputSequence, query ).toList(); - // check numBatches is as expected - int expectedNumBatches = (int) Math.ceil((double) numElements / query.getBatchSize()); - Assert.assertEquals(expectedNumBatches, output.size()); - - // check no batch has more than batchSize elements + // check each scan result value has one event for (ScanResultValue srv : output) { if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { - Assert.assertTrue(getEventsCompactedListResultFormat(srv).size() <= query.getBatchSize()); + Assert.assertTrue(getEventsCompactedListResultFormat(srv).size() == 1); } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { - Assert.assertTrue(getEventsListResultFormat(srv).size() <= query.getBatchSize()); + Assert.assertTrue(getEventsListResultFormat(srv).size() == 1); } } // check total # of rows <= limit - int numRows = 0; - for (ScanResultValue srv : output) { - if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { - numRows += getEventsCompactedListResultFormat(srv).size(); - } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { - numRows += getEventsListResultFormat(srv).size(); + Assert.assertTrue(output.size() <= query.getLimit()); + + // check ordering and values are correct + for (int i = 1; i < output.size(); i++) { + if (query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { + Assert.assertTrue(output.get(i).getFirstEventTimestamp(resultFormat) < + output.get(i - 1).getFirstEventTimestamp(resultFormat)); } else { - throw new UOE("Invalid result format [%s] not supported", resultFormat.toString()); + Assert.assertTrue(output.get(i).getFirstEventTimestamp(resultFormat) > + output.get(i - 1).getFirstEventTimestamp(resultFormat)); } } - Assert.assertTrue(numRows <= query.getLimit()); - - // check ordering and values are correct - } private ScanResultValue generateOneEventScanResultValue(long timestamp, ScanQuery.ResultFormat resultFormat) @@ -192,4 +186,4 @@ private List> getEventsCompactedListResultFormat(ScanResultValue sc { return (List>) scanResultValue.getEvents(); } -} \ No newline at end of file +} diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java index d628ef253d64..98e6f3e814a3 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerManager.java @@ -197,7 +197,7 @@ public QueryRunner apply(PartitionChunk input) ); return CPUTimeMetricQueryRunner.safeBuild( - new FinalizeResultsQueryRunner( + new FinalizeResultsQueryRunner<>( toolChest.mergeResults(factory.mergeRunners(exec, queryRunners)), toolChest ), From de83b11a1bb24a0ae964240d9cb1ed17ea4a6c26 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Feb 2019 16:40:48 -0800 Subject: [PATCH 54/91] Fixed mistakes in merge --- .../druid/sql/calcite/CalciteQueryTest.java | 81 ++++++------------- 1 file changed, 26 insertions(+), 55 deletions(-) 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 7e12553cb56a..841772a9ebdf 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 @@ -157,8 +157,7 @@ public void testSelectCountStart() throws Exception CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of(Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.of( - "2999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) + .intervals(querySegmentSpec(Intervals.of("2999-01-01T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) .granularity(Granularities.ALL) .aggregators(aggregators( new CountAggregatorFactory("a0"), @@ -309,7 +308,7 @@ public void testInformationSchemaTables() throws Exception .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE"}) .add(new Object[]{"sys", "servers", "SYSTEM_TABLE"}) .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE"}) - .build() + .build() ); } @@ -1715,8 +1714,8 @@ public void testEmptyStringEquality() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters(expressionFilter("case_searched((\"dim2\" == 'a')," - + (NullHandling.replaceWithDefault() ? "1" : "0") - + ",(\"dim2\" == ''))")) + + (NullHandling.replaceWithDefault() ? "1" : "0") + + ",(\"dim2\" == ''))")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) .build() @@ -1744,8 +1743,8 @@ public void testNullStringEquality() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters(expressionFilter("case_searched((\"dim2\" == 'a')," - + (NullHandling.replaceWithDefault() ? "1" : "0") - + ",(\"dim2\" == null))")) + + (NullHandling.replaceWithDefault() ? "1" : "0") + + ",(\"dim2\" == null))")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(TIMESERIES_CONTEXT_DEFAULT) .build() @@ -4284,11 +4283,7 @@ public void testMinMaxAvgDailyCountWithLimit() throws Exception ValueType.LONG ) ) - .setDimensions(dimensionSpec(new DefaultDimensionSpec( - "d0:v", - "d0", - ValueType.LONG - ))) + .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4347,21 +4342,13 @@ public void testAvgDailyCountDistinct() throws Exception ValueType.LONG ) ) - .setDimensions(dimensionSpec(new DefaultDimensionSpec( - "d0:v", - "d0", - ValueType.LONG - ))) + .setDimensions(dimensionSpec(new DefaultDimensionSpec("d0:v", "d0", ValueType.LONG))) .setAggregatorSpecs( aggregators( new CardinalityAggregatorFactory( "a0:a", null, - dimensionSpec(new DefaultDimensionSpec( - "cnt", - "cnt", - ValueType.LONG - )), + dimensionSpec(new DefaultDimensionSpec("cnt", "cnt", ValueType.LONG)), false, true ) @@ -7460,10 +7447,7 @@ public void testRequireTimeConditionPositive() throws Exception ImmutableList.of( GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Intervals.utc( - DateTimes.of("2000-01-01").getMillis(), - JodaUtils.MAX_INSTANT - ))) + .setInterval(querySegmentSpec(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT))) .setGranularity(Granularities.ALL) .setDimFilter(not(selector("dim1", "", null))) .setDimensions(dimensionSpec(new ExtractionDimensionSpec( @@ -7475,10 +7459,7 @@ public void testRequireTimeConditionPositive() throws Exception .build(), Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.utc( - DateTimes.of("2000-01-01").getMillis(), - JodaUtils.MAX_INSTANT - ))) + .intervals(querySegmentSpec(Intervals.utc(DateTimes.of("2000-01-01").getMillis(), JodaUtils.MAX_INSTANT))) .granularity(Granularities.ALL) .filters(in( "dim2", @@ -7562,20 +7543,15 @@ public void testFilterFloatDimension() throws Exception .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1") - << << << < HEAD - .filters(SELECTOR("f1", "0.1", null)) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - ======= .filters(selector("f1", "0.1", null)) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) - >>>>>>>master - .limit(1) - .context(QUERY_CONTEXT_DEFAULT) - .build() + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .limit(1) + .context(QUERY_CONTEXT_DEFAULT) + .build() ), - ImmutableList.of( - new Object[]{"10.1"} - ) + ImmutableList.of( + new Object[]{"10.1"} + ) ); } @@ -7589,20 +7565,15 @@ public void testFilterDoubleDimension() throws Exception .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1") - << << << < HEAD - .filters(SELECTOR("d1", "1.7", null)) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - ======= .filters(selector("d1", "1.7", null)) - .resultFormat(ScanQuery.RESULT_FORMAT_COMPACTED_LIST) - >>>>>>>master - .limit(1) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{"10.1"} - ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .limit(1) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"10.1"} + ) ); } From 806166f9777cccae5e10eabbb256c5e33b0e13f7 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 28 Feb 2019 15:49:07 -0800 Subject: [PATCH 55/91] Fixed failing tests --- .../druid/query/scan/ScanQueryConfig.java | 2 +- .../query/scan/ScanQueryLimitRowIterator.java | 9 ++++++-- .../query/scan/ScanQueryRunnerFactory.java | 1 - .../druid/query/scan/ScanQueryRunnerTest.java | 23 +++++++++++-------- 4 files changed, 21 insertions(+), 14 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 1b68c778ef87..1dd726faf479 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,7 +41,7 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsQueuedForTimeOrdering = 100000; + private int maxRowsQueuedForTimeOrdering = 10; public int getMaxRowsQueuedForTimeOrdering() { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 368e00d9cc31..4aad2f9afa69 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -19,12 +19,14 @@ package org.apache.druid.query.scan; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.UOE; 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.YieldingAccumulator; import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -51,7 +53,9 @@ public ScanQueryLimitRowIterator( this.query = (ScanQuery) queryPlus.getQuery(); this.resultFormat = query.getResultFormat(); this.limit = query.getLimit(); - Sequence baseSequence = baseRunner.run(queryPlus, responseContext); + Query historicalQuery = + queryPlus.getQuery().withOverriddenContext(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)); + Sequence baseSequence = baseRunner.run(QueryPlus.wrap(historicalQuery), responseContext); this.yielder = baseSequence.toYielder( null, new YieldingAccumulator() @@ -101,12 +105,13 @@ public ScanResultValue next() int batchSize = query.getBatchSize(); List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); - while (eventsToAdd.size() < batchSize && !yielder.isDone()) { + while (eventsToAdd.size() < batchSize && !yielder.isDone() && count < limit) { ScanResultValue srv = yielder.get(); // Only replace once using the columns from the first event columns = columns.isEmpty() ? srv.getColumns() : columns; eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); yielder = yielder.next(null); + count++; } return new ScanResultValue(TIME_ORDERING_SEGMENT_ID, columns, eventsToAdd); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 9daa9aab40f4..ebfca4865920 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -92,7 +92,6 @@ public QueryRunner mergeRunners( // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); - queryPlus.getQuery().getContext().put(ScanQuery.CTX_KEY_OUTERMOST, false); if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { // Use normal strategy return Sequences.concat( diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index f3427dfac636..70fa2033fadf 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -21,6 +21,7 @@ import com.google.common.base.Function; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.ObjectArrays; @@ -518,7 +519,7 @@ public void testFullSelectNoDimensionAndMetric() @Test public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() { - // limits + // limits shouldn't matter -> all rows should be returned if time-ordering on the broker is occurring for (int limit : new int[]{3, 1, 5, 7, 0}) { ScanQuery query = newTestQuery() .intervals(I_0112_0114) @@ -526,6 +527,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) .timeOrder(ScanQuery.TimeOrder.ASCENDING) + .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .build(); HashMap context = new HashMap<>(); @@ -567,7 +569,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() ascendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - limit + seg1Results.length + seg2Results.length ); verify(ascendingExpectedResults, results); } @@ -576,7 +578,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() @Test public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() { - // limits + // limits shouldn't matter -> all rows should be returned if time-ordering on the broker is occurring for (int limit : new int[]{3, 1, 5, 7, 0}) { ScanQuery query = newTestQuery() .intervals(I_0112_0114) @@ -627,7 +629,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() descendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - limit + seg1Results.length + seg2Results.length ); verify(descendingExpectedResults, results); } @@ -658,8 +660,8 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" }; - // limits - for (int limit : new int[]{3, 1, 5, 7, 0}) { + // limits shouldn't matter -> all rows should be returned if time-ordering on the broker is occurring + for (int limit : new int[]{3, 0}) { /* Ascending */ ScanQuery query = newTestQuery() .intervals(I_0112_0114) @@ -687,7 +689,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList ascendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - limit + seg1Results.length + seg2Results.length ); results = compactedListToRow(results); verify(ascendingExpectedResults, results); @@ -719,8 +721,8 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis "2011-01-13T00:00:00.000Z\tspot\ttechnology\tpreferred\ttpreferred\t111.356672", "2011-01-13T00:00:00.000Z\tspot\ttravel\tpreferred\ttpreferred\t106.236928" }; - // limits - for (int limit : new int[]{3, 1, 5, 7, 0}) { + // limits shouldn't matter -> all rows should be returned if time-ordering on the broker is occurring + for (int limit : new int[]{3, 1}) { /* Descending */ ScanQuery query = newTestQuery() .intervals(I_0112_0114) @@ -728,6 +730,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .limit(limit) .build(); @@ -750,7 +753,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis descendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - limit + expectedRet.length ); results = compactedListToRow(results); verify(descendingExpectedResults, results); From 5ff59f5ca6c8058c04e500662b3691a4910aa842 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 28 Feb 2019 15:58:20 -0800 Subject: [PATCH 56/91] Reset config --- .../main/java/org/apache/druid/query/scan/ScanQueryConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 1dd726faf479..1b68c778ef87 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,7 +41,7 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsQueuedForTimeOrdering = 10; + private int maxRowsQueuedForTimeOrdering = 100000; public int getMaxRowsQueuedForTimeOrdering() { From 47c970b5f476e5bfe5e03aa798f314f59aeb67db Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Mar 2019 13:38:29 -0800 Subject: [PATCH 57/91] Wrote tests and added Javadoc --- .../query/scan/ScanQueryLimitRowIterator.java | 21 +- .../scan/ScanQueryLimitRowIteratorTest.java | 211 ++++++++++++++++++ .../scan/ScanQueryRunnerFactoryTest.java | 45 +--- .../druid/query/scan/ScanQueryTestHelper.java | 71 ++++++ 4 files changed, 303 insertions(+), 45 deletions(-) create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 4aad2f9afa69..79c67031612e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -35,6 +35,21 @@ import java.util.List; import java.util.Map; + +/** + * This iterator supports iteration through a Sequence returned by a ScanResultValue QueryRunner. Its behaviour + * varies depending on whether the query is returning time-ordered values and whether the CTX_KEY_OUTERMOST flag is + * set as false. + * + * Behaviours: + * 1) No time ordering: expects a Sequence of ScanResultValues which each contain up to query.batchSize events. + * The iterator will be "done" when the limit of events is reached. The final ScanResultValue might contain + * fewer than batchSize events so that the limit number of events is returned. + * 2) Time Ordering, CTX_KEY_OUTERMOST==null or true: Same behaviour as no time ordering + * 3) Time Ordering, CTX_KEY_OUTERMOST=false: The Sequence returned in this case should contain ScanResultValues + * that contain only one event each. This iterator will perform batching according to query.batchSize until + * the limit is reached. + */ public class ScanQueryLimitRowIterator implements CloseableIterator { private static final String TIME_ORDERING_SEGMENT_ID = "No segment ID available when using time ordering"; @@ -83,7 +98,7 @@ public ScanResultValue next() throw new UOE(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); } - // We don't want to perform batching at the historical-level if we're performing time ordering + // We don't want to perform batching at the historical-level if we're time ordering if (query.getTimeOrder() == ScanQuery.TimeOrder.NONE || !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); @@ -95,9 +110,9 @@ public ScanResultValue next() } else { // last batch // single batch length is <= Integer.MAX_VALUE, so this should not overflow - int left = (int) (limit - count); + int numLeft = (int) (limit - count); count = limit; - return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, left)); + return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, numLeft)); } } else { // Perform single-event ScanResultValue batching. Each scan result value in this case will only have one event diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java new file mode 100644 index 000000000000..6609f532628e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java @@ -0,0 +1,211 @@ +/* + * 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.query.scan; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +@RunWith(Parameterized.class) +public class ScanQueryLimitRowIteratorTest +{ + private static final int NUM_ELEMENTS = 1000; + private static int batchSize; + private static int limit; + private static List singleEventScanResultValues = new ArrayList<>(); + private static List multiEventScanResultValues = new ArrayList<>(); + private static final ScanQuery.ResultFormat resultFormat = ScanQuery.ResultFormat.RESULT_FORMAT_LIST; + + public ScanQueryLimitRowIteratorTest( + final int batchSize, + final int limit + ) + { + this.batchSize = batchSize; + this.limit = limit; + } + + @Parameterized.Parameters(name = "{0} {1}") + public static Iterable constructorFeeder() + { + List batchSizes = ImmutableList.of(1, 33); + List limits = ImmutableList.of(3, 10000); + return QueryRunnerTestHelper.cartesian( + batchSizes, + limits + ); + } + + @Before + public void setup() + { + singleEventScanResultValues = new ArrayList<>(); + multiEventScanResultValues = new ArrayList<>(); + for (int i = 0; i < NUM_ELEMENTS; i++) { + singleEventScanResultValues.add( + ScanQueryTestHelper.generateScanResultValue( + ThreadLocalRandom.current().nextLong(), + resultFormat, + 1 + )); + } + for (int i = 0; i < NUM_ELEMENTS / batchSize; i++) { + multiEventScanResultValues.add( + ScanQueryTestHelper.generateScanResultValue( + ThreadLocalRandom.current().nextLong(), + resultFormat, + batchSize + )); + } + multiEventScanResultValues.add( + ScanQueryTestHelper.generateScanResultValue( + ThreadLocalRandom.current().nextLong(), + resultFormat, + NUM_ELEMENTS % batchSize + )); + } + + /** + * Expect no batching to occur and limit to be applied + */ + @Test + public void testNonTimeOrderedScan() + { + ScanQuery query = Druids.newScanQueryBuilder() + .limit(limit) + .timeOrder(ScanQuery.TimeOrder.NONE) + .dataSource("some datasource") + .batchSize(batchSize) + .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) + .resultFormat(resultFormat) + .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) + .build(); + QueryPlus queryPlus = QueryPlus.wrap(query); + ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator( + ((queryInput, responseContext) -> Sequences.simple(multiEventScanResultValues)), + queryPlus, + ImmutableMap.of() + ); + + int count = 0; + int expectedNumRows = Math.min(limit, NUM_ELEMENTS); + + while (itr.hasNext()) { + ScanResultValue curr = itr.next(); + List> events = ScanQueryTestHelper.getEventsListResultFormat(curr); + if (events.size() != batchSize) { + if (expectedNumRows - count > batchSize) { + Assert.fail("Batch size is incorrect"); + } else { + Assert.assertEquals(expectedNumRows - count, events.size()); + } + } + count += events.size(); + } + Assert.assertEquals(expectedNumRows, count); + } + + /** + * Expect batching to occur and limit to be applied on the Broker. Input from Historical + * is a sequence of single-event ScanResultValues. + */ + @Test + public void testBrokerTimeOrderedScan() + { + ScanQuery query = Druids.newScanQueryBuilder() + .limit(limit) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .dataSource("some datasource") + .batchSize(batchSize) + .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) + .resultFormat(resultFormat) + .build(); + QueryPlus queryPlus = QueryPlus.wrap(query); + ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator( + ((queryInput, responseContext) -> Sequences.simple(singleEventScanResultValues)), + queryPlus, + ImmutableMap.of() + ); + + int count = 0; + int expectedNumRows = Math.min(limit, NUM_ELEMENTS); + while (itr.hasNext()) { + ScanResultValue curr = itr.next(); + List> events = ScanQueryTestHelper.getEventsListResultFormat(curr); + if (events.size() != batchSize) { + if (expectedNumRows - count >= batchSize) { + Assert.fail("Batch size is incorrect"); + } else { + Assert.assertEquals(expectedNumRows - count, events.size()); + } + } + count += events.size(); + } + Assert.assertEquals(expectedNumRows, count); + } + + /** + * Expect no batching to occur and limit to be applied. Input is a sequence of sorted single-event ScanResultValues + * (unbatching and sorting occurs in ScanQueryRunnerFactory#mergeRunners()). + */ + @Test + public void testHistoricalTimeOrderedScan() + { + ScanQuery query = Druids.newScanQueryBuilder() + .limit(limit) + .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .dataSource("some datasource") + .batchSize(batchSize) + .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) + .resultFormat(resultFormat) + .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) + .build(); + + QueryPlus queryPlus = QueryPlus.wrap(query); + ScanQueryLimitRowIterator itr = new ScanQueryLimitRowIterator( + ((queryInput, responseContext) -> Sequences.simple(singleEventScanResultValues)), + queryPlus, + ImmutableMap.of() + ); + + int count = 0; + int expectedNumRows = Math.min(limit, NUM_ELEMENTS); + while (itr.hasNext()) { + ScanResultValue curr = itr.next(); + List> events = ScanQueryTestHelper.getEventsListResultFormat(curr); + Assert.assertEquals(1, events.size()); + count += events.size(); + } + Assert.assertEquals(expectedNumRows, count); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index d7cf516fe51d..a6d4f9de7979 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -20,24 +20,18 @@ package org.apache.druid.query.scan; import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.segment.column.ColumnHolder; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -109,7 +103,7 @@ public void testSortAndLimitScanResultValues() for (int i = 0; i < numElements; i++) { long timestamp = (long) (ThreadLocalRandom.current().nextLong()); expectedEventTimestamps.add(timestamp); - srvs.add(generateOneEventScanResultValue(timestamp, resultFormat)); + srvs.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); } expectedEventTimestamps.sort((o1, o2) -> { int retVal = 0; @@ -133,9 +127,9 @@ public void testSortAndLimitScanResultValues() // check each scan result value has one event for (ScanResultValue srv : output) { if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { - Assert.assertTrue(getEventsCompactedListResultFormat(srv).size() == 1); + Assert.assertTrue(ScanQueryTestHelper.getEventsCompactedListResultFormat(srv).size() == 1); } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { - Assert.assertTrue(getEventsListResultFormat(srv).size() == 1); + Assert.assertTrue(ScanQueryTestHelper.getEventsListResultFormat(srv).size() == 1); } } @@ -153,37 +147,4 @@ public void testSortAndLimitScanResultValues() } } } - - private ScanResultValue generateOneEventScanResultValue(long timestamp, ScanQuery.ResultFormat resultFormat) - { - String segmentId = "some_segment_id"; - List columns = new ArrayList<>(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "name", "count")); - Object event; - if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { - Map eventMap = new HashMap<>(); - eventMap.put(ColumnHolder.TIME_COLUMN_NAME, timestamp); - eventMap.put("name", "Feridun"); - eventMap.put("count", 4); - event = eventMap; - } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { - event = new ArrayList<>(Arrays.asList( - timestamp, - "Feridun", - 4 - )); - } else { - throw new UOE("Result format [%s] not supported yet", resultFormat.toString()); - } - return new ScanResultValue(segmentId, columns, Collections.singletonList(event)); - } - - private List> getEventsListResultFormat(ScanResultValue scanResultValue) - { - return (List>) scanResultValue.getEvents(); - } - - private List> getEventsCompactedListResultFormat(ScanResultValue scanResultValue) - { - return (List>) scanResultValue.getEvents(); - } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java new file mode 100644 index 000000000000..554c9229bfe7 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java @@ -0,0 +1,71 @@ +/* + * 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.query.scan; + +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.segment.column.ColumnHolder; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ScanQueryTestHelper +{ + public static ScanResultValue generateScanResultValue(long timestamp, ScanQuery.ResultFormat resultFormat, int batchSize) + { + String segmentId = "some_segment_id"; + List columns = new ArrayList<>(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "name", "count")); + List events = new ArrayList<>(); + for (int i = 0; i < batchSize; i++) { + Object event; + if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { + Map eventMap = new HashMap<>(); + eventMap.put(ColumnHolder.TIME_COLUMN_NAME, timestamp); + eventMap.put("name", "Feridun"); + eventMap.put("count", i); + event = eventMap; + } else if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { + event = new ArrayList<>(Arrays.asList( + timestamp, + "Feridun", + i + )); + } else { + throw new UOE("Result format [%s] not supported yet", resultFormat.toString()); + } + events.add(event); + } + return new ScanResultValue(segmentId, columns, events); + } + + public static List> getEventsListResultFormat(ScanResultValue scanResultValue) + { + return (List>) scanResultValue.getEvents(); + } + + public static List> getEventsCompactedListResultFormat(ScanResultValue scanResultValue) + { + return (List>) scanResultValue.getEvents(); + } + +} From 83ec3fe1f13c384aca52ceef0ba03b300b03d8d9 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 1 Mar 2019 13:40:22 -0800 Subject: [PATCH 58/91] Nit-change on javadoc --- .../src/main/java/org/apache/druid/query/scan/ScanQuery.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 7a5cfcf578d2..eacfcee6c1f8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -108,8 +108,8 @@ public byte[] getCacheKey() } /** - * This context flag corresponds to whether the query is running on a Broker or Historical. Specifically, this is - * used to perform batching exclusively at the broker level for time-ordered scans. + * This context flag corresponds to whether the query is running on the "outermost" process (i.e. the process + * the query is sent to). */ public static final String CTX_KEY_OUTERMOST = "scanOutermost"; From 35c96d355726cf5d238435655fecbfe19ea8ddb6 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 4 Mar 2019 16:00:44 -0800 Subject: [PATCH 59/91] Checkstyle fix --- .../org/apache/druid/query/scan/ScanQueryTestHelper.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java index 554c9229bfe7..90c3bda79edf 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTestHelper.java @@ -24,14 +24,17 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; public class ScanQueryTestHelper { - public static ScanResultValue generateScanResultValue(long timestamp, ScanQuery.ResultFormat resultFormat, int batchSize) + public static ScanResultValue generateScanResultValue( + long timestamp, + ScanQuery.ResultFormat resultFormat, + int batchSize + ) { String segmentId = "some_segment_id"; List columns = new ArrayList<>(Arrays.asList(ColumnHolder.TIME_COLUMN_NAME, "name", "count")); From 6dc53b311c568e29a6937fdd5f17a5623d14533f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 6 Mar 2019 10:34:13 -0800 Subject: [PATCH 60/91] Improved test and appeased TeamCity --- .../java/org/apache/druid/query/scan/ScanQuery.java | 10 ---------- .../druid/query/scan/ScanQueryRunnerFactoryTest.java | 9 +++++++-- 2 files changed, 7 insertions(+), 12 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index eacfcee6c1f8..f484e0d0c7f8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -75,11 +75,6 @@ public static ResultFormat fromString(String name) return RESULT_FORMAT_LIST; } } - - public byte[] getCacheKey() - { - return new byte[]{(byte) this.ordinal()}; - } } public enum TimeOrder @@ -100,11 +95,6 @@ public static TimeOrder fromString(String name) { return valueOf(StringUtils.toUpperCase(name)); } - - public byte[] getCacheKey() - { - return new byte[]{(byte) this.ordinal()}; - } } /** diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index a6d4f9de7979..b1e9d8ed9c60 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -101,7 +101,7 @@ public void testSortAndLimitScanResultValues() List srvs = new ArrayList<>(numElements); List expectedEventTimestamps = new ArrayList<>(); for (int i = 0; i < numElements; i++) { - long timestamp = (long) (ThreadLocalRandom.current().nextLong()); + long timestamp = (ThreadLocalRandom.current().nextLong()); expectedEventTimestamps.add(timestamp); srvs.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); } @@ -136,7 +136,7 @@ public void testSortAndLimitScanResultValues() // check total # of rows <= limit Assert.assertTrue(output.size() <= query.getLimit()); - // check ordering and values are correct + // check ordering is correct for (int i = 1; i < output.size(); i++) { if (query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { Assert.assertTrue(output.get(i).getFirstEventTimestamp(resultFormat) < @@ -146,5 +146,10 @@ public void testSortAndLimitScanResultValues() output.get(i - 1).getFirstEventTimestamp(resultFormat)); } } + + // check the values are correct + for(int i = 0; i < query.getLimit() && i < output.size(); i++) { + Assert.assertEquals((long) expectedEventTimestamps.get(i), output.get(i).getFirstEventTimestamp(resultFormat)); + } } } From fb966def8335e6808f0fe5d2d6a122dcd28f2355 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Mar 2019 11:03:01 -0800 Subject: [PATCH 61/91] Sorry, checkstyle --- .../org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index b1e9d8ed9c60..562be3d76e28 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -148,7 +148,7 @@ public void testSortAndLimitScanResultValues() } // check the values are correct - for(int i = 0; i < query.getLimit() && i < output.size(); i++) { + for (int i = 0; i < query.getLimit() && i < output.size(); i++) { Assert.assertEquals((long) expectedEventTimestamps.get(i), output.get(i).getFirstEventTimestamp(resultFormat)); } } From 73f4038068f2e30cb3487cc175730f3b97c5c8d2 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 7 Mar 2019 18:40:00 -0800 Subject: [PATCH 62/91] Applied Jon's recommended changes --- docs/content/querying/scan-query.md | 16 ++++++++-------- .../org/apache/druid/query/scan/ScanQuery.java | 3 ++- .../apache/druid/query/scan/ScanQueryEngine.java | 15 ++++++++++++--- .../query/scan/ScanQueryLimitRowIterator.java | 5 ++--- .../druid/query/scan/ScanQueryRunnerFactory.java | 16 ++++++---------- 5 files changed, 30 insertions(+), 25 deletions(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index a1123603f664..c747dcb7bb60 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -156,12 +156,12 @@ The format of the result when resultFormat equals `compactedList`: ## Time Ordering The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering -will yield results that do not indicate which segment rows are from. Furthermore, time ordering is only supported -where the result set limit is less than `druid.query.scan.maxRowsQueuedForTimeOrdering` rows and less than -`druid.query.scan.maxSegmentsTimeOrderedInMemory` segments are scanned per Historical. The reasoning behind these -limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory -if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query -result set limit and the number of segments being scanned. +will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, +time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForTimeOrdering` +rows **or** fewer than `druid.query.scan.maxSegmentsTimeOrderedInMemory` segments are scanned per Historical. The +reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too +much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on +query result set limit and the number of segments being scanned. 1. Priority Queue: Each segment on a Historical is opened sequentially. Every row is added to a bounded priority queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending) @@ -170,8 +170,8 @@ priority queue are streamed back to the Broker(s) in batches. Attempting to loa risk of Historical nodes running out of memory. The `druid.query.scan.maxRowsQueuedForTimeOrdering` property protects from this by limiting the number of rows in the query result set when time ordering is used. -2. K-Way/N-Way Merge: Each segment on a Historical is opened in parallel. Since each segment's rows are already -time-ordered, a k-way merge can be performed on the results from each segment. This approach doesn't persist the entire +2. N-Way Merge: Each segment on a Historical is opened in parallel. Since each segment's rows are already +time-ordered, an n-way merge can be performed on the results from each segment. This approach doesn't persist the entire result set in memory (like the Priority Queue) as it streams back batches as they are returned from the merge function. However, attempting to query too many segments could also result in high memory usage due to the need to open decompression and decoding buffers for each. The `druid.query.scan.maxSegmentsTimeOrderedInMemory` limit protects diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index f484e0d0c7f8..223776d8c268 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -25,6 +25,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; @@ -72,7 +73,7 @@ public static ResultFormat fromString(String name) case "list": return RESULT_FORMAT_LIST; default: - return RESULT_FORMAT_LIST; + throw new UOE("Scan query result format [%s] is not supported.", name); } } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 43bd49f5a460..a14e0a070d91 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -123,9 +123,7 @@ public Sequence process( if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) == null) { responseContext.put(ScanQueryRunnerFactory.CTX_COUNT, 0L); } - final long limit = query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) ? - query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) : - Long.MAX_VALUE; + final long limit = calculateLimit(query, responseContext); return Sequences.concat( adapter .makeCursors( @@ -259,4 +257,15 @@ public void cleanup(Iterator iterFromMake) )) ); } + + /** + * If we're performing time-ordering, we want to scan through every row in the segment (hence the maximum limit) + */ + private long calculateLimit(ScanQuery query, Map responseContext) + { + if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { + return query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); + } + return Long.MAX_VALUE; + } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 79c67031612e..ab24c2af8789 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -52,7 +52,6 @@ */ public class ScanQueryLimitRowIterator implements CloseableIterator { - private static final String TIME_ORDERING_SEGMENT_ID = "No segment ID available when using time ordering"; private Yielder yielder; private ScanQuery.ResultFormat resultFormat; private long limit; @@ -98,7 +97,7 @@ public ScanResultValue next() throw new UOE(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); } - // We don't want to perform batching at the historical-level if we're time ordering + // We want to perform batching if we are not time-ordering or are at the outer level if we are re time-ordering if (query.getTimeOrder() == ScanQuery.TimeOrder.NONE || !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); @@ -128,7 +127,7 @@ public ScanResultValue next() yielder = yielder.next(null); count++; } - return new ScanResultValue(TIME_ORDERING_SEGMENT_ID, columns, eventsToAdd); + return new ScanResultValue(null, columns, eventsToAdd); } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index ebfca4865920..dd81b5f2e736 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -130,19 +130,15 @@ public QueryRunner mergeRunners( ); return unbatched; - } else if (query.getLimit() > scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { - throw new UOE( - "Time ordering for query result set limit of %,d is not supported. Try lowering the result " - + "set size to less than or equal to the configurable time ordering limit of %,d rows.", - query.getLimit(), - scanQueryConfig.getMaxRowsQueuedForTimeOrdering() - ); } throw new UOE( - "Time ordering for queries of %,d segments per historical is not supported. Try reducing the scope " - + "of the query to scan fewer segments than the configurable time ordering limit of %,d segments", + "Time ordering for queries of %,d segments per historical and a row limit of %,d is not supported." + + " Try reducing the scope of the query to scan fewer segments than the configurable time ordering limit of" + + " %,d segments or lower the row limit below %,d.", numSegments, - scanQueryConfig.getMaxSegmentsTimeOrderedInMemory() + query.getLimit(), + scanQueryConfig.getMaxSegmentsTimeOrderedInMemory(), + scanQueryConfig.getMaxRowsQueuedForTimeOrdering() ); }; } From cce917ab846198706ec8177a91869f9aa43e0525 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 8 Mar 2019 14:11:07 -0800 Subject: [PATCH 63/91] Checkstyle fix --- .../main/java/org/apache/druid/query/scan/ScanQueryEngine.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index a14e0a070d91..f801ef130a56 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -261,7 +261,7 @@ public void cleanup(Iterator iterFromMake) /** * If we're performing time-ordering, we want to scan through every row in the segment (hence the maximum limit) */ - private long calculateLimit(ScanQuery query, Map responseContext) + private long calculateLimit(ScanQuery query, Map responseContext) { if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { return query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); From 45e95bb1f40d50ab3a0a745d2b5fca34c3f53a82 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 12 Mar 2019 11:09:08 -0700 Subject: [PATCH 64/91] Optimization --- .../java/org/apache/druid/query/scan/ScanQueryEngine.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index f801ef130a56..93482694100e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -259,13 +259,14 @@ public void cleanup(Iterator iterFromMake) } /** - * If we're performing time-ordering, we want to scan through every row in the segment (hence the maximum limit) + * If we're performing time-ordering, we want to scan through the first `limit` rows ignoring the number + * of rows already counted on other segments. */ private long calculateLimit(ScanQuery query, Map responseContext) { if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { return query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); } - return Long.MAX_VALUE; + return query.getLimit(); } } From 57b568265488066c046f225c1982dba85e8a64ba Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 12 Mar 2019 12:44:02 -0700 Subject: [PATCH 65/91] Fixed tests --- .../org/apache/druid/query/scan/ScanQueryRunnerTest.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index 70fa2033fadf..6d9834bc83d3 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -569,7 +569,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() ascendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - seg1Results.length + seg2Results.length + limit ); verify(ascendingExpectedResults, results); } @@ -629,7 +629,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() descendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - seg1Results.length + seg2Results.length + limit ); verify(descendingExpectedResults, results); } @@ -689,7 +689,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList ascendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - seg1Results.length + seg2Results.length + limit ); results = compactedListToRow(results); verify(ascendingExpectedResults, results); @@ -753,7 +753,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis descendingEvents, legacy ? Lists.newArrayList(getTimestampName(), "quality", "index") : Lists.newArrayList("quality", "index"), 0, - expectedRet.length + limit ); results = compactedListToRow(results); verify(descendingExpectedResults, results); From a032c46ee09cd80b78f25d0da51f5179774aa75f Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 12 Mar 2019 16:47:17 -0700 Subject: [PATCH 66/91] Updated error message --- .../org/apache/druid/query/scan/ScanQueryRunnerFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index dd81b5f2e736..8fad00aa8012 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -133,7 +133,7 @@ public QueryRunner mergeRunners( } throw new UOE( "Time ordering for queries of %,d segments per historical and a row limit of %,d is not supported." - + " Try reducing the scope of the query to scan fewer segments than the configurable time ordering limit of" + + " Try reducing the scope of the query to scan fewer segments than the configurable segment limit of" + " %,d segments or lower the row limit below %,d.", numSegments, query.getLimit(), From 7e49d47391d17b411b0620794e503592d8f37481 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 12 Mar 2019 16:51:25 -0700 Subject: [PATCH 67/91] Added error message for UOE --- .../org/apache/druid/query/QueryInterruptedException.java | 5 ++++- .../apache/druid/query/QueryInterruptedExceptionTest.java | 2 ++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java index 947a6566bdfb..54bb132d79de 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java @@ -46,7 +46,8 @@ public class QueryInterruptedException extends RuntimeException public static final String QUERY_TIMEOUT = "Query timeout"; public static final String QUERY_CANCELLED = "Query cancelled"; public static final String RESOURCE_LIMIT_EXCEEDED = "Resource limit exceeded"; - public static final String UNAUTHORIZED = "Unauthorized request."; + public static final String UNAUTHORIZED = "Unauthorized request"; + public static final String UNSUPPORTED_OPERATION = "Unsupported operation"; public static final String UNKNOWN_EXCEPTION = "Unknown exception"; private final String errorCode; @@ -135,6 +136,8 @@ private static String getErrorCodeFromThrowable(Throwable e) return QUERY_TIMEOUT; } else if (e instanceof ResourceLimitExceededException) { return RESOURCE_LIMIT_EXCEEDED; + } else if (e instanceof UnsupportedOperationException) { + return UNSUPPORTED_OPERATION; } else { return UNKNOWN_EXCEPTION; } diff --git a/processing/src/test/java/org/apache/druid/query/QueryInterruptedExceptionTest.java b/processing/src/test/java/org/apache/druid/query/QueryInterruptedExceptionTest.java index ecfe3c084152..bf5d7a405490 100644 --- a/processing/src/test/java/org/apache/druid/query/QueryInterruptedExceptionTest.java +++ b/processing/src/test/java/org/apache/druid/query/QueryInterruptedExceptionTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.TestHelper; import org.junit.Assert; import org.junit.Test; @@ -43,6 +44,7 @@ public void testErrorCode() Assert.assertEquals("Query cancelled", new QueryInterruptedException(new CancellationException()).getErrorCode()); Assert.assertEquals("Query interrupted", new QueryInterruptedException(new InterruptedException()).getErrorCode()); Assert.assertEquals("Query timeout", new QueryInterruptedException(new TimeoutException()).getErrorCode()); + Assert.assertEquals("Unsupported operation", new QueryInterruptedException(new UOE("Unsupported")).getErrorCode()); Assert.assertEquals("Unknown exception", new QueryInterruptedException(null).getErrorCode()); Assert.assertEquals("Unknown exception", new QueryInterruptedException(new ISE("Something bad!")).getErrorCode()); Assert.assertEquals( From 2528a5614267c48714abdb30fd7a2ccdb61b802d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 18 Mar 2019 14:00:50 -0700 Subject: [PATCH 68/91] Renaming --- .../druid/benchmark/query/ScanBenchmark.java | 10 +++++----- docs/content/querying/scan-query.md | 2 +- .../main/java/org/apache/druid/query/Druids.java | 12 ++++++------ .../org/apache/druid/query/scan/ScanQuery.java | 14 +++++++------- .../apache/druid/query/scan/ScanQueryEngine.java | 8 ++++---- .../query/scan/ScanQueryLimitRowIterator.java | 2 +- .../druid/query/scan/ScanQueryRunnerFactory.java | 2 +- .../scan/ScanResultValueTimestampComparator.java | 2 +- .../scan/ScanQueryLimitRowIteratorTest.java | 6 +++--- .../query/scan/ScanQueryRunnerFactoryTest.java | 16 ++++++++-------- .../druid/query/scan/ScanQueryRunnerTest.java | 8 ++++---- .../druid/query/scan/ScanQuerySpecTest.java | 2 +- .../ScanResultValueTimestampComparatorTest.java | 8 ++++---- 13 files changed, 46 insertions(+), 46 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index 6b250af21206..f5eeba3c766d 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -114,7 +114,7 @@ public class ScanBenchmark private int limit; @Param({"NONE", "DESCENDING", "ASCENDING"}) - private static ScanQuery.TimeOrder timeOrdering; + private static ScanQuery.Order ordering; private static final Logger log = new Logger(ScanBenchmark.class); private static final ObjectMapper JSON_MAPPER; @@ -182,7 +182,7 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .dataSource("blah") .intervals(intervalSpec) - .timeOrder(timeOrdering); + .timeOrder(ordering); } private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) @@ -203,7 +203,7 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch .filters(filter) .dataSource("blah") .intervals(intervalSpec) - .timeOrder(timeOrdering); + .timeOrder(ordering); } private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) @@ -216,7 +216,7 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) .intervals(intervalSpec) .dataSource("blah") - .timeOrder(timeOrdering); + .timeOrder(ordering); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -231,7 +231,7 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) .intervals(intervalSpec) .dataSource("blah") - .timeOrder(timeOrdering); + .timeOrder(ordering); } @Setup diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index c747dcb7bb60..0ad7bf167629 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -61,7 +61,7 @@ The following are the main parameters for Scan queries: |columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no| |batchSize|How many rows buffered before return to client. Default is `20480`|no| |limit|How many rows to return. If not specified, all rows will be returned.|no| -|timeOrder|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsTimeOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsTimeOrderedInMemory` will not be time-ordered and default to a timeOrder of "none".|none| +|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsTimeOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsTimeOrderedInMemory` will not be time-ordered and default to a order of "none".|none| |legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no| |context|An additional JSON Object which can be used to specify certain flags.|no| diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index a0d85cbb5f95..22d1e93b5a1a 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -924,7 +924,7 @@ public static class ScanQueryBuilder private DimFilter dimFilter; private List columns; private Boolean legacy; - private ScanQuery.TimeOrder timeOrder; + private ScanQuery.Order order; public ScanQueryBuilder() { @@ -938,7 +938,7 @@ public ScanQueryBuilder() dimFilter = null; columns = new ArrayList<>(); legacy = null; - timeOrder = null; + order = null; } public ScanQuery build() @@ -950,7 +950,7 @@ public ScanQuery build() resultFormat, batchSize, limit, - timeOrder, + order, dimFilter, columns, legacy, @@ -971,7 +971,7 @@ public static ScanQueryBuilder copy(ScanQuery query) .columns(query.getColumns()) .legacy(query.isLegacy()) .context(query.getContext()) - .timeOrder(query.getTimeOrder()); + .timeOrder(query.getOrder()); } public ScanQueryBuilder dataSource(String ds) @@ -1051,9 +1051,9 @@ public ScanQueryBuilder legacy(Boolean legacy) return this; } - public ScanQueryBuilder timeOrder(ScanQuery.TimeOrder timeOrder) + public ScanQueryBuilder timeOrder(ScanQuery.Order order) { - this.timeOrder = timeOrder; + this.order = order; return this; } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 223776d8c268..fb5da14be3a0 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -78,7 +78,7 @@ public static ResultFormat fromString(String name) } } - public enum TimeOrder + public enum Order { ASCENDING, DESCENDING, @@ -92,7 +92,7 @@ public String toString() } @JsonCreator - public static TimeOrder fromString(String name) + public static Order fromString(String name) { return valueOf(StringUtils.toUpperCase(name)); } @@ -111,7 +111,7 @@ public static TimeOrder fromString(String name) private final DimFilter dimFilter; private final List columns; private final Boolean legacy; - private final TimeOrder timeOrder; + private final Order order; @JsonCreator public ScanQuery( @@ -121,7 +121,7 @@ public ScanQuery( @JsonProperty("resultFormat") ResultFormat resultFormat, @JsonProperty("batchSize") int batchSize, @JsonProperty("limit") long limit, - @JsonProperty("timeOrder") TimeOrder timeOrder, + @JsonProperty("order") Order order, @JsonProperty("filter") DimFilter dimFilter, @JsonProperty("columns") List columns, @JsonProperty("legacy") Boolean legacy, @@ -138,7 +138,7 @@ public ScanQuery( this.dimFilter = dimFilter; this.columns = columns; this.legacy = legacy; - this.timeOrder = timeOrder == null ? TimeOrder.NONE : timeOrder; + this.order = order == null ? Order.NONE : order; } @JsonProperty @@ -166,9 +166,9 @@ public long getLimit() } @JsonProperty - public TimeOrder getTimeOrder() + public Order getOrder() { - return timeOrder; + return order; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index 93482694100e..ad38c87e3289 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -68,7 +68,7 @@ public Sequence process( if (responseContext.get(ScanQueryRunnerFactory.CTX_COUNT) != null) { long count = (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); - if (count >= query.getLimit() && query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { + if (count >= query.getLimit() && query.getOrder().equals(ScanQuery.Order.NONE)) { return Sequences.empty(); } } @@ -131,8 +131,8 @@ public Sequence process( intervals.get(0), query.getVirtualColumns(), Granularities.ALL, - query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING) || - (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE) && query.isDescending()), + query.getOrder().equals(ScanQuery.Order.DESCENDING) || + (query.getOrder().equals(ScanQuery.Order.NONE) && query.isDescending()), null ) .map(cursor -> new BaseSequence<>( @@ -264,7 +264,7 @@ public void cleanup(Iterator iterFromMake) */ private long calculateLimit(ScanQuery query, Map responseContext) { - if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { + if (query.getOrder().equals(ScanQuery.Order.NONE)) { return query.getLimit() - (long) responseContext.get(ScanQueryRunnerFactory.CTX_COUNT); } return query.getLimit(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index ab24c2af8789..4ec29f5208a3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -98,7 +98,7 @@ public ScanResultValue next() } // We want to perform batching if we are not time-ordering or are at the outer level if we are re time-ordering - if (query.getTimeOrder() == ScanQuery.TimeOrder.NONE || + if (query.getOrder() == ScanQuery.Order.NONE || !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); List events = (List) batch.getEvents(); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 8fad00aa8012..6a45c64d5bf2 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -92,7 +92,7 @@ public QueryRunner mergeRunners( // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); - if (query.getTimeOrder().equals(ScanQuery.TimeOrder.NONE)) { + if (query.getOrder().equals(ScanQuery.Order.NONE)) { // Use normal strategy return Sequences.concat( Sequences.map( diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java index f94020bf81be..dcf3bade136f 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueTimestampComparator.java @@ -46,7 +46,7 @@ public int compare(ScanResultValue o1, ScanResultValue o2) comparison = Longs.compare( o1.getFirstEventTimestamp(scanQuery.getResultFormat()), o2.getFirstEventTimestamp(scanQuery.getResultFormat())); - if (scanQuery.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { + if (scanQuery.getOrder().equals(ScanQuery.Order.DESCENDING)) { return comparison; } return comparison * -1; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java index 6609f532628e..862df857e154 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java @@ -103,7 +103,7 @@ public void testNonTimeOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.TimeOrder.NONE) + .timeOrder(ScanQuery.Order.NONE) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) @@ -144,7 +144,7 @@ public void testBrokerTimeOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) @@ -183,7 +183,7 @@ public void testHistoricalTimeOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 562be3d76e28..d93a9e123c89 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -56,14 +56,14 @@ public ScanQueryRunnerFactoryTest( final int batchSize, final long limit, final ScanQuery.ResultFormat resultFormat, - final ScanQuery.TimeOrder timeOrder + final ScanQuery.Order order ) { this.numElements = numElements; this.query = Druids.newScanQueryBuilder() .batchSize(batchSize) .limit(limit) - .timeOrder(timeOrder) + .timeOrder(order) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .dataSource("some datasource") .resultFormat(resultFormat) @@ -81,9 +81,9 @@ public static Iterable constructorFeeder() ScanQuery.ResultFormat.RESULT_FORMAT_LIST, ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST ); - List timeOrder = ImmutableList.of( - ScanQuery.TimeOrder.ASCENDING, - ScanQuery.TimeOrder.DESCENDING + List order = ImmutableList.of( + ScanQuery.Order.ASCENDING, + ScanQuery.Order.DESCENDING ); return QueryRunnerTestHelper.cartesian( @@ -91,7 +91,7 @@ public static Iterable constructorFeeder() batchSizes, limits, resultFormats, - timeOrder + order ); } @@ -112,7 +112,7 @@ public void testSortAndLimitScanResultValues() } else if (o1 < o2) { retVal = -1; } - if (query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { return retVal * -1; } return retVal; @@ -138,7 +138,7 @@ public void testSortAndLimitScanResultValues() // check ordering is correct for (int i = 1; i < output.size(); i++) { - if (query.getTimeOrder().equals(ScanQuery.TimeOrder.DESCENDING)) { + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { Assert.assertTrue(output.get(i).getFirstEventTimestamp(resultFormat) < output.get(i - 1).getFirstEventTimestamp(resultFormat)); } else { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index 6d9834bc83d3..9d1e2adaea5e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -526,7 +526,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder(ScanQuery.TimeOrder.ASCENDING) + .timeOrder(ScanQuery.Order.ASCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .build(); @@ -585,7 +585,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .build(); HashMap context = new HashMap<>(); @@ -668,7 +668,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TimeOrder.ASCENDING) + .timeOrder(ScanQuery.Order.ASCENDING) .limit(limit) .build(); @@ -729,7 +729,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .limit(limit) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index b4cf5631c487..5d7e80c60425 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -68,7 +68,7 @@ public void testSerializationLegacyString() throws Exception ScanQuery.ResultFormat.RESULT_FORMAT_LIST, 0, 3, - ScanQuery.TimeOrder.NONE, + ScanQuery.Order.NONE, null, Arrays.asList("market", "quality", "index"), null, diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index 293462217f2b..ea1b36769120 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -52,7 +52,7 @@ public static void setup() public void comparisonDescendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -89,7 +89,7 @@ public void comparisonDescendingListTest() public void comparisonAscendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TimeOrder.ASCENDING) + .timeOrder(ScanQuery.Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -126,7 +126,7 @@ public void comparisonAscendingListTest() public void comparisonDescendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TimeOrder.DESCENDING) + .timeOrder(ScanQuery.Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -161,7 +161,7 @@ public void comparisonDescendingCompactedListTest() public void comparisonAscendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.TimeOrder.ASCENDING) + .timeOrder(ScanQuery.Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) From 4823dab895770a87356fe2ae4e9858bb4ba03fc3 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 20 Mar 2019 16:05:53 -0700 Subject: [PATCH 69/91] Finish rename --- .../druid/benchmark/query/ScanBenchmark.java | 8 ++++---- docs/content/querying/scan-query.md | 16 ++++++++-------- .../java/org/apache/druid/query/Druids.java | 4 ++-- .../druid/query/scan/ScanQueryConfig.java | 12 ++++++------ .../query/scan/ScanQueryRunnerFactory.java | 8 ++++---- .../druid/query/scan/ScanQueryConfigTest.java | 18 +++++++++--------- .../scan/ScanQueryLimitRowIteratorTest.java | 12 ++++++------ .../query/scan/ScanQueryRunnerFactoryTest.java | 2 +- .../druid/query/scan/ScanQueryRunnerTest.java | 8 ++++---- .../druid/query/scan/ScanQuerySpecTest.java | 2 +- ...ScanResultValueTimestampComparatorTest.java | 8 ++++---- .../druid/sql/calcite/CalciteQueryTest.java | 8 ++++---- 12 files changed, 53 insertions(+), 53 deletions(-) diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java index f5eeba3c766d..876aa95b6751 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -182,7 +182,7 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch return Druids.newScanQueryBuilder() .dataSource("blah") .intervals(intervalSpec) - .timeOrder(ordering); + .order(ordering); } private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) @@ -203,7 +203,7 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch .filters(filter) .dataSource("blah") .intervals(intervalSpec) - .timeOrder(ordering); + .order(ordering); } private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) @@ -216,7 +216,7 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch .filters(new SelectorDimFilter(dimName, "3", StrlenExtractionFn.instance())) .intervals(intervalSpec) .dataSource("blah") - .timeOrder(ordering); + .order(ordering); } private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) @@ -231,7 +231,7 @@ private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSch .filters(new BoundDimFilter(dimName, "100", "10000", true, true, true, null, null)) .intervals(intervalSpec) .dataSource("blah") - .timeOrder(ordering); + .order(ordering); } @Setup diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index 0ad7bf167629..341519699c45 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -61,7 +61,7 @@ The following are the main parameters for Scan queries: |columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no| |batchSize|How many rows buffered before return to client. Default is `20480`|no| |limit|How many rows to return. If not specified, all rows will be returned.|no| -|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsTimeOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsTimeOrderedInMemory` will not be time-ordered and default to a order of "none".|none| +|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsOrderedInMemory` will not be time-ordered and default to a order of "none".|none| |legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no| |context|An additional JSON Object which can be used to specify certain flags.|no| @@ -157,8 +157,8 @@ The format of the result when resultFormat equals `compactedList`: The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, -time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForTimeOrdering` -rows **or** fewer than `druid.query.scan.maxSegmentsTimeOrderedInMemory` segments are scanned per Historical. The +time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` +rows **or** fewer than `druid.query.scan.maxSegmentsOrderedInMemory` segments are scanned per Historical. The reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set limit and the number of segments being scanned. @@ -167,17 +167,17 @@ query result set limit and the number of segments being scanned. queue which is ordered by timestamp. For every row above the result set limit, the row with the earliest (if descending) or latest (if ascending) timestamp will be dequeued. After every row has been processed, the sorted contents of the priority queue are streamed back to the Broker(s) in batches. Attempting to load too many rows into memory runs the -risk of Historical nodes running out of memory. The `druid.query.scan.maxRowsQueuedForTimeOrdering` property protects +risk of Historical nodes running out of memory. The `druid.query.scan.maxRowsQueuedForOrdering` property protects from this by limiting the number of rows in the query result set when time ordering is used. 2. N-Way Merge: Each segment on a Historical is opened in parallel. Since each segment's rows are already time-ordered, an n-way merge can be performed on the results from each segment. This approach doesn't persist the entire result set in memory (like the Priority Queue) as it streams back batches as they are returned from the merge function. However, attempting to query too many segments could also result in high memory usage due to the need to open -decompression and decoding buffers for each. The `druid.query.scan.maxSegmentsTimeOrderedInMemory` limit protects +decompression and decoding buffers for each. The `druid.query.scan.maxSegmentsOrderedInMemory` limit protects from this by capping the number of segments opened per historical when time ordering is used. -Both `druid.query.scan.maxRowsQueuedForTimeOrdering` and `druid.query.scan.maxSegmentsTimeOrderedInMemory` are +Both `druid.query.scan.maxRowsQueuedForOrdering` and `druid.query.scan.maxSegmentsOrderedInMemory` are configurable and can be tuned based on hardware specs and number of dimensions being queried. ## Legacy mode @@ -199,6 +199,6 @@ is complete. |property|description|values|default| |--------|-----------|------|-------| -|druid.query.scan.maxRowsQueuedForTimeOrdering|The maximum number of rows returned when time ordering is used|An integer in [0, 2147483647]|100000| -|druid.query.scan.maxSegmentsTimeOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [0, 2147483647]|50| +|druid.query.scan.maxRowsQueuedForOrdering|The maximum number of rows returned when time ordering is used|An integer in [0, 2147483647]|100000| +|druid.query.scan.maxSegmentsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [0, 2147483647]|50| |druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/query/Druids.java b/processing/src/main/java/org/apache/druid/query/Druids.java index 22d1e93b5a1a..47e3ede9a339 100644 --- a/processing/src/main/java/org/apache/druid/query/Druids.java +++ b/processing/src/main/java/org/apache/druid/query/Druids.java @@ -971,7 +971,7 @@ public static ScanQueryBuilder copy(ScanQuery query) .columns(query.getColumns()) .legacy(query.isLegacy()) .context(query.getContext()) - .timeOrder(query.getOrder()); + .order(query.getOrder()); } public ScanQueryBuilder dataSource(String ds) @@ -1051,7 +1051,7 @@ public ScanQueryBuilder legacy(Boolean legacy) return this; } - public ScanQueryBuilder timeOrder(ScanQuery.Order order) + public ScanQueryBuilder order(ScanQuery.Order order) { this.order = order; return this; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 1b68c778ef87..1a71ba8de615 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -41,19 +41,19 @@ public ScanQueryConfig setLegacy(final boolean legacy) } @JsonProperty - private int maxRowsQueuedForTimeOrdering = 100000; + private int maxRowsQueuedForOrdering = 100000; - public int getMaxRowsQueuedForTimeOrdering() + public int getMaxRowsQueuedForOrdering() { - return maxRowsQueuedForTimeOrdering; + return maxRowsQueuedForOrdering; } @JsonProperty - private int maxSegmentsTimeOrderedInMemory = 50; + private int maxSegmentsOrderedInMemory = 50; - public int getMaxSegmentsTimeOrderedInMemory() + public int getMaxSegmentsOrderedInMemory() { - return maxSegmentsTimeOrderedInMemory; + return maxSegmentsOrderedInMemory; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 6a45c64d5bf2..a90e619f805d 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -100,7 +100,7 @@ public QueryRunner mergeRunners( input -> input.run(queryPlus, responseContext) ) ); - } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForTimeOrdering()) { + } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { // Use priority queue strategy return sortAndLimitScanResultValues( Sequences.concat(Sequences.map( @@ -109,7 +109,7 @@ public QueryRunner mergeRunners( )), query ); - } else if (numSegments <= scanQueryConfig.getMaxSegmentsTimeOrderedInMemory()) { + } else if (numSegments <= scanQueryConfig.getMaxSegmentsOrderedInMemory()) { // Use n-way merge strategy final Sequence unbatched = Sequences.map( @@ -137,8 +137,8 @@ public QueryRunner mergeRunners( + " %,d segments or lower the row limit below %,d.", numSegments, query.getLimit(), - scanQueryConfig.getMaxSegmentsTimeOrderedInMemory(), - scanQueryConfig.getMaxRowsQueuedForTimeOrdering() + scanQueryConfig.getMaxSegmentsOrderedInMemory(), + scanQueryConfig.getMaxRowsQueuedForOrdering() ); }; } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java index 2bbc6e51f559..7c1f5d5e6314 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java @@ -31,15 +31,15 @@ public class ScanQueryConfigTest private final ImmutableMap CONFIG_MAP = ImmutableMap .builder() - .put("maxSegmentsTimeOrderedInMemory", "1") - .put("maxRowsQueuedForTimeOrdering", "1") + .put("maxSegmentsOrderedInMemory", "1") + .put("maxRowsQueuedForOrdering", "1") .put("legacy", "true") .build(); private final ImmutableMap CONFIG_MAP2 = ImmutableMap .builder() .put("legacy", "false") - .put("maxSegmentsTimeOrderedInMemory", "42") + .put("maxSegmentsOrderedInMemory", "42") .build(); private final ImmutableMap CONFIG_MAP_EMPTY = ImmutableMap @@ -50,18 +50,18 @@ public class ScanQueryConfigTest public void testSerde() { final ScanQueryConfig config = MAPPER.convertValue(CONFIG_MAP, ScanQueryConfig.class); - Assert.assertEquals(1, config.getMaxRowsQueuedForTimeOrdering()); - Assert.assertEquals(1, config.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertEquals(1, config.getMaxRowsQueuedForOrdering()); + Assert.assertEquals(1, config.getMaxSegmentsOrderedInMemory()); Assert.assertTrue(config.isLegacy()); final ScanQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, ScanQueryConfig.class); - Assert.assertEquals(100000, config2.getMaxRowsQueuedForTimeOrdering()); - Assert.assertEquals(42, config2.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertEquals(100000, config2.getMaxRowsQueuedForOrdering()); + Assert.assertEquals(42, config2.getMaxSegmentsOrderedInMemory()); Assert.assertFalse(config2.isLegacy()); final ScanQueryConfig config3 = MAPPER.convertValue(CONFIG_MAP_EMPTY, ScanQueryConfig.class); - Assert.assertEquals(100000, config3.getMaxRowsQueuedForTimeOrdering()); - Assert.assertEquals(50, config3.getMaxSegmentsTimeOrderedInMemory()); + Assert.assertEquals(100000, config3.getMaxRowsQueuedForOrdering()); + Assert.assertEquals(50, config3.getMaxSegmentsOrderedInMemory()); Assert.assertFalse(config3.isLegacy()); } } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java index 862df857e154..f0dafbc4370c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryLimitRowIteratorTest.java @@ -99,11 +99,11 @@ public void setup() * Expect no batching to occur and limit to be applied */ @Test - public void testNonTimeOrderedScan() + public void testNonOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.Order.NONE) + .order(ScanQuery.Order.NONE) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) @@ -140,11 +140,11 @@ public void testNonTimeOrderedScan() * is a sequence of single-event ScanResultValues. */ @Test - public void testBrokerTimeOrderedScan() + public void testBrokerOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) @@ -179,11 +179,11 @@ public void testBrokerTimeOrderedScan() * (unbatching and sorting occurs in ScanQueryRunnerFactory#mergeRunners()). */ @Test - public void testHistoricalTimeOrderedScan() + public void testHistoricalOrderedScan() { ScanQuery query = Druids.newScanQueryBuilder() .limit(limit) - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .dataSource("some datasource") .batchSize(batchSize) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index d93a9e123c89..e7fd85cea73e 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -63,7 +63,7 @@ public ScanQueryRunnerFactoryTest( this.query = Druids.newScanQueryBuilder() .batchSize(batchSize) .limit(limit) - .timeOrder(order) + .order(order) .intervals(QueryRunnerTestHelper.fullOnIntervalSpec) .dataSource("some datasource") .resultFormat(resultFormat) diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java index 9d1e2adaea5e..b3a0d0069ac1 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerTest.java @@ -526,7 +526,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder(ScanQuery.Order.ASCENDING) + .order(ScanQuery.Order.ASCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .build(); @@ -585,7 +585,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingListFormat() .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .limit(limit) - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .build(); HashMap context = new HashMap<>(); @@ -668,7 +668,7 @@ public void testFullOnSelectWithFilterLimitAndAscendingTimeOrderingCompactedList .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.Order.ASCENDING) + .order(ScanQuery.Order.ASCENDING) .limit(limit) .build(); @@ -729,7 +729,7 @@ public void testFullOnSelectWithFilterLimitAndDescendingTimeOrderingCompactedLis .filters(new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "spot", null)) .columns(QueryRunnerTestHelper.qualityDimension, QueryRunnerTestHelper.indexMetric) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .context(ImmutableMap.of(ScanQuery.CTX_KEY_OUTERMOST, false)) .limit(limit) .build(); diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 5d7e80c60425..b49a1033f2e8 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -53,7 +53,7 @@ public void testSerializationLegacyString() throws Exception + "\"resultFormat\":\"list\"," + "\"batchSize\":20480," + "\"limit\":3," - + "\"timeOrder\":\"none\"," + + "\"order\":\"none\"," + "\"filter\":null," + "\"columns\":[\"market\",\"quality\",\"index\"]," + "\"legacy\":null," diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java index ea1b36769120..70f2e080b9f2 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanResultValueTimestampComparatorTest.java @@ -52,7 +52,7 @@ public static void setup() public void comparisonDescendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -89,7 +89,7 @@ public void comparisonDescendingListTest() public void comparisonAscendingListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.Order.ASCENDING) + .order(ScanQuery.Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -126,7 +126,7 @@ public void comparisonAscendingListTest() public void comparisonDescendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.Order.DESCENDING) + .order(ScanQuery.Order.DESCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) @@ -161,7 +161,7 @@ public void comparisonDescendingCompactedListTest() public void comparisonAscendingCompactedListTest() { ScanQuery query = Druids.newScanQueryBuilder() - .timeOrder(ScanQuery.Order.ASCENDING) + .order(ScanQuery.Order.ASCENDING) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .dataSource("some src") .intervals(intervalSpec) 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 0a77b16e2f84..687887cc5b62 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 @@ -511,7 +511,7 @@ public void testExplainSelectStar() throws Exception ImmutableList.of(), ImmutableList.of( new Object[]{ - "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" } ) ); @@ -767,10 +767,10 @@ public void testExplainSelfJoinWithFallback() throws Exception String emptyStringEq = NullHandling.replaceWithDefault() ? null : "\"\""; final String explanation = "BindableJoin(condition=[=($0, $2)], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":" + emptyStringEq + ",\"extractionFn\":null}},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"; + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n"; testQuery( PLANNER_CONFIG_FALLBACK, @@ -6669,7 +6669,7 @@ public void testUsingSubqueryAsPartOfOrFilter() throws Exception + " BindableFilter(condition=[OR(=($0, 'xxx'), CAST(AND(IS NOT NULL($4), <>($2, 0), IS NOT NULL($1))):BOOLEAN)])\n" + " BindableJoin(condition=[=($1, $3)], joinType=[left])\n" + " BindableJoin(condition=[true], joinType=[inner])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"timeOrder\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[],\"resultFormat\":\"compactedList\",\"batchSize\":20480,\"limit\":9223372036854775807,\"order\":\"none\",\"filter\":null,\"columns\":[\"dim1\",\"dim2\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false,\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING, dim2:STRING}])\n" + " DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"descending\":false,\"virtualColumns\":[],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"postAggregations\":[],\"limit\":2147483647,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"skipEmptyBuckets\":true,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"}}], signature=[{a0:LONG}])\n" + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"1\",\"outputType\":\"LONG\"}],\"filter\":{\"type\":\"like\",\"dimension\":\"dim1\",\"pattern\":\"%bc\",\"escape\":null,\"extractionFn\":null},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\"},{\"type\":\"default\",\"dimension\":\"v0\",\"outputName\":\"v0\",\"outputType\":\"LONG\"}],\"aggregations\":[],\"postAggregations\":[],\"having\":null,\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\"},\"descending\":false}], signature=[{d0:STRING, v0:LONG}])\n"; From 42f5246b8d0c1879c2dc45334966bf52f543ea74 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 20 Mar 2019 17:40:19 -0700 Subject: [PATCH 70/91] Smarter limiting for pQueue method --- .../apache/druid/query/scan/ScanQuery.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 53 ++++++++++++++++--- .../scan/ScanQueryRunnerFactoryTest.java | 3 +- 3 files changed, 50 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index fb5da14be3a0..d68f2c82ba1b 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -130,7 +130,7 @@ public ScanQuery( { super(dataSource, querySegmentSpec, false, context); this.virtualColumns = VirtualColumns.nullToEmpty(virtualColumns); - this.resultFormat = resultFormat == null ? ResultFormat.RESULT_FORMAT_LIST : resultFormat; + this.resultFormat = (resultFormat == null) ? ResultFormat.RESULT_FORMAT_LIST : resultFormat; this.batchSize = (batchSize == 0) ? 4096 * 5 : batchSize; this.limit = (limit == 0) ? Long.MAX_VALUE : limit; Preconditions.checkArgument(this.batchSize > 0, "batchSize must be greater than 0"); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index a90e619f805d..4bde1d3e8d98 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -20,6 +20,7 @@ package org.apache.druid.query.scan; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; @@ -35,7 +36,11 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; +import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.Segment; +import org.joda.time.Interval; import java.util.ArrayDeque; import java.util.Comparator; @@ -92,22 +97,38 @@ public QueryRunner mergeRunners( // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); responseContext.put(CTX_TIMEOUT_AT, timeoutAt); + if (query.getOrder().equals(ScanQuery.Order.NONE)) { // Use normal strategy - return Sequences.concat( + Sequence returnedRows = Sequences.concat( Sequences.map( Sequences.simple(queryRunners), input -> input.run(queryPlus, responseContext) ) ); + if (query.getLimit() <= Integer.MAX_VALUE) { + return returnedRows.limit(Math.toIntExact(query.getLimit())); + } else { + return returnedRows; + } } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { // Use priority queue strategy + List descriptorsOrdered = + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order + List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default + + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { + descriptorsOrdered = Lists.reverse(descriptorsOrdered); + queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); + } + return sortAndLimitScanResultValues( Sequences.concat(Sequences.map( - Sequences.simple(queryRunners), + Sequences.simple(queryRunnersOrdered), input -> input.run(queryPlus, responseContext) )), - query + query, + descriptorsOrdered ); } else if (numSegments <= scanQueryConfig.getMaxSegmentsOrderedInMemory()) { // Use n-way merge strategy @@ -124,7 +145,7 @@ public QueryRunner mergeRunners( seq -> seq, Ordering.from(new ScanResultValueTimestampComparator( query - )).reverse() // This needs to be reversed because + )).reverse() ).limit( Math.toIntExact(query.getLimit()) ); @@ -146,7 +167,8 @@ public QueryRunner mergeRunners( @VisibleForTesting Sequence sortAndLimitScanResultValues( Sequence inputSequence, - ScanQuery scanQuery + ScanQuery scanQuery, + List descriptorsOrdered ) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -169,8 +191,13 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } } ); - while (!yielder.isDone()) { + boolean doneScanning = false; + // We need to scan limit elements and anything else in the last segment + int numRowsScanned = 0; + Interval finalInterval = null; + while (!doneScanning) { ScanResultValue next = yielder.get(); + numRowsScanned++; List singleEventScanResultValues = next.toSingleEventScanResultValues(); for (ScanResultValue srv : singleEventScanResultValues) { // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list @@ -181,6 +208,20 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } } yielder = yielder.next(null); + if (numRowsScanned > limit && finalInterval == null) { + long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); + for (SegmentDescriptor descriptor : descriptorsOrdered) { + if (descriptor.getInterval().contains(timestampOfLimitRow)) { + finalInterval = descriptor.getInterval(); + } + } + if (finalInterval == null) { + throw new ISE("WTH??? Row came from an unscanned interval?"); + } + } + doneScanning = yielder.isDone() || + (finalInterval != null && + !finalInterval.contains(next.getFirstEventTimestamp(scanQuery.getResultFormat()))); } // Need to convert to a Deque because Priority Queue's iterator doesn't guarantee that the sorted order // will be maintained. Deque was chosen over list because its addFirst is O(1). diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index e7fd85cea73e..0d32be9e0c0d 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -121,7 +121,8 @@ public void testSortAndLimitScanResultValues() List output = factory.sortAndLimitScanResultValues( inputSequence, - query + query, + null ).toList(); // check each scan result value has one event From 43d490cc3ae697d0a61159ed6ae06906006cdf31 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Thu, 21 Mar 2019 13:16:58 -0700 Subject: [PATCH 71/91] Optimized n-way merge strategy --- .../query/scan/ScanQueryLimitRowIterator.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 128 +++++++++++------- .../scan/ScanQueryRunnerFactoryTest.java | 2 +- 3 files changed, 84 insertions(+), 48 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 4ec29f5208a3..4103b1476978 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -123,7 +123,7 @@ public ScanResultValue next() ScanResultValue srv = yielder.get(); // Only replace once using the columns from the first event columns = columns.isEmpty() ? srv.getColumns() : columns; - eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); + eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); yielder = yielder.next(null); count++; } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 4bde1d3e8d98..645f57bf4258 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -20,11 +20,13 @@ package org.apache.druid.query.scan; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -38,18 +40,19 @@ import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; -import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.Segment; import org.joda.time.Interval; import java.util.ArrayDeque; +import java.util.ArrayList; import java.util.Comparator; import java.util.Deque; -import java.util.Iterator; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.PriorityQueue; import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; public class ScanQueryRunnerFactory implements QueryRunnerFactory { @@ -88,11 +91,16 @@ public QueryRunner mergeRunners( // in single thread and in jetty thread instead of processing thread return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - int numSegments = 0; - final Iterator> segmentIt = queryRunners.iterator(); - for (; segmentIt.hasNext(); numSegments++) { - segmentIt.next(); + + List descriptorsOrdered = + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order + List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default + + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { + descriptorsOrdered = Lists.reverse(descriptorsOrdered); + queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } + // Note: this variable is effective only when queryContext has a timeout. // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); @@ -113,16 +121,7 @@ public QueryRunner mergeRunners( } } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { // Use priority queue strategy - List descriptorsOrdered = - ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order - List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default - - if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - descriptorsOrdered = Lists.reverse(descriptorsOrdered); - queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); - } - - return sortAndLimitScanResultValues( + return sortAndLimitScanResultValuesPriorityQueue( Sequences.concat(Sequences.map( Sequences.simple(queryRunnersOrdered), input -> input.run(queryPlus, responseContext) @@ -130,42 +129,79 @@ public QueryRunner mergeRunners( query, descriptorsOrdered ); - } else if (numSegments <= scanQueryConfig.getMaxSegmentsOrderedInMemory()) { - // Use n-way merge strategy - final Sequence unbatched = - Sequences.map( - Sequences.simple(queryRunners), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) - ) - ) - ).flatMerge( - seq -> seq, - Ordering.from(new ScanResultValueTimestampComparator( - query - )).reverse() - ).limit( - Math.toIntExact(query.getLimit()) - ); + } else { + Preconditions.checkState( + descriptorsOrdered.size() == queryRunnersOrdered.size(), + "Number of segment descriptors does not equal number of " + + "query runners...something went wrong!" + ); - return unbatched; + List>> descriptorsAndRunnersOrdered = new ArrayList<>(); + + for (int i = 0; i < queryRunnersOrdered.size(); i++) { + descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); + } + + LinkedHashMap>>> partitionsGroupedByInterval = + descriptorsAndRunnersOrdered.stream() + .collect(Collectors.groupingBy( + x -> x.lhs.getInterval(), + LinkedHashMap::new, + Collectors.toList() + )); + + int maxNumPartitionsInSegment = + partitionsGroupedByInterval.values() + .stream() + .map(x -> x.size()) + .max(Comparator.comparing(Integer::valueOf)).get(); + + if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentsOrderedInMemory()) { + // Use n-way merge strategy + List>> groupedRunners = new ArrayList<>(descriptorsAndRunnersOrdered.size()); + for (Map.Entry>>> entry : + partitionsGroupedByInterval.entrySet()) { + groupedRunners.add(entry.getValue().stream().map(x -> x.rhs).collect(Collectors.toList())); + } + return Sequences.concat( + Sequences.map( + Sequences.simple(groupedRunners), // Sequence of runnerGroups + runnerGroup -> + Sequences.map( + Sequences.simple(runnerGroup), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) + ) + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + query + )).reverse() + ) + ) + ).limit( + Math.toIntExact(query.getLimit()) + ); + } + throw new UOE( + "Time ordering for queries of %,d partitions per segment and a row limit of %,d is not supported." + + " Try reducing the scope of the query to scan fewer partitions than the configurable limit of" + + " %,d partitions or lower the row limit below %,d.", + maxNumPartitionsInSegment, + query.getLimit(), + scanQueryConfig.getMaxSegmentsOrderedInMemory(), + scanQueryConfig.getMaxRowsQueuedForOrdering() + ); } - throw new UOE( - "Time ordering for queries of %,d segments per historical and a row limit of %,d is not supported." - + " Try reducing the scope of the query to scan fewer segments than the configurable segment limit of" - + " %,d segments or lower the row limit below %,d.", - numSegments, - query.getLimit(), - scanQueryConfig.getMaxSegmentsOrderedInMemory(), - scanQueryConfig.getMaxRowsQueuedForOrdering() - ); + }; } @VisibleForTesting - Sequence sortAndLimitScanResultValues( + Sequence sortAndLimitScanResultValuesPriorityQueue( Sequence inputSequence, ScanQuery scanQuery, List descriptorsOrdered diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 0d32be9e0c0d..763191d8b828 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -119,7 +119,7 @@ public void testSortAndLimitScanResultValues() }); Sequence inputSequence = Sequences.simple(srvs); List output = - factory.sortAndLimitScanResultValues( + factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, null From 49472162b7fc0879159866c3736e192fc88837a4 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 10:27:41 -0700 Subject: [PATCH 72/91] Rename segment limit -> segment partitions limit --- docs/content/querying/scan-query.md | 8 ++--- .../druid/query/scan/ScanQueryConfig.java | 6 ++-- .../query/scan/ScanQueryRunnerFactory.java | 30 +++++++++++++------ .../druid/query/scan/ScanQueryConfigTest.java | 10 +++---- 4 files changed, 33 insertions(+), 21 deletions(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index 341519699c45..fd43fc092ebd 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -158,7 +158,7 @@ The format of the result when resultFormat equals `compactedList`: The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` -rows **or** fewer than `druid.query.scan.maxSegmentsOrderedInMemory` segments are scanned per Historical. The +rows **or** fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` segments are scanned per Historical. The reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set limit and the number of segments being scanned. @@ -174,10 +174,10 @@ from this by limiting the number of rows in the query result set when time order time-ordered, an n-way merge can be performed on the results from each segment. This approach doesn't persist the entire result set in memory (like the Priority Queue) as it streams back batches as they are returned from the merge function. However, attempting to query too many segments could also result in high memory usage due to the need to open -decompression and decoding buffers for each. The `druid.query.scan.maxSegmentsOrderedInMemory` limit protects +decompression and decoding buffers for each. The `druid.query.scan.maxSegmentPartitionsOrderedInMemory` limit protects from this by capping the number of segments opened per historical when time ordering is used. -Both `druid.query.scan.maxRowsQueuedForOrdering` and `druid.query.scan.maxSegmentsOrderedInMemory` are +Both `druid.query.scan.maxRowsQueuedForOrdering` and `druid.query.scan.maxSegmentPartitionsOrderedInMemory` are configurable and can be tuned based on hardware specs and number of dimensions being queried. ## Legacy mode @@ -200,5 +200,5 @@ is complete. |property|description|values|default| |--------|-----------|------|-------| |druid.query.scan.maxRowsQueuedForOrdering|The maximum number of rows returned when time ordering is used|An integer in [0, 2147483647]|100000| -|druid.query.scan.maxSegmentsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [0, 2147483647]|50| +|druid.query.scan.maxSegmentPartitionsOrderedInMemory|The maximum number of segments scanned per historical when time ordering is used|An integer in [0, 2147483647]|50| |druid.query.scan.legacy|Whether legacy mode should be turned on for Scan queries|true or false|false| \ No newline at end of file diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java index 1a71ba8de615..8121f47b48ac 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryConfig.java @@ -49,11 +49,11 @@ public int getMaxRowsQueuedForOrdering() } @JsonProperty - private int maxSegmentsOrderedInMemory = 50; + private int maxSegmentPartitionsOrderedInMemory = 50; - public int getMaxSegmentsOrderedInMemory() + public int getMaxSegmentPartitionsOrderedInMemory() { - return maxSegmentsOrderedInMemory; + return maxSegmentPartitionsOrderedInMemory; } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 645f57bf4258..e88ee35a01dd 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -136,12 +136,15 @@ public QueryRunner mergeRunners( + "query runners...something went wrong!" ); + // Combine the two lists of segment descriptors and query runners into a single list of + // segment descriptors - query runner pairs List>> descriptorsAndRunnersOrdered = new ArrayList<>(); - for (int i = 0; i < queryRunnersOrdered.size(); i++) { descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); } + // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the + // query runners for that segment LinkedHashMap>>> partitionsGroupedByInterval = descriptorsAndRunnersOrdered.stream() .collect(Collectors.groupingBy( @@ -150,19 +153,25 @@ public QueryRunner mergeRunners( Collectors.toList() )); + // Find the segment with the largest numbers of partitions. This will be used to compare with the + // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. int maxNumPartitionsInSegment = partitionsGroupedByInterval.values() .stream() .map(x -> x.size()) - .max(Comparator.comparing(Integer::valueOf)).get(); + .max(Comparator.comparing(Integer::valueOf)) + .get(); - if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentsOrderedInMemory()) { + if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory()) { // Use n-way merge strategy - List>> groupedRunners = new ArrayList<>(descriptorsAndRunnersOrdered.size()); - for (Map.Entry>>> entry : - partitionsGroupedByInterval.entrySet()) { - groupedRunners.add(entry.getValue().stream().map(x -> x.rhs).collect(Collectors.toList())); - } + List>> groupedRunners = + partitionsGroupedByInterval.entrySet() + .stream() + .map(entry -> entry.getValue() + .stream() + .map(segQueryRunnerPair -> segQueryRunnerPair.rhs) + .collect(Collectors.toList())) + .collect(Collectors.toList()); return Sequences.concat( Sequences.map( Sequences.simple(groupedRunners), // Sequence of runnerGroups @@ -192,7 +201,7 @@ public QueryRunner mergeRunners( + " %,d partitions or lower the row limit below %,d.", maxNumPartitionsInSegment, query.getLimit(), - scanQueryConfig.getMaxSegmentsOrderedInMemory(), + scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory(), scanQueryConfig.getMaxRowsQueuedForOrdering() ); } @@ -269,6 +278,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i return Sequences.simple(sortedElements); } + @VisibleForTesting + + @Override public QueryToolChest getToolchest() { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java index 7c1f5d5e6314..7a71c44e28c2 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryConfigTest.java @@ -31,7 +31,7 @@ public class ScanQueryConfigTest private final ImmutableMap CONFIG_MAP = ImmutableMap .builder() - .put("maxSegmentsOrderedInMemory", "1") + .put("maxSegmentPartitionsOrderedInMemory", "1") .put("maxRowsQueuedForOrdering", "1") .put("legacy", "true") .build(); @@ -39,7 +39,7 @@ public class ScanQueryConfigTest private final ImmutableMap CONFIG_MAP2 = ImmutableMap .builder() .put("legacy", "false") - .put("maxSegmentsOrderedInMemory", "42") + .put("maxSegmentPartitionsOrderedInMemory", "42") .build(); private final ImmutableMap CONFIG_MAP_EMPTY = ImmutableMap @@ -51,17 +51,17 @@ public void testSerde() { final ScanQueryConfig config = MAPPER.convertValue(CONFIG_MAP, ScanQueryConfig.class); Assert.assertEquals(1, config.getMaxRowsQueuedForOrdering()); - Assert.assertEquals(1, config.getMaxSegmentsOrderedInMemory()); + Assert.assertEquals(1, config.getMaxSegmentPartitionsOrderedInMemory()); Assert.assertTrue(config.isLegacy()); final ScanQueryConfig config2 = MAPPER.convertValue(CONFIG_MAP2, ScanQueryConfig.class); Assert.assertEquals(100000, config2.getMaxRowsQueuedForOrdering()); - Assert.assertEquals(42, config2.getMaxSegmentsOrderedInMemory()); + Assert.assertEquals(42, config2.getMaxSegmentPartitionsOrderedInMemory()); Assert.assertFalse(config2.isLegacy()); final ScanQueryConfig config3 = MAPPER.convertValue(CONFIG_MAP_EMPTY, ScanQueryConfig.class); Assert.assertEquals(100000, config3.getMaxRowsQueuedForOrdering()); - Assert.assertEquals(50, config3.getMaxSegmentsOrderedInMemory()); + Assert.assertEquals(50, config3.getMaxSegmentPartitionsOrderedInMemory()); Assert.assertFalse(config3.isLegacy()); } } From 1b46b58aeccf13adc516a1d94054a98efc32184c Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 15:19:52 -0700 Subject: [PATCH 73/91] Added a bit of docs --- .../query/scan/ScanQueryRunnerFactory.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index e88ee35a01dd..78595179a714 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -164,6 +164,9 @@ public QueryRunner mergeRunners( if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory()) { // Use n-way merge strategy + + // Create a list of grouped runner lists (i.e. each sublist/"runner group" corresponds to an interval) -> + // there should be no interval overlap List>> groupedRunners = partitionsGroupedByInterval.entrySet() .stream() @@ -172,14 +175,16 @@ public QueryRunner mergeRunners( .map(segQueryRunnerPair -> segQueryRunnerPair.rhs) .collect(Collectors.toList())) .collect(Collectors.toList()); - return Sequences.concat( - Sequences.map( - Sequences.simple(groupedRunners), // Sequence of runnerGroups + + + return Sequences.concat( // 5) Join all the results into a single sequence + Sequences.map( // 4) Create a sequence of results from each runner group + Sequences.simple(groupedRunners), runnerGroup -> - Sequences.map( + Sequences.map( // 3) Create a sequence of results from each runner in the group and flatmerge based on timestamp Sequences.simple(runnerGroup), - (input) -> Sequences.concat( - Sequences.map( + (input) -> Sequences.concat( // 2) Combine the deaggregated ScanResultValues into a single sequence + Sequences.map( // 1) Deaggregate each ScanResultValue returned by the query runners input.run(queryPlus, responseContext), srv -> Sequences.simple(srv.toSingleEventScanResultValues()) ) @@ -278,8 +283,6 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i return Sequences.simple(sortedElements); } - @VisibleForTesting - @Override public QueryToolChest getToolchest() From 62dcedacdeeed570134e8b5185633b207e91a547 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 15:30:41 -0700 Subject: [PATCH 74/91] More comments --- .../query/scan/ScanQueryRunnerFactory.java | 21 ++++++++++++------- 1 file changed, 13 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 78595179a714..e00ad34cbba3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -137,7 +137,7 @@ public QueryRunner mergeRunners( ); // Combine the two lists of segment descriptors and query runners into a single list of - // segment descriptors - query runner pairs + // segment descriptors - query runner pairs. This makes it easier to use stream operators. List>> descriptorsAndRunnersOrdered = new ArrayList<>(); for (int i = 0; i < queryRunnersOrdered.size(); i++) { descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); @@ -166,7 +166,8 @@ public QueryRunner mergeRunners( // Use n-way merge strategy // Create a list of grouped runner lists (i.e. each sublist/"runner group" corresponds to an interval) -> - // there should be no interval overlap + // there should be no interval overlap. We create a list of lists so we can create a sequence of sequences. + // There's no easy way to convert a LinkedHashMap to a sequence because it's non-iterable. List>> groupedRunners = partitionsGroupedByInterval.entrySet() .stream() @@ -176,15 +177,20 @@ public QueryRunner mergeRunners( .collect(Collectors.toList())) .collect(Collectors.toList()); + // (1) Deaggregate each ScanResultValue returned by the query runners + // (2) Combine the deaggregated ScanResultValues into a single sequence + // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp + // (4) Create a sequence of results from each runner group + // (5) Join all the results into a single sequence - return Sequences.concat( // 5) Join all the results into a single sequence - Sequences.map( // 4) Create a sequence of results from each runner group + return Sequences.concat( // (5) + Sequences.map( // (4) Sequences.simple(groupedRunners), runnerGroup -> - Sequences.map( // 3) Create a sequence of results from each runner in the group and flatmerge based on timestamp + Sequences.map( // (3) Sequences.simple(runnerGroup), - (input) -> Sequences.concat( // 2) Combine the deaggregated ScanResultValues into a single sequence - Sequences.map( // 1) Deaggregate each ScanResultValue returned by the query runners + (input) -> Sequences.concat( // (2) + Sequences.map( // (1) input.run(queryPlus, responseContext), srv -> Sequences.simple(srv.toSingleEventScanResultValues()) ) @@ -210,7 +216,6 @@ public QueryRunner mergeRunners( scanQueryConfig.getMaxRowsQueuedForOrdering() ); } - }; } From a87d02127c72aa5e307af94b12b6be25150349be Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 15:54:42 -0700 Subject: [PATCH 75/91] Fix checkstyle and test --- .../druid/query/scan/ScanQueryRunnerFactory.java | 13 +++++++------ .../query/scan/ScanQueryRunnerFactoryTest.java | 8 +++++--- 2 files changed, 12 insertions(+), 9 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index e00ad34cbba3..fa70627f84d8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -177,20 +177,21 @@ public QueryRunner mergeRunners( .collect(Collectors.toList())) .collect(Collectors.toList()); + // Starting from the innermost Sequences.map: // (1) Deaggregate each ScanResultValue returned by the query runners // (2) Combine the deaggregated ScanResultValues into a single sequence // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp // (4) Create a sequence of results from each runner group // (5) Join all the results into a single sequence - return Sequences.concat( // (5) - Sequences.map( // (4) + return Sequences.concat( + Sequences.map( Sequences.simple(groupedRunners), runnerGroup -> - Sequences.map( // (3) + Sequences.map( Sequences.simple(runnerGroup), - (input) -> Sequences.concat( // (2) - Sequences.map( // (1) + (input) -> Sequences.concat( + Sequences.map( input.run(queryPlus, responseContext), srv -> Sequences.simple(srv.toSingleEventScanResultValues()) ) @@ -246,7 +247,7 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } } ); - boolean doneScanning = false; + boolean doneScanning = yielder.isDone(); // We need to scan limit elements and anything else in the last segment int numRowsScanned = 0; Interval finalInterval = null; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 763191d8b828..cbf3a9d5cf6c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -20,11 +20,14 @@ package org.apache.druid.query.scan; import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.SegmentDescriptor; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; @@ -32,7 +35,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.concurrent.ThreadLocalRandom; @RunWith(Parameterized.class) @@ -101,7 +103,7 @@ public void testSortAndLimitScanResultValues() List srvs = new ArrayList<>(numElements); List expectedEventTimestamps = new ArrayList<>(); for (int i = 0; i < numElements; i++) { - long timestamp = (ThreadLocalRandom.current().nextLong()); + long timestamp = DateTimes.of("2015-01-01").plusHours(i).getMillis(); expectedEventTimestamps.add(timestamp); srvs.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); } @@ -122,7 +124,7 @@ public void testSortAndLimitScanResultValues() factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, - null + ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) ).toList(); // check each scan result value has one event From 8b3b6b51ed0d3bc3c937620d5b92096998e32080 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Fri, 22 Mar 2019 16:01:56 -0700 Subject: [PATCH 76/91] Nit comment --- .../java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java | 1 + 1 file changed, 1 insertion(+) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index fa70627f84d8..dbad5fb00703 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -264,6 +264,7 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } } yielder = yielder.next(null); + // Finish scanning the interval containing the limit row if (numRowsScanned > limit && finalInterval == null) { long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); for (SegmentDescriptor descriptor : descriptorsOrdered) { From 86d9730fc9f241b3010b123a45b1fc38a206a9af Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 11:01:35 -0700 Subject: [PATCH 77/91] Fixed failing tests -> allow usage of all types of segment spec --- .../query/scan/ScanQueryRunnerFactory.java | 31 ++++++++++--------- .../scan/ScanQueryRunnerFactoryTest.java | 2 +- 2 files changed, 17 insertions(+), 16 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index dbad5fb00703..8ca9ce55da84 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -67,6 +67,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory mergeRunners( return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - List descriptorsOrdered = - ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order + List intervalsOrdered = + query.getQuerySegmentSpec().getIntervals(); // Ascending time order List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - descriptorsOrdered = Lists.reverse(descriptorsOrdered); + intervalsOrdered = Lists.reverse(intervalsOrdered); queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } @@ -127,28 +128,28 @@ public QueryRunner mergeRunners( input -> input.run(queryPlus, responseContext) )), query, - descriptorsOrdered + intervalsOrdered ); } else { Preconditions.checkState( - descriptorsOrdered.size() == queryRunnersOrdered.size(), - "Number of segment descriptors does not equal number of " + intervalsOrdered.size() == queryRunnersOrdered.size(), + "Number of intervals from the query segment spec does not equal number of " + "query runners...something went wrong!" ); // Combine the two lists of segment descriptors and query runners into a single list of // segment descriptors - query runner pairs. This makes it easier to use stream operators. - List>> descriptorsAndRunnersOrdered = new ArrayList<>(); + List>> intervalsAndRunnersOrdered = new ArrayList<>(); for (int i = 0; i < queryRunnersOrdered.size(); i++) { - descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); + intervalsAndRunnersOrdered.add(new Pair<>(intervalsOrdered.get(i), queryRunnersOrdered.get(i))); } // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the // query runners for that segment - LinkedHashMap>>> partitionsGroupedByInterval = - descriptorsAndRunnersOrdered.stream() + LinkedHashMap>>> partitionsGroupedByInterval = + intervalsAndRunnersOrdered.stream() .collect(Collectors.groupingBy( - x -> x.lhs.getInterval(), + x -> x.lhs, LinkedHashMap::new, Collectors.toList() )); @@ -224,7 +225,7 @@ public QueryRunner mergeRunners( Sequence sortAndLimitScanResultValuesPriorityQueue( Sequence inputSequence, ScanQuery scanQuery, - List descriptorsOrdered + List intervalsOrdered ) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -267,9 +268,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i // Finish scanning the interval containing the limit row if (numRowsScanned > limit && finalInterval == null) { long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (SegmentDescriptor descriptor : descriptorsOrdered) { - if (descriptor.getInterval().contains(timestampOfLimitRow)) { - finalInterval = descriptor.getInterval(); + for (Interval interval : intervalsOrdered) { + if (interval.contains(timestampOfLimitRow)) { + finalInterval = interval; } } if (finalInterval == null) { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index cbf3a9d5cf6c..53e2b2115d55 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -124,7 +124,7 @@ public void testSortAndLimitScanResultValues() factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, - ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) + ImmutableList.of(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1))) ).toList(); // check each scan result value has one event From ec470288c7b725f5310bcf69d1db9f85ff509c8d Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 11:01:35 -0700 Subject: [PATCH 78/91] Fixed failing tests -> allow usage of all types of segment spec --- .../query/scan/ScanQueryRunnerFactory.java | 41 +++++++++---------- .../scan/ScanQueryRunnerFactoryTest.java | 3 +- 2 files changed, 21 insertions(+), 23 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index dbad5fb00703..e88389952207 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -38,8 +38,6 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.Segment; import org.joda.time.Interval; @@ -67,6 +65,7 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory mergeRunners( return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - List descriptorsOrdered = - ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order + List intervalsOrdered = + query.getQuerySegmentSpec().getIntervals(); // Ascending time order List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - descriptorsOrdered = Lists.reverse(descriptorsOrdered); + intervalsOrdered = Lists.reverse(intervalsOrdered); queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } @@ -127,31 +126,31 @@ public QueryRunner mergeRunners( input -> input.run(queryPlus, responseContext) )), query, - descriptorsOrdered + intervalsOrdered ); } else { Preconditions.checkState( - descriptorsOrdered.size() == queryRunnersOrdered.size(), - "Number of segment descriptors does not equal number of " + intervalsOrdered.size() == queryRunnersOrdered.size(), + "Number of intervals from the query segment spec does not equal number of " + "query runners...something went wrong!" ); // Combine the two lists of segment descriptors and query runners into a single list of // segment descriptors - query runner pairs. This makes it easier to use stream operators. - List>> descriptorsAndRunnersOrdered = new ArrayList<>(); + List>> intervalsAndRunnersOrdered = new ArrayList<>(); for (int i = 0; i < queryRunnersOrdered.size(); i++) { - descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); + intervalsAndRunnersOrdered.add(new Pair<>(intervalsOrdered.get(i), queryRunnersOrdered.get(i))); } // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the // query runners for that segment - LinkedHashMap>>> partitionsGroupedByInterval = - descriptorsAndRunnersOrdered.stream() - .collect(Collectors.groupingBy( - x -> x.lhs.getInterval(), - LinkedHashMap::new, - Collectors.toList() - )); + LinkedHashMap>>> partitionsGroupedByInterval = + intervalsAndRunnersOrdered.stream() + .collect(Collectors.groupingBy( + x -> x.lhs, + LinkedHashMap::new, + Collectors.toList() + )); // Find the segment with the largest numbers of partitions. This will be used to compare with the // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. @@ -224,7 +223,7 @@ public QueryRunner mergeRunners( Sequence sortAndLimitScanResultValuesPriorityQueue( Sequence inputSequence, ScanQuery scanQuery, - List descriptorsOrdered + List intervalsOrdered ) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -267,9 +266,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i // Finish scanning the interval containing the limit row if (numRowsScanned > limit && finalInterval == null) { long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (SegmentDescriptor descriptor : descriptorsOrdered) { - if (descriptor.getInterval().contains(timestampOfLimitRow)) { - finalInterval = descriptor.getInterval(); + for (Interval interval : intervalsOrdered) { + if (interval.contains(timestampOfLimitRow)) { + finalInterval = interval; } } if (finalInterval == null) { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index cbf3a9d5cf6c..2287b45b5885 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -26,7 +26,6 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.query.SegmentDescriptor; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -124,7 +123,7 @@ public void testSortAndLimitScanResultValues() factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, - ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) + ImmutableList.of(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1))) ).toList(); // check each scan result value has one event From 8f01d8dd16f40d10c60519ca0ec0d2e6b2dde941 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 13:13:32 -0700 Subject: [PATCH 79/91] Revert "Fixed failing tests -> allow usage of all types of segment spec" This reverts commit ec470288c7b725f5310bcf69d1db9f85ff509c8d. --- .../query/scan/ScanQueryRunnerFactory.java | 41 ++++++++++--------- .../scan/ScanQueryRunnerFactoryTest.java | 3 +- 2 files changed, 23 insertions(+), 21 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index e88389952207..dbad5fb00703 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -38,6 +38,8 @@ import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.Segment; import org.joda.time.Interval; @@ -65,7 +67,6 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory mergeRunners( return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - List intervalsOrdered = - query.getQuerySegmentSpec().getIntervals(); // Ascending time order + List descriptorsOrdered = + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - intervalsOrdered = Lists.reverse(intervalsOrdered); + descriptorsOrdered = Lists.reverse(descriptorsOrdered); queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } @@ -126,31 +127,31 @@ public QueryRunner mergeRunners( input -> input.run(queryPlus, responseContext) )), query, - intervalsOrdered + descriptorsOrdered ); } else { Preconditions.checkState( - intervalsOrdered.size() == queryRunnersOrdered.size(), - "Number of intervals from the query segment spec does not equal number of " + descriptorsOrdered.size() == queryRunnersOrdered.size(), + "Number of segment descriptors does not equal number of " + "query runners...something went wrong!" ); // Combine the two lists of segment descriptors and query runners into a single list of // segment descriptors - query runner pairs. This makes it easier to use stream operators. - List>> intervalsAndRunnersOrdered = new ArrayList<>(); + List>> descriptorsAndRunnersOrdered = new ArrayList<>(); for (int i = 0; i < queryRunnersOrdered.size(); i++) { - intervalsAndRunnersOrdered.add(new Pair<>(intervalsOrdered.get(i), queryRunnersOrdered.get(i))); + descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); } // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the // query runners for that segment - LinkedHashMap>>> partitionsGroupedByInterval = - intervalsAndRunnersOrdered.stream() - .collect(Collectors.groupingBy( - x -> x.lhs, - LinkedHashMap::new, - Collectors.toList() - )); + LinkedHashMap>>> partitionsGroupedByInterval = + descriptorsAndRunnersOrdered.stream() + .collect(Collectors.groupingBy( + x -> x.lhs.getInterval(), + LinkedHashMap::new, + Collectors.toList() + )); // Find the segment with the largest numbers of partitions. This will be used to compare with the // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. @@ -223,7 +224,7 @@ public QueryRunner mergeRunners( Sequence sortAndLimitScanResultValuesPriorityQueue( Sequence inputSequence, ScanQuery scanQuery, - List intervalsOrdered + List descriptorsOrdered ) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -266,9 +267,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i // Finish scanning the interval containing the limit row if (numRowsScanned > limit && finalInterval == null) { long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (Interval interval : intervalsOrdered) { - if (interval.contains(timestampOfLimitRow)) { - finalInterval = interval; + for (SegmentDescriptor descriptor : descriptorsOrdered) { + if (descriptor.getInterval().contains(timestampOfLimitRow)) { + finalInterval = descriptor.getInterval(); } } if (finalInterval == null) { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 2287b45b5885..cbf3a9d5cf6c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -26,6 +26,7 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.SegmentDescriptor; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Test; @@ -123,7 +124,7 @@ public void testSortAndLimitScanResultValues() factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, - ImmutableList.of(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1))) + ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) ).toList(); // check each scan result value has one event From b822fc73dfba7f69c7e960bb95b31cab8d27ef25 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 13:19:02 -0700 Subject: [PATCH 80/91] Revert "Merge branch '6088-Time-Ordering-On-Scans-N-Way-Merge' of github.com:justinborromeo/incubator-druid into 6088-Time-Ordering-On-Scans-N-Way-Merge" This reverts commit 57033f36df6e3ef887e5f0399ad74bb091306de8, reversing changes made to 8f01d8dd16f40d10c60519ca0ec0d2e6b2dde941. --- .../query/scan/ScanQueryRunnerFactory.java | 31 +++++++++---------- .../scan/ScanQueryRunnerFactoryTest.java | 2 +- 2 files changed, 16 insertions(+), 17 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 8ca9ce55da84..dbad5fb00703 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -67,7 +67,6 @@ public class ScanQueryRunnerFactory implements QueryRunnerFactory mergeRunners( return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - List intervalsOrdered = - query.getQuerySegmentSpec().getIntervals(); // Ascending time order + List descriptorsOrdered = + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - intervalsOrdered = Lists.reverse(intervalsOrdered); + descriptorsOrdered = Lists.reverse(descriptorsOrdered); queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } @@ -128,28 +127,28 @@ public QueryRunner mergeRunners( input -> input.run(queryPlus, responseContext) )), query, - intervalsOrdered + descriptorsOrdered ); } else { Preconditions.checkState( - intervalsOrdered.size() == queryRunnersOrdered.size(), - "Number of intervals from the query segment spec does not equal number of " + descriptorsOrdered.size() == queryRunnersOrdered.size(), + "Number of segment descriptors does not equal number of " + "query runners...something went wrong!" ); // Combine the two lists of segment descriptors and query runners into a single list of // segment descriptors - query runner pairs. This makes it easier to use stream operators. - List>> intervalsAndRunnersOrdered = new ArrayList<>(); + List>> descriptorsAndRunnersOrdered = new ArrayList<>(); for (int i = 0; i < queryRunnersOrdered.size(); i++) { - intervalsAndRunnersOrdered.add(new Pair<>(intervalsOrdered.get(i), queryRunnersOrdered.get(i))); + descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); } // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the // query runners for that segment - LinkedHashMap>>> partitionsGroupedByInterval = - intervalsAndRunnersOrdered.stream() + LinkedHashMap>>> partitionsGroupedByInterval = + descriptorsAndRunnersOrdered.stream() .collect(Collectors.groupingBy( - x -> x.lhs, + x -> x.lhs.getInterval(), LinkedHashMap::new, Collectors.toList() )); @@ -225,7 +224,7 @@ public QueryRunner mergeRunners( Sequence sortAndLimitScanResultValuesPriorityQueue( Sequence inputSequence, ScanQuery scanQuery, - List intervalsOrdered + List descriptorsOrdered ) { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -268,9 +267,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i // Finish scanning the interval containing the limit row if (numRowsScanned > limit && finalInterval == null) { long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (Interval interval : intervalsOrdered) { - if (interval.contains(timestampOfLimitRow)) { - finalInterval = interval; + for (SegmentDescriptor descriptor : descriptorsOrdered) { + if (descriptor.getInterval().contains(timestampOfLimitRow)) { + finalInterval = descriptor.getInterval(); } } if (finalInterval == null) { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 53e2b2115d55..cbf3a9d5cf6c 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -124,7 +124,7 @@ public void testSortAndLimitScanResultValues() factory.sortAndLimitScanResultValuesPriorityQueue( inputSequence, query, - ImmutableList.of(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1))) + ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) ).toList(); // check each scan result value has one event From da4fc664031debae1dc3b4a0190125e979564aac Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 15:19:45 -0700 Subject: [PATCH 81/91] Check type of segment spec before using for time ordering --- .../query/scan/ScanQueryRunnerFactory.java | 194 +++++++++--------- 1 file changed, 100 insertions(+), 94 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index dbad5fb00703..6f14d70d5276 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -92,15 +92,6 @@ public QueryRunner mergeRunners( return (queryPlus, responseContext) -> { ScanQuery query = (ScanQuery) queryPlus.getQuery(); - List descriptorsOrdered = - ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order - List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default - - if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { - descriptorsOrdered = Lists.reverse(descriptorsOrdered); - queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); - } - // Note: this variable is effective only when queryContext has a timeout. // See the comment of CTX_TIMEOUT_AT. final long timeoutAt = System.currentTimeMillis() + QueryContexts.getTimeout(queryPlus.getQuery()); @@ -119,103 +110,118 @@ public QueryRunner mergeRunners( } else { return returnedRows; } - } else if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { - // Use priority queue strategy - return sortAndLimitScanResultValuesPriorityQueue( - Sequences.concat(Sequences.map( - Sequences.simple(queryRunnersOrdered), - input -> input.run(queryPlus, responseContext) - )), - query, - descriptorsOrdered - ); } else { - Preconditions.checkState( - descriptorsOrdered.size() == queryRunnersOrdered.size(), - "Number of segment descriptors does not equal number of " - + "query runners...something went wrong!" - ); + if (!(query.getQuerySegmentSpec() instanceof MultipleSpecificSegmentSpec)) { + throw new UOE("Time-ordering on scan queries is only supported for queries with segment specs" + + "of type MultipleSpecificSegmentSpec"); + } + List descriptorsOrdered = + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order + List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default - // Combine the two lists of segment descriptors and query runners into a single list of - // segment descriptors - query runner pairs. This makes it easier to use stream operators. - List>> descriptorsAndRunnersOrdered = new ArrayList<>(); - for (int i = 0; i < queryRunnersOrdered.size(); i++) { - descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { + descriptorsOrdered = Lists.reverse(descriptorsOrdered); + queryRunnersOrdered = Lists.reverse(queryRunnersOrdered); } - // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the - // query runners for that segment - LinkedHashMap>>> partitionsGroupedByInterval = - descriptorsAndRunnersOrdered.stream() - .collect(Collectors.groupingBy( - x -> x.lhs.getInterval(), - LinkedHashMap::new, - Collectors.toList() - )); + if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { + // Use priority queue strategy + return sortAndLimitScanResultValuesPriorityQueue( + Sequences.concat(Sequences.map( + Sequences.simple(queryRunnersOrdered), + input -> input.run(queryPlus, responseContext) + )), + query, + descriptorsOrdered + ); + } else { + Preconditions.checkState( + descriptorsOrdered.size() == queryRunnersOrdered.size(), + "Number of segment descriptors does not equal number of " + + "query runners...something went wrong!" + ); - // Find the segment with the largest numbers of partitions. This will be used to compare with the - // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. - int maxNumPartitionsInSegment = - partitionsGroupedByInterval.values() - .stream() - .map(x -> x.size()) - .max(Comparator.comparing(Integer::valueOf)) - .get(); + // Combine the two lists of segment descriptors and query runners into a single list of + // segment descriptors - query runner pairs. This makes it easier to use stream operators. + List>> descriptorsAndRunnersOrdered = new ArrayList<>(); + for (int i = 0; i < queryRunnersOrdered.size(); i++) { + descriptorsAndRunnersOrdered.add(new Pair<>(descriptorsOrdered.get(i), queryRunnersOrdered.get(i))); + } - if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory()) { - // Use n-way merge strategy + // Group the list of pairs by interval. The LinkedHashMap will have an interval paired with a list of all the + // query runners for that segment + LinkedHashMap>>> partitionsGroupedByInterval = + descriptorsAndRunnersOrdered.stream() + .collect(Collectors.groupingBy( + x -> x.lhs.getInterval(), + LinkedHashMap::new, + Collectors.toList() + )); - // Create a list of grouped runner lists (i.e. each sublist/"runner group" corresponds to an interval) -> - // there should be no interval overlap. We create a list of lists so we can create a sequence of sequences. - // There's no easy way to convert a LinkedHashMap to a sequence because it's non-iterable. - List>> groupedRunners = - partitionsGroupedByInterval.entrySet() + // Find the segment with the largest numbers of partitions. This will be used to compare with the + // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. + int maxNumPartitionsInSegment = + partitionsGroupedByInterval.values() .stream() - .map(entry -> entry.getValue() - .stream() - .map(segQueryRunnerPair -> segQueryRunnerPair.rhs) - .collect(Collectors.toList())) - .collect(Collectors.toList()); + .map(x -> x.size()) + .max(Comparator.comparing(Integer::valueOf)) + .get(); + + if (maxNumPartitionsInSegment <= scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory()) { + // Use n-way merge strategy - // Starting from the innermost Sequences.map: - // (1) Deaggregate each ScanResultValue returned by the query runners - // (2) Combine the deaggregated ScanResultValues into a single sequence - // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp - // (4) Create a sequence of results from each runner group - // (5) Join all the results into a single sequence + // Create a list of grouped runner lists (i.e. each sublist/"runner group" corresponds to an interval) -> + // there should be no interval overlap. We create a list of lists so we can create a sequence of sequences. + // There's no easy way to convert a LinkedHashMap to a sequence because it's non-iterable. + List>> groupedRunners = + partitionsGroupedByInterval.entrySet() + .stream() + .map(entry -> entry.getValue() + .stream() + .map(segQueryRunnerPair -> segQueryRunnerPair.rhs) + .collect(Collectors.toList())) + .collect(Collectors.toList()); - return Sequences.concat( - Sequences.map( - Sequences.simple(groupedRunners), - runnerGroup -> - Sequences.map( - Sequences.simple(runnerGroup), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) - ) - ) - ).flatMerge( - seq -> seq, - Ordering.from(new ScanResultValueTimestampComparator( - query - )).reverse() - ) - ) - ).limit( - Math.toIntExact(query.getLimit()) + // Starting from the innermost Sequences.map: + // (1) Deaggregate each ScanResultValue returned by the query runners + // (2) Combine the deaggregated ScanResultValues into a single sequence + // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp + // (4) Create a sequence of results from each runner group + // (5) Join all the results into a single sequence + + return Sequences.concat( + Sequences.map( + Sequences.simple(groupedRunners), + runnerGroup -> + Sequences.map( + Sequences.simple(runnerGroup), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) + ) + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + query + )).reverse() + ) + ) + ).limit( + Math.toIntExact(query.getLimit()) + ); + } + throw new UOE( + "Time ordering for queries of %,d partitions per segment and a row limit of %,d is not supported." + + " Try reducing the scope of the query to scan fewer partitions than the configurable limit of" + + " %,d partitions or lower the row limit below %,d.", + maxNumPartitionsInSegment, + query.getLimit(), + scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory(), + scanQueryConfig.getMaxRowsQueuedForOrdering() ); } - throw new UOE( - "Time ordering for queries of %,d partitions per segment and a row limit of %,d is not supported." - + " Try reducing the scope of the query to scan fewer partitions than the configurable limit of" - + " %,d partitions or lower the row limit below %,d.", - maxNumPartitionsInSegment, - query.getLimit(), - scanQueryConfig.getMaxSegmentPartitionsOrderedInMemory(), - scanQueryConfig.getMaxRowsQueuedForOrdering() - ); } }; } From 219af478c8ec243700973e616c5be556a83422e2 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 15:57:55 -0700 Subject: [PATCH 82/91] Fix bug in numRowsScanned --- .../org/apache/druid/query/scan/ScanQueryRunnerFactory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 6f14d70d5276..034a0d2e36f3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -259,9 +259,9 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i Interval finalInterval = null; while (!doneScanning) { ScanResultValue next = yielder.get(); - numRowsScanned++; List singleEventScanResultValues = next.toSingleEventScanResultValues(); for (ScanResultValue srv : singleEventScanResultValues) { + numRowsScanned++; // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list // needs to be preserved for queries using the compactedList result format q.offer(srv); From 35692680fc7aba21c498a92307ef082a581cb23a Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 16:15:49 -0700 Subject: [PATCH 83/91] Fix bug messing up count of rows --- .../query/scan/ScanQueryRunnerFactory.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 034a0d2e36f3..7078b9f915ab 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -268,20 +268,21 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i if (q.size() > limit) { q.poll(); } - } - yielder = yielder.next(null); - // Finish scanning the interval containing the limit row - if (numRowsScanned > limit && finalInterval == null) { - long timestampOfLimitRow = next.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (SegmentDescriptor descriptor : descriptorsOrdered) { - if (descriptor.getInterval().contains(timestampOfLimitRow)) { - finalInterval = descriptor.getInterval(); + + // Finish scanning the interval containing the limit row + if (numRowsScanned > limit && finalInterval == null) { + long timestampOfLimitRow = srv.getFirstEventTimestamp(scanQuery.getResultFormat()); + for (SegmentDescriptor descriptor : descriptorsOrdered) { + if (descriptor.getInterval().contains(timestampOfLimitRow)) { + finalInterval = descriptor.getInterval(); + } + } + if (finalInterval == null) { + throw new ISE("WTH??? Row came from an unscanned interval?"); } - } - if (finalInterval == null) { - throw new ISE("WTH??? Row came from an unscanned interval?"); } } + yielder = yielder.next(null); doneScanning = yielder.isDone() || (finalInterval != null && !finalInterval.contains(next.getFirstEventTimestamp(scanQuery.getResultFormat()))); From 8a6bb1127c1814470424da2e9d6bfdd55e726199 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Mon, 25 Mar 2019 17:17:41 -0700 Subject: [PATCH 84/91] Fix docs and flipped boolean in ScanQueryLimitRowIterator --- docs/content/querying/scan-query.md | 10 +++++----- .../druid/query/scan/ScanQueryLimitRowIterator.java | 9 +++++---- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index fd43fc092ebd..1bb67158de99 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -158,7 +158,7 @@ The format of the result when resultFormat equals `compactedList`: The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` -rows **or** fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` segments are scanned per Historical. The +rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. The reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set limit and the number of segments being scanned. @@ -170,12 +170,12 @@ priority queue are streamed back to the Broker(s) in batches. Attempting to loa risk of Historical nodes running out of memory. The `druid.query.scan.maxRowsQueuedForOrdering` property protects from this by limiting the number of rows in the query result set when time ordering is used. -2. N-Way Merge: Each segment on a Historical is opened in parallel. Since each segment's rows are already -time-ordered, an n-way merge can be performed on the results from each segment. This approach doesn't persist the entire +2. N-Way Merge: For each segment, each partition is opened in parallel. Since each partition's rows are already +time-ordered, an n-way merge can be performed on the results from each partition. This approach doesn't persist the entire result set in memory (like the Priority Queue) as it streams back batches as they are returned from the merge function. -However, attempting to query too many segments could also result in high memory usage due to the need to open +However, attempting to query too many partition could also result in high memory usage due to the need to open decompression and decoding buffers for each. The `druid.query.scan.maxSegmentPartitionsOrderedInMemory` limit protects -from this by capping the number of segments opened per historical when time ordering is used. +from this by capping the number of partitions opened at any times when time ordering is used. Both `druid.query.scan.maxRowsQueuedForOrdering` and `druid.query.scan.maxSegmentPartitionsOrderedInMemory` are configurable and can be tuned based on hardware specs and number of dimensions being queried. diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 4103b1476978..7165de4134ef 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -97,9 +97,10 @@ public ScanResultValue next() throw new UOE(ScanQuery.ResultFormat.RESULT_FORMAT_VALUE_VECTOR + " is not supported yet"); } - // We want to perform batching if we are not time-ordering or are at the outer level if we are re time-ordering + // We want to perform multi-event ScanResultValue limiting if we are not time-ordering or are at the + // outer level if we are time-ordering if (query.getOrder() == ScanQuery.Order.NONE || - !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { + query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); List events = (List) batch.getEvents(); if (events.size() <= limit - count) { @@ -114,8 +115,8 @@ public ScanResultValue next() return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, numLeft)); } } else { - // Perform single-event ScanResultValue batching. Each scan result value in this case will only have one event - // so there's no need to iterate through events. + // Perform single-event ScanResultValue batching. Each scan result value from the yielder in this case will only + // have one event so there's no need to iterate through events. int batchSize = query.getBatchSize(); List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); From 376e8bf90610d43d2c7b278bf64525cab80267c5 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Mar 2019 11:42:54 -0700 Subject: [PATCH 85/91] Refactor n-way merge --- .../druid/query/scan/ScanQueryEngine.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 71 +++++++++++-------- .../scan/ScanQueryRunnerFactoryTest.java | 8 ++- 3 files changed, 48 insertions(+), 33 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java index ad38c87e3289..2a5edbd14d34 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryEngine.java @@ -259,7 +259,7 @@ public void cleanup(Iterator iterFromMake) } /** - * If we're performing time-ordering, we want to scan through the first `limit` rows ignoring the number + * If we're performing time-ordering, we want to scan through the first `limit` rows in each segment ignoring the number * of rows already counted on other segments. */ private long calculateLimit(ScanQuery query, Map responseContext) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 7078b9f915ab..600fdb04fdfc 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -126,7 +126,7 @@ public QueryRunner mergeRunners( if (query.getLimit() <= scanQueryConfig.getMaxRowsQueuedForOrdering()) { // Use priority queue strategy - return sortAndLimitScanResultValuesPriorityQueue( + return priorityQueueSortAndLimit( Sequences.concat(Sequences.map( Sequences.simple(queryRunnersOrdered), input -> input.run(queryPlus, responseContext) @@ -182,35 +182,7 @@ public QueryRunner mergeRunners( .collect(Collectors.toList())) .collect(Collectors.toList()); - // Starting from the innermost Sequences.map: - // (1) Deaggregate each ScanResultValue returned by the query runners - // (2) Combine the deaggregated ScanResultValues into a single sequence - // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp - // (4) Create a sequence of results from each runner group - // (5) Join all the results into a single sequence - - return Sequences.concat( - Sequences.map( - Sequences.simple(groupedRunners), - runnerGroup -> - Sequences.map( - Sequences.simple(runnerGroup), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) - ) - ) - ).flatMerge( - seq -> seq, - Ordering.from(new ScanResultValueTimestampComparator( - query - )).reverse() - ) - ) - ).limit( - Math.toIntExact(query.getLimit()) - ); + return nWayMergeAndLimit(groupedRunners, queryPlus, responseContext); } throw new UOE( "Time ordering for queries of %,d partitions per segment and a row limit of %,d is not supported." @@ -227,7 +199,7 @@ public QueryRunner mergeRunners( } @VisibleForTesting - Sequence sortAndLimitScanResultValuesPriorityQueue( + Sequence priorityQueueSortAndLimit( Sequence inputSequence, ScanQuery scanQuery, List descriptorsOrdered @@ -297,6 +269,43 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i return Sequences.simple(sortedElements); } + @VisibleForTesting + Sequence nWayMergeAndLimit( + List>> groupedRunners, + QueryPlus queryPlus, + Map responseContext + ) + { + // Starting from the innermost Sequences.map: + // (1) Deaggregate each ScanResultValue returned by the query runners + // (2) Combine the deaggregated ScanResultValues into a single sequence + // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp + // (4) Create a sequence of results from each runner group + // (5) Join all the results into a single sequence + + return Sequences.concat( + Sequences.map( + Sequences.simple(groupedRunners), + runnerGroup -> + Sequences.map( + Sequences.simple(runnerGroup), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) + ) + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + (ScanQuery) queryPlus.getQuery() + )).reverse() + ) + ) + ).limit( + Math.toIntExact(((ScanQuery) (queryPlus.getQuery())).getLimit()) + ); + } @Override public QueryToolChest getToolchest() diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index cbf3a9d5cf6c..08abc595b218 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -121,7 +121,7 @@ public void testSortAndLimitScanResultValues() }); Sequence inputSequence = Sequences.simple(srvs); List output = - factory.sortAndLimitScanResultValuesPriorityQueue( + factory.priorityQueueSortAndLimit( inputSequence, query, ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) @@ -155,4 +155,10 @@ public void testSortAndLimitScanResultValues() Assert.assertEquals((long) expectedEventTimestamps.get(i), output.get(i).getFirstEventTimestamp(resultFormat)); } } + + @Test + public void testNWayMerge() + { + + } } From fb858efbb75218bb80b8c77effb2456554aa57b2 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Mar 2019 13:14:48 -0700 Subject: [PATCH 86/91] Added test for n-way merge --- .../query/scan/ScanQueryQueryToolChest.java | 3 + .../scan/ScanQueryRunnerFactoryTest.java | 100 ++++++++++++++++-- 2 files changed, 96 insertions(+), 7 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index f313e2ed5d2c..6d6758b19260 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -61,6 +61,9 @@ public QueryRunner mergeResults(final QueryRunner queryPlusWithNonNullLegacy = queryPlus.withQuery(scanQuery); + if (scanQuery.getLimit() == Long.MAX_VALUE) { + return runner.run(queryPlusWithNonNullLegacy, responseContext); + } return new BaseSequence<>( new BaseSequence.IteratorMaker() { diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 08abc595b218..ef8dc7ff6a0a 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -20,11 +20,14 @@ package org.apache.druid.query.scan; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.SegmentDescriptor; import org.joda.time.Interval; @@ -34,7 +37,10 @@ import org.junit.runners.Parameterized; import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; +import java.util.Map; @RunWith(Parameterized.class) @@ -124,9 +130,95 @@ public void testSortAndLimitScanResultValues() factory.priorityQueueSortAndLimit( inputSequence, query, - ImmutableList.of(new SegmentDescriptor(new Interval(DateTimes.of("2010-01-01"), DateTimes.of("2019-01-01").plusHours(1)), "1", 0)) + ImmutableList.of(new SegmentDescriptor(new Interval( + DateTimes.of("2010-01-01"), + DateTimes.of("2019-01-01").plusHours(1) + ), "1", 0)) ).toList(); + validateSortedOutput(output, expectedEventTimestamps); + } + + @Test + public void testNWayMerge() + { + List expectedEventTimestamps = new ArrayList<>(numElements * 3); + + List scanResultValues1 = new ArrayList<>(numElements); + for (int i = 0; i < numElements; i++) { + long timestamp = DateTimes.of("2015-01-01").plusMinutes(i * 2).getMillis(); + expectedEventTimestamps.add(timestamp); + scanResultValues1.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); + } + + List scanResultValues2 = new ArrayList<>(numElements); + for (int i = 0; i < numElements; i++) { + long timestamp = DateTimes.of("2015-01-01").plusMinutes(i * 2 + 1).getMillis(); + expectedEventTimestamps.add(timestamp); + scanResultValues2.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); + } + + List scanResultValues3 = new ArrayList<>(numElements); + for (int i = 0; i < numElements; i++) { + long timestamp = DateTimes.of("2015-01-02").plusMinutes(i).getMillis(); + expectedEventTimestamps.add(timestamp); + scanResultValues3.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); + } + + if ( query.getOrder() == ScanQuery.Order.DESCENDING) { + Collections.reverse(scanResultValues1); + Collections.reverse(scanResultValues2); + Collections.reverse(scanResultValues3); + } + + QueryRunner runnerSegment1Partition1 = + (queryPlus, responseContext) -> Sequences.simple(scanResultValues1); + + QueryRunner runnerSegment1Partition2 = + (queryPlus, responseContext) -> Sequences.simple(scanResultValues2); + + + QueryRunner runnerSegment2Partition1 = + (queryPlus, responseContext) -> Sequences.simple(scanResultValues3); + + QueryRunner runnerSegment2Partition2 = + (queryPlus, responseContext) -> Sequences.empty(); + + List>> groupedRunners = new ArrayList<>(2); + + if (query.getOrder() == ScanQuery.Order.DESCENDING) { + groupedRunners.add(Arrays.asList(runnerSegment2Partition1, runnerSegment2Partition2)); + groupedRunners.add(Arrays.asList(runnerSegment1Partition1, runnerSegment1Partition2)); + } else { + groupedRunners.add(Arrays.asList(runnerSegment1Partition1, runnerSegment1Partition2)); + groupedRunners.add(Arrays.asList(runnerSegment2Partition1, runnerSegment2Partition2)); + } + + expectedEventTimestamps.sort((o1, o2) -> { + int retVal = 0; + if (o1 > o2) { + retVal = 1; + } else if (o1 < o2) { + retVal = -1; + } + if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { + return retVal * -1; + } + return retVal; + }); + + List output = + factory.nWayMergeAndLimit( + groupedRunners, + QueryPlus.wrap(query), + ImmutableMap.of() + ).toList(); + + validateSortedOutput(output, expectedEventTimestamps); + } + + private void validateSortedOutput(List output, List expectedEventTimestamps) + { // check each scan result value has one event for (ScanResultValue srv : output) { if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) { @@ -155,10 +247,4 @@ public void testSortAndLimitScanResultValues() Assert.assertEquals((long) expectedEventTimestamps.get(i), output.get(i).getFirstEventTimestamp(resultFormat)); } } - - @Test - public void testNWayMerge() - { - - } } From 487f31fcf63a5e1fa9e802212b62206aec47fe25 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Mar 2019 14:39:25 -0700 Subject: [PATCH 87/91] Refixed regression --- .../query/scan/ScanQueryLimitRowIterator.java | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index 7165de4134ef..f8c76d03da03 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -45,10 +45,10 @@ * 1) No time ordering: expects a Sequence of ScanResultValues which each contain up to query.batchSize events. * The iterator will be "done" when the limit of events is reached. The final ScanResultValue might contain * fewer than batchSize events so that the limit number of events is returned. - * 2) Time Ordering, CTX_KEY_OUTERMOST==null or true: Same behaviour as no time ordering - * 3) Time Ordering, CTX_KEY_OUTERMOST=false: The Sequence returned in this case should contain ScanResultValues - * that contain only one event each. This iterator will perform batching according to query.batchSize until - * the limit is reached. + * 2) Time Ordering, CTX_KEY_OUTERMOST false: Same behaviour as no time ordering + * 3) Time Ordering, CTX_KEY_OUTERMOST=true or null: The Sequence processed in this case should contain ScanResultValues + * that contain only one event each for the CachingClusteredClient n-way merge. This iterator will perform + * batching according to query batch size until the limit is reached. */ public class ScanQueryLimitRowIterator implements CloseableIterator { @@ -98,9 +98,9 @@ public ScanResultValue next() } // We want to perform multi-event ScanResultValue limiting if we are not time-ordering or are at the - // outer level if we are time-ordering + // inner-level if we are time-ordering if (query.getOrder() == ScanQuery.Order.NONE || - query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { + !query.getContextBoolean(ScanQuery.CTX_KEY_OUTERMOST, true)) { ScanResultValue batch = yielder.get(); List events = (List) batch.getEvents(); if (events.size() <= limit - count) { @@ -115,8 +115,8 @@ public ScanResultValue next() return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, numLeft)); } } else { - // Perform single-event ScanResultValue batching. Each scan result value from the yielder in this case will only - // have one event so there's no need to iterate through events. + // Perform single-event ScanResultValue batching at the outer level. Each scan result value from the yielder + // in this case will only have one event so there's no need to iterate through events. int batchSize = query.getBatchSize(); List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); From 480e932fdf02ef85ba81181deb865d9977dfed24 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Tue, 26 Mar 2019 14:58:04 -0700 Subject: [PATCH 88/91] Checkstyle and doc update --- docs/content/querying/scan-query.md | 3 ++- .../org/apache/druid/query/scan/ScanQueryLimitRowIterator.java | 1 - .../apache/druid/query/scan/ScanQueryRunnerFactoryTest.java | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index 1bb67158de99..cd32e51378b5 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -158,7 +158,8 @@ The format of the result when resultFormat equals `compactedList`: The Scan query currently supports ordering based on timestamp for non-legacy queries. Note that using time ordering will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` -rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. The +rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, +time ordering is not support for queries issued directly to historicals unless a list of segments is specified. The reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set limit and the number of segments being scanned. diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java index f8c76d03da03..f246d7c5fcf3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryLimitRowIterator.java @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; - /** * This iterator supports iteration through a Sequence returned by a ScanResultValue QueryRunner. Its behaviour * varies depending on whether the query is returning time-ordered values and whether the CTX_KEY_OUTERMOST flag is diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index ef8dc7ff6a0a..70cf77098095 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -40,7 +40,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Map; @RunWith(Parameterized.class) @@ -165,7 +164,7 @@ public void testNWayMerge() scanResultValues3.add(ScanQueryTestHelper.generateScanResultValue(timestamp, resultFormat, 1)); } - if ( query.getOrder() == ScanQuery.Order.DESCENDING) { + if (query.getOrder() == ScanQuery.Order.DESCENDING) { Collections.reverse(scanResultValues1); Collections.reverse(scanResultValues2); Collections.reverse(scanResultValues3); From 231a72e7d9c0f4bb2b3272134cf53fc8db8f0e73 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 27 Mar 2019 17:38:20 -0700 Subject: [PATCH 89/91] Modified sequence limit to accept longs and added test for long limits --- .../util/common/guava/LimitedSequence.java | 6 ++-- .../java/util/common/guava/Sequence.java | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 9 +++-- .../scan/ScanQueryRunnerFactoryTest.java | 33 ++++++++++++------- 4 files changed, 32 insertions(+), 18 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/LimitedSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/LimitedSequence.java index 9f76e46c8d31..784161dd0edc 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/LimitedSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/LimitedSequence.java @@ -30,11 +30,11 @@ final class LimitedSequence extends YieldingSequenceBase { private final Sequence baseSequence; - private final int limit; + private final long limit; LimitedSequence( Sequence baseSequence, - int limit + long limit ) { Preconditions.checkNotNull(baseSequence); @@ -106,7 +106,7 @@ public void close() throws IOException private class LimitedYieldingAccumulator extends DelegatingYieldingAccumulator { - int count; + long count; boolean interruptYield = false; LimitedYieldingAccumulator(YieldingAccumulator accumulator) diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java index cd074b1e8590..555c768df68a 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/Sequence.java @@ -82,7 +82,7 @@ default List toList() return accumulate(new ArrayList<>(), Accumulators.list()); } - default Sequence limit(int limit) + default Sequence limit(long limit) { return new LimitedSequence<>(this, limit); } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 600fdb04fdfc..5ad510874bc3 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -207,6 +207,12 @@ Sequence priorityQueueSortAndLimit( { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); + if (scanQuery.getLimit() > Integer.MAX_VALUE) { + throw new UOE("Limit of %,d rows not supported for priority queue strategy of time-ordering scan results", + scanQuery.getLimit() + ); + } + // Converting the limit from long to int could theoretically throw an ArithmeticException but this branch // only runs if limit < MAX_LIMIT_FOR_IN_MEMORY_TIME_ORDERING (which should be < Integer.MAX_VALUE) int limit = Math.toIntExact(scanQuery.getLimit()); @@ -282,7 +288,6 @@ Sequence nWayMergeAndLimit( // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp // (4) Create a sequence of results from each runner group // (5) Join all the results into a single sequence - return Sequences.concat( Sequences.map( Sequences.simple(groupedRunners), @@ -303,7 +308,7 @@ Sequence nWayMergeAndLimit( ) ) ).limit( - Math.toIntExact(((ScanQuery) (queryPlus.getQuery())).getLimit()) + ((ScanQuery) (queryPlus.getQuery())).getLimit() ); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java index 70cf77098095..a7cf3c60e3fd 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryRunnerFactoryTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; @@ -83,7 +84,7 @@ public static Iterable constructorFeeder() { List numsElements = ImmutableList.of(0, 10, 100); List batchSizes = ImmutableList.of(1, 100); - List limits = ImmutableList.of(3L, 1000L); + List limits = ImmutableList.of(3L, 1000L, Long.MAX_VALUE); List resultFormats = ImmutableList.of( ScanQuery.ResultFormat.RESULT_FORMAT_LIST, ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST @@ -125,17 +126,25 @@ public void testSortAndLimitScanResultValues() return retVal; }); Sequence inputSequence = Sequences.simple(srvs); - List output = - factory.priorityQueueSortAndLimit( - inputSequence, - query, - ImmutableList.of(new SegmentDescriptor(new Interval( - DateTimes.of("2010-01-01"), - DateTimes.of("2019-01-01").plusHours(1) - ), "1", 0)) - ).toList(); - - validateSortedOutput(output, expectedEventTimestamps); + try { + List output = factory.priorityQueueSortAndLimit( + inputSequence, + query, + ImmutableList.of(new SegmentDescriptor(new Interval( + DateTimes.of("2010-01-01"), + DateTimes.of("2019-01-01").plusHours(1) + ), "1", 0)) + ).toList(); + if (query.getLimit() > Integer.MAX_VALUE) { + Assert.fail("Unsupported exception should have been thrown due to high limit"); + } + validateSortedOutput(output, expectedEventTimestamps); + } + catch (UOE e) { + if (query.getLimit() <= Integer.MAX_VALUE) { + Assert.fail("Unsupported operation exception should not have been thrown here"); + } + } } @Test From 07503ea5c00892bf904c0e16e7062fadabcb7830 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 27 Mar 2019 17:49:09 -0700 Subject: [PATCH 90/91] doc fix --- docs/content/querying/scan-query.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index cd32e51378b5..1b97ed1c49d9 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -61,7 +61,7 @@ The following are the main parameters for Scan queries: |columns|A String array of dimensions and metrics to scan. If left empty, all dimensions and metrics are returned.|no| |batchSize|How many rows buffered before return to client. Default is `20480`|no| |limit|How many rows to return. If not specified, all rows will be returned.|no| -|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsOrderedInMemory`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsOrderedInMemory` will not be time-ordered and default to a order of "none".|none| +|order|The ordering of returned rows based on timestamp. "ascending", "descending", and "none" (default) are supported. Currently, "ascending" and "descending" are only supported for queries where the limit is less than `druid.query.scan.maxRowsQueuedForOrdering`. Scan queries that are either legacy mode or have a limit greater than `druid.query.scan.maxRowsQueuedForOrdering` will not be time-ordered and default to a order of "none".|none| |legacy|Return results consistent with the legacy "scan-query" contrib extension. Defaults to the value set by `druid.query.scan.legacy`, which in turn defaults to false. See [Legacy mode](#legacy-mode) for details.|no| |context|An additional JSON Object which can be used to specify certain flags.|no| From 287a367f4170e7d0b3010d57788ea993688b9335 Mon Sep 17 00:00:00 2001 From: Justin Borromeo Date: Wed, 27 Mar 2019 20:03:41 -0700 Subject: [PATCH 91/91] Implemented Clint's recommendations --- docs/content/querying/scan-query.md | 2 +- .../query/scan/ScanQueryRunnerFactory.java | 58 +++++++++++-------- 2 files changed, 35 insertions(+), 25 deletions(-) diff --git a/docs/content/querying/scan-query.md b/docs/content/querying/scan-query.md index 1b97ed1c49d9..ee44681b0253 100644 --- a/docs/content/querying/scan-query.md +++ b/docs/content/querying/scan-query.md @@ -159,7 +159,7 @@ The Scan query currently supports ordering based on timestamp for non-legacy que will yield results that do not indicate which segment rows are from (`segmentId` will show up as `null`). Furthermore, time ordering is only supported where the result set limit is less than `druid.query.scan.maxRowsQueuedForOrdering` rows **or** all segments scanned have fewer than `druid.query.scan.maxSegmentPartitionsOrderedInMemory` partitions. Also, -time ordering is not support for queries issued directly to historicals unless a list of segments is specified. The +time ordering is not supported for queries issued directly to historicals unless a list of segments is specified. The reasoning behind these limitations is that the implementation of time ordering uses two strategies that can consume too much heap memory if left unbounded. These strategies (listed below) are chosen on a per-Historical basis depending on query result set limit and the number of segments being scanned. diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 5ad510874bc3..41729f786386 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -111,13 +111,18 @@ public QueryRunner mergeRunners( return returnedRows; } } else { + // Query segment spec must be an instance of MultipleSpecificSegmentSpec because segment descriptors need + // to be present for a 1:1 matching of intervals with query runners. The other types of segment spec condense + // the intervals (i.e. merge neighbouring intervals), eliminating the 1:1 relationship between intervals + // and query runners. if (!(query.getQuerySegmentSpec() instanceof MultipleSpecificSegmentSpec)) { throw new UOE("Time-ordering on scan queries is only supported for queries with segment specs" + "of type MultipleSpecificSegmentSpec"); } + // Ascending time order for both descriptors and query runners by default List descriptorsOrdered = - ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); // Ascending time order - List> queryRunnersOrdered = Lists.newArrayList(queryRunners); // Ascending time order by default + ((MultipleSpecificSegmentSpec) query.getQuerySegmentSpec()).getDescriptors(); + List> queryRunnersOrdered = Lists.newArrayList(queryRunners); if (query.getOrder().equals(ScanQuery.Order.DESCENDING)) { descriptorsOrdered = Lists.reverse(descriptorsOrdered); @@ -208,8 +213,9 @@ Sequence priorityQueueSortAndLimit( Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); if (scanQuery.getLimit() > Integer.MAX_VALUE) { - throw new UOE("Limit of %,d rows not supported for priority queue strategy of time-ordering scan results", - scanQuery.getLimit() + throw new UOE( + "Limit of %,d rows not supported for priority queue strategy of time-ordering scan results", + scanQuery.getLimit() ); } @@ -288,28 +294,32 @@ Sequence nWayMergeAndLimit( // (3) Create a sequence of results from each runner in the group and flatmerge based on timestamp // (4) Create a sequence of results from each runner group // (5) Join all the results into a single sequence - return Sequences.concat( - Sequences.map( - Sequences.simple(groupedRunners), - runnerGroup -> - Sequences.map( - Sequences.simple(runnerGroup), - (input) -> Sequences.concat( - Sequences.map( - input.run(queryPlus, responseContext), - srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + Sequence resultSequence = + Sequences.concat( + Sequences.map( + Sequences.simple(groupedRunners), + runnerGroup -> + Sequences.map( + Sequences.simple(runnerGroup), + (input) -> Sequences.concat( + Sequences.map( + input.run(queryPlus, responseContext), + srv -> Sequences.simple(srv.toSingleEventScanResultValues()) + ) ) + ).flatMerge( + seq -> seq, + Ordering.from(new ScanResultValueTimestampComparator( + (ScanQuery) queryPlus.getQuery() + )).reverse() ) - ).flatMerge( - seq -> seq, - Ordering.from(new ScanResultValueTimestampComparator( - (ScanQuery) queryPlus.getQuery() - )).reverse() - ) - ) - ).limit( - ((ScanQuery) (queryPlus.getQuery())).getLimit() - ); + ) + ); + long limit = ((ScanQuery) (queryPlus.getQuery())).getLimit(); + if (limit == Long.MAX_VALUE) { + return resultSequence; + } + return resultSequence.limit(limit); } @Override