From 1647d5f4a07b76a8bf7cd190c5778ec07e99b3d2 Mon Sep 17 00:00:00 2001 From: Laksh Singla Date: Mon, 26 Jun 2023 18:12:28 +0530 Subject: [PATCH] Limit the subquery results by memory usage (#13952) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Users can now add a guardrail to prevent subquery’s results from exceeding the set number of bytes by setting druid.server.http.maxSubqueryRows in Broker's config or maxSubqueryRows in the query context. This feature is experimental for now and would default back to row-based limiting in case it fails to get the accurate size of the results consumed by the query. --- docs/configuration/index.md | 14 + docs/querying/query-execution.md | 9 +- .../input/inline/InlineInputSliceReader.java | 3 +- .../msq/test/CalciteSelectQueryMSQTest.java | 7 - .../apache/druid/frame/read/FrameReader.java | 4 + .../druid/frame/segment/FrameCursorUtils.java | 68 ++ .../segment/columnar/FrameQueryableIndex.java | 2 +- .../druid/frame/write/FrameWriterUtils.java | 11 + .../DruidDefaultSerializersModule.java | 4 + .../query/FrameBasedInlineDataSource.java | 197 ++++ .../FrameBasedInlineDataSourceSerializer.java | 103 ++ .../druid/query/FrameSignaturePair.java | 52 + .../apache/druid/query/InlineDataSource.java | 1 + .../druid/query/IterableRowsCursorHelper.java | 106 ++ .../org/apache/druid/query/QueryContext.java | 10 + .../org/apache/druid/query/QueryContexts.java | 2 + .../apache/druid/query/QueryToolChest.java | 32 +- .../groupby/GroupByQueryQueryToolChest.java | 45 + .../apache/druid/query/scan/ConcatCursor.java | 328 ++++++ .../druid/query/scan/ScanQueryEngine.java | 17 +- .../query/scan/ScanQueryLimitRowIterator.java | 12 +- .../query/scan/ScanQueryOffsetSequence.java | 2 +- .../query/scan/ScanQueryQueryToolChest.java | 170 +++- .../druid/query/scan/ScanResultValue.java | 33 +- .../TimeseriesQueryQueryToolChest.java | 46 +- .../query/topn/TopNQueryQueryToolChest.java | 44 + .../apache/druid/segment/RowBasedCursor.java | 2 +- .../org/apache/druid/segment/RowWalker.java | 2 +- .../druid/segment/column/RowSignature.java | 3 +- .../join/FrameBasedInlineJoinableFactory.java | 64 ++ .../segment/join/InlineJoinableFactory.java | 5 +- .../join/table/FrameBasedIndexedTable.java | 336 +++++++ ...meBasedInlineDataSourceSerializerTest.java | 156 +++ .../query/IterableRowsCursorHelperTest.java | 82 ++ .../druid/query/scan/ConcatCursorTest.java | 287 ++++++ .../scan/ScanQueryQueryToolChestTest.java | 270 ++++- .../table/FrameBasedIndexedTableTest.java | 372 +++++++ .../druid/guice/JoinableFactoryModule.java | 3 + .../druid/guice/SegmentWranglerModule.java | 3 + .../FrameBasedInlineSegmentWrangler.java | 55 + .../server/ClientQuerySegmentWalker.java | 239 ++++- .../server/ClientQuerySegmentWalkerUtils.java | 40 + .../server/initialization/ServerConfig.java | 29 + .../jetty/CliIndexerServerModule.java | 2 + .../initialization/ServerConfigTest.java | 2 + .../server/ClientQuerySegmentWalkerTest.java | 59 +- .../apache/druid/server/QueryStackTests.java | 5 +- .../sql/calcite/BaseCalciteQueryTest.java | 4 + .../druid/sql/calcite/CalciteQueryTest.java | 775 -------------- .../sql/calcite/CalciteSubqueryTest.java | 945 ++++++++++++++++++ .../DecoupledPlanningCalciteQueryTest.java | 84 -- .../SpecificSegmentsQuerySegmentWalker.java | 3 + 52 files changed, 4206 insertions(+), 943 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java create mode 100644 processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java create mode 100644 processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java create mode 100644 processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java create mode 100644 processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java create mode 100644 processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java create mode 100644 processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java create mode 100644 processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java create mode 100644 server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java create mode 100644 server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java create mode 100644 sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 37f237d75682..8c8ba45b106f 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1840,6 +1840,19 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h |--------|-----------|-------| |`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be an integer in the range 1 to 100, and will be rounded up|No default, must be set if using this mode| +##### Guardrails for materialization of subqueries +Druid stores the subquery rows in temporary tables that live in the Java heap. It is a good practice to avoid large subqueries in Druid. +Therefore there are guardrails that are built in Druid to prevent the queries from generating subquery results which can exhaust the heap +space. They can be set on a cluster level or modified per query level as desired. +Note the following guardrails that can be set by the cluster admin to limit the subquery results: + +1. `druid.server.http.maxSubqueryRows` in broker's config to set a default for the entire cluster or `maxSubqueryRows` in the query context to set an upper limit on the number of rows a subquery can generate +2. `druid.server.http.maxSubqueryBytes` in broker's config to set a default for the entire cluster or `maxSubqueryBytes` in the query context to set an upper limit on the number of bytes a subquery can generate + +Note that limiting the subquery by bytes is a newer feature therefore it is experimental as it materializes the results differently. + +If you choose to modify or set any of the above limits, you must also think about the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results. +There is no formula to calculate the correct value. Trial and error is the best approach. ###### 'Manual' laning strategy This laning strategy is best suited for cases where one or more external applications which query Druid are capable of manually deciding what lane a given query should belong to. Configured with a map of lane names to percent or exact max capacities, queries with a matching `lane` parameter in the [query context](../querying/query-context.md) will be subjected to those limits. @@ -1862,6 +1875,7 @@ Druid uses Jetty to serve HTTP requests. Each query being processed consumes a s |`druid.server.http.defaultQueryTimeout`|Query timeout in millis, beyond which unfinished queries will be cancelled|300000| |`druid.server.http.maxScatterGatherBytes`|Maximum number of bytes gathered from data processes such as Historicals and realtime processes to execute a query. Queries that exceed this limit will fail. This is an advance configuration that allows to protect in case Broker is under heavy load and not utilizing the data gathered in memory fast enough and leading to OOMs. This limit can be further reduced at query time using `maxScatterGatherBytes` in the context. Note that having large limit is not necessarily bad if broker is never under heavy concurrent load in which case data gathered is processed quickly and freeing up the memory used. Human-readable format is supported, see [here](human-readable-byte.md). |Long.MAX_VALUE| |`druid.server.http.maxSubqueryRows`|Maximum number of rows from all subqueries per query. Druid stores the subquery rows in temporary tables that live in the Java heap. `druid.server.http.maxSubqueryRows` is a guardrail to prevent the system from exhausting available heap. When a subquery exceeds the row limit, Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum."

It is a good practice to avoid large subqueries in Druid. However, if you choose to raise the subquery row limit, you must also increase the heap size of all Brokers, Historicals, and task Peons that process data for the subqueries to accommodate the subquery results.

There is no formula to calculate the correct value. Trial and error is the best approach.|100000| +|`druid.server.http.maxSubqueryBytes`|Maximum number of bytes from all subqueries per query. Since the results are stored on the Java heap, `druid.server.http.maxSubqueryBytes` is a guardrail like `druid.server.http.maxSubqueryRows` to prevent the heap space from exhausting. When a subquery exceeds the byte limit, Druid throws a resource limit exceeded exception. A negative value for the guardrail indicates that Druid won't guardrail by memory. Check the docs for `druid.server.http.maxSubqueryRows` to see how to set the optimal value for a cluster. This is an experimental feature for now as this materializes the results in a different format.|-1| |`druid.server.http.gracefulShutdownTimeout`|The maximum amount of time Jetty waits after receiving shutdown signal. After this timeout the threads will be forcefully shutdown. This allows any queries that are executing to complete(Only values greater than zero are valid).|`PT30S`| |`druid.server.http.unannouncePropagationDelay`|How long to wait for ZooKeeper unannouncements to propagate before shutting down Jetty. This is a minimum and `druid.server.http.gracefulShutdownTimeout` does not start counting down until after this period elapses.|`PT0S` (do not wait)| |`druid.server.http.maxQueryTimeout`|Maximum allowed value (in milliseconds) for `timeout` parameter. See [query-context](../querying/query-context.md) to know more about `timeout`. Query is rejected if the query context `timeout` is greater than this value. |Long.MAX_VALUE| diff --git a/docs/querying/query-execution.md b/docs/querying/query-execution.md index 565f977e802a..b51ee96bb0d4 100644 --- a/docs/querying/query-execution.md +++ b/docs/querying/query-execution.md @@ -85,10 +85,15 @@ their "base" (bottom-leftmost) datasource, as described in the [join](#join) sec the results are brought back to the Broker. Then, the Broker continues on with the rest of the query as if the subquery was replaced with an inline datasource. -In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds. Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows. Otherwise Druid throws a resource limit exceeded exception: "Subquery generated results beyond maximum." +In most cases, Druid buffers subquery results in memory on the Broker before the rest of the query proceeds. +Therefore, subqueries execute sequentially. The total number of rows buffered across all subqueries of a given query +cannot exceed the [`druid.server.http.maxSubqueryRows`](../configuration/index.md) which defaults to 100000 rows, or the +[`druid.server.http.maxSubqueryBytes`](../configuration/index.md) if set. Otherwise, Druid throws a resource limit exceeded +exception. There is one exception: if the outer query and all subqueries are the [groupBy](groupbyquery.md) type, then subquery -results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` limit does not apply. +results can be processed in a streaming fashion and the `druid.server.http.maxSubqueryRows` and `druid.server.http.maxSubqueryBytes` +limits do not apply. ### `join` diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java index 4c3e8be9c58c..362ba5580afb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/input/inline/InlineInputSliceReader.java @@ -30,6 +30,7 @@ import org.apache.druid.msq.input.table.SegmentWithDescriptor; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.timeline.SegmentId; @@ -37,7 +38,7 @@ /** * Reads {@link InlineInputSlice} using {@link SegmentWrangler} (which is expected to contain an - * {@link org.apache.druid.segment.InlineSegmentWrangler}). + * {@link InlineSegmentWrangler}). */ public class InlineInputSliceReader implements InputSliceReader { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java index 904c84063c90..5ee3ba875388 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteSelectQueryMSQTest.java @@ -149,13 +149,6 @@ public void testUnplannableQueries() } - @Ignore - @Override - public void testMaxSubqueryRows() - { - - } - @Ignore @Override public void testQueryWithMoreThanMaxNumericInFilter() diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index af11a58a35a0..e5cc20746b49 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -71,6 +71,10 @@ private FrameReader( /** * Create a reader for frames with a given {@link RowSignature}. The signature must exactly match the frames to be * read, or else behavior is undefined. + * If the columnType is null, we store the data as {@link ColumnType#NESTED_DATA}. This can be done if we know that + * the data that we receive can be serded generically using the nested data. It is currently used in the brokers to + * store the data with unknown types into frames. + * @param signature signature used to generate the reader */ public static FrameReader create(final RowSignature signature) { diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java index 0073f1b40205..27da7cd51cec 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameCursorUtils.java @@ -19,10 +19,17 @@ package org.apache.druid.frame.segment; +import org.apache.druid.error.DruidException; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.filter.BoundFilter; import org.apache.druid.segment.filter.Filters; @@ -30,6 +37,8 @@ import javax.annotation.Nullable; import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; public class FrameCursorUtils { @@ -67,4 +76,63 @@ public static Filter buildFilter(@Nullable Filter filter, Interval interval) ); } } + + /** + * Writes a {@link Cursor} to a sequence of {@link Frame}. This method iterates over the rows of the cursor, + * and writes the columns to the frames + * + * @param cursor Cursor to write to the frame + * @param frameWriterFactory Frame writer factory to write to the frame. + * Determines the signature of the rows that are written to the frames + */ + public static Sequence cursorToFrames( + Cursor cursor, + FrameWriterFactory frameWriterFactory + ) + { + + return Sequences.simple( + () -> new Iterator() + { + @Override + public boolean hasNext() + { + return !cursor.isDone(); + } + + @Override + public Frame next() + { + // Makes sure that cursor contains some elements prior. This ensures if no row is written, then the row size + // is larger than the MemoryAllocators returned by the provided factory + if (!hasNext()) { + throw new NoSuchElementException(); + } + boolean firstRowWritten = false; + Frame frame; + try (final FrameWriter frameWriter = frameWriterFactory.newFrameWriter(cursor.getColumnSelectorFactory())) { + while (!cursor.isDone()) { + if (!frameWriter.addSelection()) { + break; + } + firstRowWritten = true; + cursor.advance(); + } + + if (!firstRowWritten) { + throw DruidException + .forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) + .build("Subquery's row size exceeds the frame size and therefore cannot write the subquery's " + + "row to the frame. This is a non-configurable static limit that can only be modified by the " + + "developer."); + } + + frame = Frame.wrap(frameWriter.toByteArray()); + } + return frame; + } + } + ); + } } diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java index c071b86b4365..f42a33ce6f78 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameQueryableIndex.java @@ -52,7 +52,7 @@ public class FrameQueryableIndex implements QueryableIndex private final RowSignature signature; private final List columnReaders; - FrameQueryableIndex( + public FrameQueryableIndex( final Frame frame, final RowSignature signature, final List columnReaders diff --git a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java index 955b9e845d58..1df2702d97a0 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java @@ -291,4 +291,15 @@ private static boolean areKeyColumnsPrefixOfSignature( return true; } + + public static RowSignature replaceUnknownTypesWithNestedColumns(final RowSignature rowSignature) + { + RowSignature.Builder retBuilder = RowSignature.builder(); + for (int i = 0; i < rowSignature.size(); ++i) { + String columnName = rowSignature.getColumnName(i); + ColumnType columnType = rowSignature.getColumnType(i).orElse(ColumnType.NESTED_DATA); + retBuilder.add(columnName, columnType); + } + return retBuilder.build(); + } } diff --git a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java index b5bc6f5fe529..ddbe253de016 100644 --- a/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java +++ b/processing/src/main/java/org/apache/druid/jackson/DruidDefaultSerializersModule.java @@ -32,6 +32,8 @@ 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.jackson.JacksonUtils; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.query.FrameBasedInlineDataSourceSerializer; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.context.ResponseContextDeserializer; import org.joda.time.DateTimeZone; @@ -52,6 +54,8 @@ public DruidDefaultSerializersModule() JodaStuff.register(this); + addSerializer(FrameBasedInlineDataSource.class, new FrameBasedInlineDataSourceSerializer()); + addDeserializer( DateTimeZone.class, new JsonDeserializer() diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java new file mode 100644 index 000000000000..a0b7bda150d0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSource.java @@ -0,0 +1,197 @@ +/* + * 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; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +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.guava.Sequences; +import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Represents an inline datasource where the rows are embedded within the DataSource object itself. + *

+ * The rows are backed by a sequence of {@link FrameSignaturePair}, which contain the Frame representation of the rows + * represented by the datasource. + *

+ * Note that the signature of the datasource can be different from the signatures of the constituent frames that it + * consists of. While fetching the iterables, it is the job of this class to make sure that the rows correspond to the + * {@link #rowSignature}. For frames that donot contain the columns present in the {@link #rowSignature}, they are + * populated with {@code null}. + */ +public class FrameBasedInlineDataSource implements DataSource +{ + + final List frames; + final RowSignature rowSignature; + + public FrameBasedInlineDataSource( + List frames, + RowSignature rowSignature + ) + { + this.frames = frames; + this.rowSignature = rowSignature; + } + + public List getFrames() + { + return frames; + } + + public RowSignature getRowSignature() + { + return rowSignature; + } + + public Sequence getRowsAsSequence() + { + + final Sequence cursorSequence = + Sequences.simple(frames) + .flatMap( + frameSignaturePair -> { + Frame frame = frameSignaturePair.getFrame(); + RowSignature frameSignature = frameSignaturePair.getRowSignature(); + FrameReader frameReader = FrameReader.create(frameSignature); + return new FrameStorageAdapter(frame, frameReader, Intervals.ETERNITY) + .makeCursors(null, Intervals.ETERNITY, VirtualColumns.EMPTY, Granularities.ALL, false, null); + } + ); + + return cursorSequence.flatMap( + (cursor) -> { + final ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + final List selectors = rowSignature + .getColumnNames() + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + return Sequences.simple( + () -> new Iterator() + { + @Override + public boolean hasNext() + { + return !cursor.isDone(); + } + + @Override + public Object[] next() + { + + Object[] row = new Object[rowSignature.size()]; + for (int i = 0; i < rowSignature.size(); ++i) { + row[i] = selectors.get(i).getObject(); + } + + cursor.advance(); + + return row; + } + } + ); + } + ); + } + + @Override + public Set getTableNames() + { + return Collections.emptySet(); + } + + @Override + public List getChildren() + { + return Collections.emptyList(); + } + + @Override + public DataSource withChildren(List children) + { + if (!children.isEmpty()) { + throw new IAE("Cannot accept children"); + } + + return this; + } + + @Override + public boolean isCacheable(boolean isBroker) + { + return false; + } + + @Override + public boolean isGlobal() + { + return true; + } + + @Override + public boolean isConcrete() + { + return true; + } + + @Override + public Function createSegmentMapFunction(Query query, AtomicLong cpuTimeAcc) + { + return Function.identity(); + } + + @Override + public DataSource withUpdatedDataSource(DataSource newSource) + { + return newSource; + } + + @Override + public byte[] getCacheKey() + { + return null; + } + + @Override + public DataSourceAnalysis getAnalysis() + { + return new DataSourceAnalysis(this, null, null, Collections.emptyList()); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java new file mode 100644 index 000000000000..0a4feaca9bdb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializer.java @@ -0,0 +1,103 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.jsontype.TypeSerializer; +import com.fasterxml.jackson.databind.ser.std.StdSerializer; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +/** + * Serializes {@link FrameBasedInlineDataSource} to the representation of {@link InlineDataSource} + * so that the servers' on wire transfer data doesn't change. {@link FrameBasedInlineDataSource} is currently limited + * to the brokers only and therefore this aids in conversion of the object to a representation that the data servers + * can recognize + */ +public class FrameBasedInlineDataSourceSerializer extends StdSerializer +{ + public FrameBasedInlineDataSourceSerializer() + { + super(FrameBasedInlineDataSource.class); + } + + @Override + public void serialize(FrameBasedInlineDataSource value, JsonGenerator jg, SerializerProvider serializers) + throws IOException + { + jg.writeStartObject(); + jg.writeStringField("type", "inline"); + + RowSignature rowSignature = value.getRowSignature(); + jg.writeObjectField("columnNames", rowSignature.getColumnNames()); + List columnTypes = IntStream.range(0, rowSignature.size()) + .mapToObj(i -> rowSignature.getColumnType(i).orElse(null)) + .collect(Collectors.toList()); + jg.writeObjectField("columnTypes", columnTypes); + + jg.writeArrayFieldStart("rows"); + + value.getRowsAsSequence().forEach(row -> { + try { + JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, row); + } + catch (IOException e) { + // Ideally, this shouldn't be reachable. + // Wrap the IO exception in the runtime exception and propogate it forward + List elements = new ArrayList<>(); + for (Object o : row) { + elements.add(o.toString()); + } + throw new RE( + e, + "Exception encountered while serializing [%s] in [%s]", + String.join(", ", elements), + FrameBasedInlineDataSource.class + ); + } + }); + + jg.writeEndArray(); + jg.writeEndObject(); + } + + /** + * Required because {@link DataSource} is polymorphic + */ + @Override + public void serializeWithType( + FrameBasedInlineDataSource value, + JsonGenerator jg, + SerializerProvider serializers, + TypeSerializer typeSer + ) throws IOException + { + serialize(value, jg, serializers); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java b/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java new file mode 100644 index 000000000000..ef279ea73b82 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/FrameSignaturePair.java @@ -0,0 +1,52 @@ +/* + * 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; + +import com.google.common.base.Preconditions; +import org.apache.druid.frame.Frame; +import org.apache.druid.segment.column.RowSignature; + +/** + * Encapsulates a {@link Frame} and the {@link RowSignature} of the rows that are encapsulated in it + */ +public class FrameSignaturePair +{ + final Frame frame; + final RowSignature rowSignature; + + public FrameSignaturePair( + Frame frame, + RowSignature rowSignature + ) + { + this.frame = Preconditions.checkNotNull(frame, "'frame' must be non null"); + this.rowSignature = Preconditions.checkNotNull(rowSignature, "'rowSignature' must be non null"); + } + + public Frame getFrame() + { + return frame; + } + + public RowSignature getRowSignature() + { + return rowSignature; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java index 3fb015d369ec..a14eb63fe6f5 100644 --- a/processing/src/main/java/org/apache/druid/query/InlineDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/InlineDataSource.java @@ -190,6 +190,7 @@ public List getColumnTypes() @JsonProperty("rows") public List getRowsAsList() { + Iterable rows = getRows(); return rows instanceof List ? ((List) rows) : Lists.newArrayList(rows); } diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java new file mode 100644 index 000000000000..c5bb271213e8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java @@ -0,0 +1,106 @@ +/* + * 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; + +import org.apache.druid.java.util.common.Intervals; +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.guava.Sequences; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.segment.RowAdapter; +import org.apache.druid.segment.RowBasedCursor; +import org.apache.druid.segment.RowWalker; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Iterator; + +/** + * Helper methods to create cursor from iterable of rows + */ +public class IterableRowsCursorHelper +{ + + /** + * Creates a cursor that iterates over all the rows generated by the iterable. Presence of __time column is not a + * necessity + */ + public static RowBasedCursor getCursorFromIterable(Iterable rows, RowSignature rowSignature) + { + RowAdapter rowAdapter = columnName -> { + if (rowSignature == null) { + return row -> null; + } + final int columnIndex = rowSignature.indexOf(columnName); + if (columnIndex < 0) { + return row -> null; + } + return row -> row[columnIndex]; + }; + RowWalker rowWalker = new RowWalker<>(Sequences.simple(rows), rowAdapter); + return new RowBasedCursor<>( + rowWalker, + rowAdapter, + null, + Intervals.ETERNITY, // Setting the interval to eternity ensures that we are iterating over all of the rows + VirtualColumns.EMPTY, + Granularities.ALL, + false, + rowSignature != null ? rowSignature : RowSignature.empty() + ); + } + + /** + * Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a + * necessity + */ + public static RowBasedCursor getCursorFromSequence(Sequence rows, RowSignature rowSignature) + { + return getCursorFromIterable( + new Iterable() + { + Yielder yielder = Yielders.each(rows); + + @Override + public Iterator iterator() + { + return new Iterator() + { + @Override + public boolean hasNext() + { + return !yielder.isDone(); + } + + @Override + public Object[] next() + { + Object[] retVal = yielder.get(); + yielder = yielder.next(null); + return retVal; + } + }; + } + }, + rowSignature + ); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryContext.java b/processing/src/main/java/org/apache/druid/query/QueryContext.java index 624bc1cb3d15..147ebbec18eb 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContext.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContext.java @@ -357,6 +357,16 @@ public int getMaxSubqueryRows(int defaultSize) return getInt(QueryContexts.MAX_SUBQUERY_ROWS_KEY, defaultSize); } + public long getMaxSubqueryMemoryBytes(long defaultMemoryBytes) + { + return getLong(QueryContexts.MAX_SUBQUERY_BYTES_KEY, defaultMemoryBytes); + } + + public boolean isUseNestedForUnknownTypeInSubquery(boolean defaultUseNestedForUnkownTypeInSubquery) + { + return getBoolean(QueryContexts.USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY, defaultUseNestedForUnkownTypeInSubquery); + } + public int getUncoveredIntervalsLimit() { return getUncoveredIntervalsLimit(QueryContexts.DEFAULT_UNCOVERED_INTERVALS_LIMIT); diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index f3106d9a6019..ea21987bd18f 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -54,6 +54,8 @@ public class QueryContexts public static final String VECTORIZE_VIRTUAL_COLUMNS_KEY = "vectorizeVirtualColumns"; public static final String VECTOR_SIZE_KEY = "vectorSize"; public static final String MAX_SUBQUERY_ROWS_KEY = "maxSubqueryRows"; + public static final String MAX_SUBQUERY_BYTES_KEY = "maxSubqueryBytes"; + public static final String USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = "useNestedForUnknownTypeInSubquery"; public static final String JOIN_FILTER_PUSH_DOWN_KEY = "enableJoinFilterPushDown"; public static final String JOIN_FILTER_REWRITE_ENABLE_KEY = "enableJoinFilterRewrite"; public static final String JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY = "enableJoinFilterRewriteValueColumnFilters"; diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index ee400f814fb8..2657f4f49f7c 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.type.TypeFactory; import com.google.common.base.Function; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Sequence; @@ -34,6 +35,7 @@ import javax.annotation.Nullable; import java.util.Comparator; import java.util.List; +import java.util.Optional; import java.util.function.BinaryOperator; /** @@ -318,7 +320,7 @@ public RowSignature resultArraySignature(QueryType query) * query, each {@link org.apache.druid.query.topn.TopNResultValue} will generate a separate array for each of its * {@code values}. * - * By convention, the array form should include the __time column, if present, as a long (milliseconds since epoch). + * By convention, the array form should include the __time column, if present, as a long (milliseconds since epoch). * * @param resultSequence results of the form returned by {@link #mergeResults} * @@ -330,4 +332,32 @@ public Sequence resultsAsArrays(QueryType query, Sequence { throw new UOE("Query type '%s' does not support returning results as arrays", query.getType()); } + + /** + * Converts a sequence of this query's ResultType into a sequence of {@link FrameSignaturePair}. The array signature + * is the one give by {@link #resultArraySignature(Query)}. If the toolchest doesn't support this method, then it can + * return an empty optional. It is the duty of the callees to throw an appropriate exception in that case or use an + * alternative fallback approach + * + * Check documentation of {@link #resultsAsArrays(Query, Sequence)} as the behaviour of the rows represented by the + * frame sequence is identical. + * + * Each Frame has a separate {@link RowSignature} because for some query types like the Scan query, every + * column in the final result might not be present in the individual ResultType (and subsequently Frame). Therefore, + * this is done to preserve the space by not populating the column in that particular Frame and omitting it from its + * signature + * @param query Query being executed by the toolchest. Used to determine the rowSignature of the Frames + * @param resultSequence results of the form returned by {@link #mergeResults(QueryRunner)} + * @param memoryAllocatorFactory + * @param useNestedForUnknownTypes true if the unknown types in the results can be serded using complex types + */ + public Optional> resultsAsFrames( + QueryType query, + Sequence resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + return Optional.empty(); + } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index 173e6babd0d4..d13998c8340f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -36,6 +36,13 @@ import com.google.common.collect.Lists; import com.google.inject.Inject; import org.apache.druid.data.input.Row; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.MappedSequence; @@ -44,6 +51,8 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.CacheStrategy; import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryPlus; @@ -62,6 +71,7 @@ import org.apache.druid.query.groupby.resource.GroupByQueryResource; import org.apache.druid.query.groupby.strategy.GroupByStrategy; import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; @@ -73,6 +83,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.TreeMap; import java.util.function.BinaryOperator; @@ -705,6 +716,40 @@ public Sequence resultsAsArrays(final GroupByQuery query, final Sequen return resultSequence.map(ResultRow::getArray); } + /** + * This returns a single frame containing the results of the group by query. + */ + @Override + public Optional> resultsAsFrames( + GroupByQuery query, + Sequence resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + RowSignature rowSignature = resultArraySignature(query); + RowSignature modifiedRowSignature = useNestedForUnknownTypes + ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) + : rowSignature; + + FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + memoryAllocatorFactory, + modifiedRowSignature, + new ArrayList<>() + ); + + + Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + resultsAsArrays(query, resultSequence), + rowSignature + ); + + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + + return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); + } + /** * This function checks the query for dimensions which can be optimized by applying the dimension extraction * as the final step of the query instead of on every event. diff --git a/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java new file mode 100644 index 000000000000..71ba425d4578 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java @@ -0,0 +1,328 @@ +/* + * 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.base.Predicate; +import com.google.common.math.IntMath; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.RowIdSupplier; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.data.IndexedInts; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import java.util.List; + +/** + * Combines multiple cursors and iterates over them. It skips over the empty cursors + * The {@link DimensionSelector} and {@link ColumnValueSelector} it generates hold the reference to the original object + * because the cursor might be advanced independently after extracting out the {@link ColumnSelectorFactory} like in + * {@link org.apache.druid.frame.segment.FrameCursorUtils#cursorToFrames}. This ensures that the selectors always return + * the value pointed by the {@link #currentCursor}. + */ +public class ConcatCursor implements Cursor +{ + + private final List cursors; + private int currentCursor; + + public ConcatCursor( + List cursors + ) + { + this.cursors = cursors; + currentCursor = 0; + skipEmptyCursors(); + } + + @Override + public ColumnSelectorFactory getColumnSelectorFactory() + { + return new ColumnSelectorFactory() + { + @Override + public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec) + { + return new DimensionSelector() + { + @Override + public IndexedInts getRow() + { + return cursors.get(currentCursor).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec).getRow(); + } + + @Override + public ValueMatcher makeValueMatcher(@Nullable String value) + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .makeValueMatcher(value); + } + + @Override + public ValueMatcher makeValueMatcher(Predicate predicate) + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .makeValueMatcher(predicate); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .inspectRuntimeShape(inspector); + } + + @Nullable + @Override + public Object getObject() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .getObject(); + } + + @Override + public Class classOfObject() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .classOfObject(); + } + + @Override + public int getValueCardinality() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .getValueCardinality(); + } + + @Nullable + @Override + public String lookupName(int id) + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .lookupName(id); + } + + @Override + public boolean nameLookupPossibleInAdvance() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .nameLookupPossibleInAdvance(); + } + + @Nullable + @Override + public IdLookup idLookup() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeDimensionSelector(dimensionSpec) + .idLookup(); + } + }; + } + + @Override + public ColumnValueSelector makeColumnValueSelector(String columnName) + { + return new ColumnValueSelector() + { + @Override + public double getDouble() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeColumnValueSelector(columnName) + .getDouble(); + } + + @Override + public float getFloat() + { + return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).getFloat(); + } + + @Override + public long getLong() + { + return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).getLong(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeColumnValueSelector(columnName) + .inspectRuntimeShape(inspector); + } + + @Override + public boolean isNull() + { + return cursors.get(currentCursor).getColumnSelectorFactory().makeColumnValueSelector(columnName).isNull(); + } + + @Nullable + @Override + public Object getObject() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeColumnValueSelector(columnName) + .getObject(); + } + + @Override + public Class classOfObject() + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .makeColumnValueSelector(columnName) + .classOfObject(); + } + }; + } + + @Override + public ColumnCapabilities getColumnCapabilitiesWithDefault(String column, ColumnCapabilities defaultCapabilites) + { + return cursors.get(currentCursor) + .getColumnSelectorFactory() + .getColumnCapabilitiesWithDefault(column, defaultCapabilites); + } + + @Nullable + @Override + public ExpressionType getType(String name) + { + return cursors.get(currentCursor).getColumnSelectorFactory().getType(name); + } + + @Nullable + @Override + public ColumnCapabilities getColumnCapabilities(String column) + { + return cursors.get(currentCursor).getColumnSelectorFactory().getColumnCapabilities(column); + } + + @Nullable + @Override + public RowIdSupplier getRowIdSupplier() + { + return cursors.get(currentCursor).getColumnSelectorFactory().getRowIdSupplier(); + } + }; + } + + @Override + public DateTime getTime() + { + return cursors.get(currentCursor).getTime(); + } + + @Override + public void advance() + { + if (currentCursor < cursors.size()) { + cursors.get(currentCursor).advance(); + advanceCursor(); + } + } + + @Override + public void advanceUninterruptibly() + { + if (currentCursor < cursors.size()) { + cursors.get(currentCursor).advanceUninterruptibly(); + advanceCursor(); + } + } + + @Override + public boolean isDone() + { + return currentCursor == cursors.size(); + } + + @Override + public boolean isDoneOrInterrupted() + { + return isDone() || Thread.currentThread().isInterrupted(); + } + + @Override + public void reset() + { + while (currentCursor >= 0) { + if (currentCursor < cursors.size()) { + cursors.get(currentCursor).reset(); + } + currentCursor = IntMath.checkedSubtract(currentCursor, 1); + } + currentCursor = 0; + skipEmptyCursors(); + } + + /** + * This method should be called whenever the currentCursor gets updated. It skips over the empty cursors so that the + * current pointer is pointing to a valid cursor + */ + private void skipEmptyCursors() + { + while (currentCursor < cursors.size() && cursors.get(currentCursor).isDone()) { + currentCursor = IntMath.checkedAdd(currentCursor, 1); + } + } + + /** + * This method updates the current cursor. This is used to update the current cursor under question. + */ + private void advanceCursor() + { + if (cursors.get(currentCursor).isDone()) { + currentCursor = IntMath.checkedAdd(currentCursor, 1); + skipEmptyCursors(); + } + } +} 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 efcbe51c0c4f..2246e349cbb4 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 @@ -39,7 +39,9 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.Filters; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -146,6 +148,7 @@ public Sequence process( public Iterator make() { final List columnSelectors = new ArrayList<>(allColumns.size()); + final RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); for (String column : allColumns) { final BaseObjectColumnValueSelector selector; @@ -153,8 +156,20 @@ public Iterator make() if (legacy && LEGACY_TIMESTAMP_KEY.equals(column)) { selector = cursor.getColumnSelectorFactory() .makeColumnValueSelector(ColumnHolder.TIME_COLUMN_NAME); + ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory() + .getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME); + rowSignatureBuilder.add( + column, + columnCapabilities == null ? null : columnCapabilities.toColumnType() + ); } else { selector = cursor.getColumnSelectorFactory().makeColumnValueSelector(column); + ColumnCapabilities columnCapabilities = cursor.getColumnSelectorFactory() + .getColumnCapabilities(column); + rowSignatureBuilder.add( + column, + columnCapabilities == null ? null : columnCapabilities.toColumnType() + ); } columnSelectors.add(selector); @@ -191,7 +206,7 @@ public ScanResultValue next() throw new UOE("resultFormat[%s] is not supported", resultFormat.toString()); } responseContext.addRowScanCount(offset - lastOffset); - return new ScanResultValue(segmentId.toString(), allColumns, events); + return new ScanResultValue(segmentId.toString(), allColumns, events, rowSignatureBuilder.build()); } @Override 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 68b311e83a30..785b71ed42d1 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 @@ -30,6 +30,7 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.column.RowSignature; import java.io.IOException; import java.util.ArrayList; @@ -111,7 +112,12 @@ public ScanResultValue next() // single batch length is <= Integer.MAX_VALUE, so this should not overflow int numLeft = (int) (limit - count); count = limit; - return new ScanResultValue(batch.getSegmentId(), batch.getColumns(), events.subList(0, numLeft)); + return new ScanResultValue( + batch.getSegmentId(), + batch.getColumns(), + events.subList(0, numLeft), + batch.getRowSignature() + ); } } else { // Perform single-event ScanResultValue batching at the outer level. Each scan result value from the yielder @@ -119,15 +125,17 @@ public ScanResultValue next() int batchSize = query.getBatchSize(); List eventsToAdd = new ArrayList<>(batchSize); List columns = new ArrayList<>(); + RowSignature rowSignature = null; 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; + rowSignature = rowSignature == null ? srv.getRowSignature() : rowSignature; eventsToAdd.add(Iterables.getOnlyElement((List) srv.getEvents())); yielder = yielder.next(null); count++; } - return new ScanResultValue(null, columns, eventsToAdd); + return new ScanResultValue(null, columns, eventsToAdd, rowSignature); } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java index 6685a40d9735..57bf6672d702 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryOffsetSequence.java @@ -117,7 +117,7 @@ public OutType accumulate(OutType accumulated, ScanResultValue result) skipped += toSkip; return super.accumulate( accumulated, - new ScanResultValue(result.getSegmentId(), result.getColumns(), newEvents) + new ScanResultValue(result.getSegmentId(), result.getColumns(), newEvents, result.getRowSignature()) ); } } else { 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 4a48563689a7..b7253d70fe93 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,27 +22,46 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.collect.PeekingIterator; import com.google.inject.Inject; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; 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.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.query.FrameSignaturePair; import org.apache.druid.query.GenericQueryMetricsFactory; +import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.aggregation.MetricManipulationFn; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.utils.CloseableUtils; +import java.util.ArrayList; +import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; public class ScanQueryQueryToolChest extends QueryToolChest { @@ -187,13 +206,146 @@ public RowSignature resultArraySignature(final ScanQuery query) } } + /** + * This batches the fetched {@link ScanResultValue}s which have similar signatures and are consecutives. In best case + * it would return a single frame, and in the worst case, it would return as many frames as the number of {@link ScanResultValue} + * passed. + */ + @Override + public Optional> resultsAsFrames( + final ScanQuery query, + final Sequence resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + final RowSignature defaultRowSignature = resultArraySignature(query); + Iterator resultSequenceIterator = new Iterator() + { + Yielder yielder = Yielders.each(resultSequence); + + @Override + public boolean hasNext() + { + return !yielder.isDone(); + } + + @Override + public ScanResultValue next() + { + ScanResultValue scanResultValue = yielder.get(); + yielder = yielder.next(null); + return scanResultValue; + } + }; + + Iterable> retVal = () -> new Iterator>() + { + PeekingIterator scanResultValuePeekingIterator = Iterators.peekingIterator(resultSequenceIterator); + + @Override + public boolean hasNext() + { + return scanResultValuePeekingIterator.hasNext(); + } + + @Override + public Sequence next() + { + final List batch = new ArrayList<>(); + final ScanResultValue scanResultValue = scanResultValuePeekingIterator.next(); + batch.add(scanResultValue); + // If the rowSignature is not provided, assume that the scanResultValue can contain any number of the columns + // that appear in the original scan query + final RowSignature rowSignature = scanResultValue.getRowSignature() != null + ? scanResultValue.getRowSignature() + : defaultRowSignature; + while (scanResultValuePeekingIterator.hasNext()) { + RowSignature nextRowSignature = scanResultValuePeekingIterator.peek().getRowSignature(); + if (nextRowSignature == null) { + nextRowSignature = defaultRowSignature; + } + if (nextRowSignature != null && nextRowSignature.equals(rowSignature)) { + batch.add(scanResultValuePeekingIterator.next()); + } else { + break; + } + } + return convertScanResultValuesToFrame( + batch, + rowSignature, + query, + memoryAllocatorFactory, + useNestedForUnknownTypes + ); + } + }; + return Optional.of(Sequences.concat(retVal)); + } + + private Sequence convertScanResultValuesToFrame( + List batch, + RowSignature rowSignature, + ScanQuery query, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + Preconditions.checkNotNull(rowSignature, "'rowSignature' must be provided"); + + List cursors = new ArrayList<>(); + + for (ScanResultValue scanResultValue : batch) { + final List rows = (List) scanResultValue.getEvents(); + final Function mapper = getResultFormatMapper(query.getResultFormat(), rowSignature.getColumnNames()); + final Iterable formattedRows = Lists.newArrayList(Iterables.transform(rows, (Function) mapper)); + + cursors.add(IterableRowsCursorHelper.getCursorFromIterable( + formattedRows, + rowSignature + )); + } + + RowSignature modifiedRowSignature = useNestedForUnknownTypes + ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) + : rowSignature; + FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + memoryAllocatorFactory, + modifiedRowSignature, + new ArrayList<>() + ); + + + Cursor concatCursor = new ConcatCursor(cursors); + Sequence frames = FrameCursorUtils.cursorToFrames( + concatCursor, + frameWriterFactory + ); + + return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)); + } + @Override public Sequence resultsAsArrays(final ScanQuery query, final Sequence resultSequence) { - final List fields = resultArraySignature(query).getColumnNames(); - final Function mapper; + final Function mapper = getResultFormatMapper(query.getResultFormat(), resultArraySignature(query).getColumnNames()); + + return resultSequence.flatMap( + result -> { + // Generics? Where we're going, we don't need generics. + final List rows = (List) result.getEvents(); + final Iterable arrays = Iterables.transform(rows, (Function) mapper); + return Sequences.simple(arrays); + } + ); + } + + private Function getResultFormatMapper(ScanQuery.ResultFormat resultFormat, List fields) + { + Function mapper; - switch (query.getResultFormat()) { + switch (resultFormat) { case RESULT_FORMAT_LIST: mapper = (Map row) -> { final Object[] rowArray = new Object[fields.size()]; @@ -220,16 +372,8 @@ public Sequence resultsAsArrays(final ScanQuery query, final Sequence< }; break; default: - throw new UOE("Unsupported resultFormat for array-based results: %s", query.getResultFormat()); + throw new UOE("Unsupported resultFormat for array-based results: %s", resultFormat); } - - return resultSequence.flatMap( - result -> { - // Generics? Where we're going, we don't need generics. - final List rows = (List) result.getEvents(); - final Iterable arrays = Iterables.transform(rows, (Function) mapper); - return Sequences.simple(arrays); - } - ); + return mapper; } } 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 aa368102dd4f..ab65d3fc6c37 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,12 +25,14 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; public class ScanResultValue implements Comparable { @@ -45,17 +47,25 @@ public class ScanResultValue implements Comparable private final String segmentId; private final List columns; private final Object events; + private final RowSignature rowSignature; @JsonCreator public ScanResultValue( @JsonProperty("segmentId") @Nullable String segmentId, @JsonProperty("columns") List columns, - @JsonProperty("events") Object events + @JsonProperty("events") Object events, + @Nullable @JsonProperty("rowSignature") RowSignature rowSignature ) { this.segmentId = segmentId; this.columns = columns; this.events = events; + this.rowSignature = rowSignature; + } + + public ScanResultValue(String segmentId, List columns, Object events) + { + this(segmentId, columns, events, null); } @Nullable @@ -77,6 +87,14 @@ public Object getEvents() return events; } + @Nullable + @JsonProperty + public RowSignature getRowSignature() + { + return rowSignature; + } + + public long getFirstEventTimestamp(ScanQuery.ResultFormat resultFormat) { if (resultFormat.equals(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) { @@ -101,7 +119,7 @@ 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))); + singleEventScanResultValues.add(new ScanResultValue(segmentId, columns, Collections.singletonList(event), rowSignature)); } return singleEventScanResultValues; } @@ -118,13 +136,16 @@ public boolean equals(Object o) ScanResultValue that = (ScanResultValue) o; - if (segmentId != null ? !segmentId.equals(that.segmentId) : that.segmentId != null) { + if (!Objects.equals(segmentId, that.segmentId)) { + return false; + } + if (!Objects.equals(columns, that.columns)) { return false; } - if (columns != null ? !columns.equals(that.columns) : that.columns != null) { + if (!Objects.equals(rowSignature, that.rowSignature)) { return false; } - return events != null ? events.equals(that.events) : that.events == null; + return Objects.equals(events, that.events); } @Override @@ -133,6 +154,7 @@ public int hashCode() int result = segmentId != null ? segmentId.hashCode() : 0; result = 31 * result + (columns != null ? columns.hashCode() : 0); result = 31 * result + (events != null ? events.hashCode() : 0); + result = 31 * result + (rowSignature != null ? rowSignature.hashCode() : 0); return result; } @@ -143,6 +165,7 @@ public String toString() "segmentId='" + segmentId + '\'' + ", columns=" + columns + ", events=" + events + + ", rowSignature=" + rowSignature + '}'; } diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index 5a4417aa719f..54a17d267d18 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -30,12 +30,21 @@ import com.google.inject.Inject; import org.apache.commons.lang.StringUtils; import org.apache.druid.data.input.MapBasedRow; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.CacheStrategy; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; @@ -49,18 +58,21 @@ import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.function.BinaryOperator; /** @@ -431,7 +443,6 @@ public Sequence resultsAsArrays( ) { final List fields = resultArraySignature(query).getColumnNames(); - return Sequences.map( resultSequence, result -> { @@ -451,6 +462,39 @@ public Sequence resultsAsArrays( ); } + /** + * This returns a single frame containing the results of the timeseries query + */ + @Override + public Optional> resultsAsFrames( + TimeseriesQuery query, + Sequence> resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + final RowSignature rowSignature = resultArraySignature(query); + final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + resultsAsArrays(query, resultSequence), + rowSignature + ); + + RowSignature modifiedRowSignature = useNestedForUnknownTypes + ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) + : rowSignature; + FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + memoryAllocatorFactory, + modifiedRowSignature, + new ArrayList<>() + ); + + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + + // All frames are generated with the same signature therefore we can attach the row signature + return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); + } + private Function, Result> makeComputeManipulatorFn( final TimeseriesQuery query, final MetricManipulationFn fn, diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index a7785a01be1a..0109c9828e98 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -26,12 +26,21 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.MemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.BySegmentResultValue; import org.apache.druid.query.CacheStrategy; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.query.IterableRowsCursorHelper; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -47,15 +56,18 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +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.Optional; import java.util.function.BinaryOperator; /** @@ -548,6 +560,38 @@ public Sequence resultsAsArrays(TopNQuery query, Sequence> resultsAsFrames( + TopNQuery query, + Sequence> resultSequence, + MemoryAllocatorFactory memoryAllocatorFactory, + boolean useNestedForUnknownTypes + ) + { + final RowSignature rowSignature = resultArraySignature(query); + final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + resultsAsArrays(query, resultSequence), + rowSignature + ); + + RowSignature modifiedRowSignature = useNestedForUnknownTypes + ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) + : rowSignature; + FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + memoryAllocatorFactory, + rowSignature, + new ArrayList<>() + ); + + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + + return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); + } + static class ThresholdAdjustingQueryRunner implements QueryRunner> { private final QueryRunner> runner; diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index c0c952511778..eb7a870c172f 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -48,7 +48,7 @@ public class RowBasedCursor implements Cursor private long rowId = 0; - RowBasedCursor( + public RowBasedCursor( final RowWalker rowWalker, final RowAdapter rowAdapter, @Nullable final Filter filter, diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java index 9f5c89a6437d..d6241f197e85 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java +++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java @@ -41,7 +41,7 @@ public class RowWalker @Nullable // null = closed private Yielder rowYielder; - RowWalker(final Sequence rowSequence, final RowAdapter rowAdapter) + public RowWalker(final Sequence rowSequence, final RowAdapter rowAdapter) { this.rowSequence = rowSequence; this.timestampFunction = rowAdapter.timestampFunction(); diff --git a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java index 770d3632bd2e..0fa8935533db 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java +++ b/processing/src/main/java/org/apache/druid/segment/column/RowSignature.java @@ -26,6 +26,7 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.dimension.DimensionSpec; @@ -44,7 +45,7 @@ * Type signature for a row in a Druid datasource or query result. * * @see org.apache.druid.query.QueryToolChest#resultArraySignature which returns signatures for query results - * @see org.apache.druid.query.InlineDataSource#getRowSignature which returns signatures for inline datasources + * @see InlineDataSource#getRowSignature which returns signatures for inline datasources */ public class RowSignature implements ColumnInspector { diff --git a/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java new file mode 100644 index 000000000000..b35bda644a84 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/FrameBasedInlineJoinableFactory.java @@ -0,0 +1,64 @@ +/* + * 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.segment.join; + +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.query.InlineDataSource; +import org.apache.druid.segment.join.table.FrameBasedIndexedTable; +import org.apache.druid.segment.join.table.IndexedTableJoinable; + +import java.util.Optional; +import java.util.Set; + +/** + * Creates a joinable from the {@link FrameBasedInlineDataSource}. This materializes the datasource to an + * {@link InlineDataSource}, before creating the joinable on it, which carries the overhead of this conversion. + */ +public class FrameBasedInlineJoinableFactory implements JoinableFactory +{ + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return dataSource instanceof FrameBasedInlineDataSource; + } + + @Override + public Optional build(DataSource dataSource, JoinConditionAnalysis condition) + { + FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource; + + if (condition.canHashJoin()) { + final Set rightKeyColumns = condition.getRightEquiConditionKeys(); + return Optional.of( + new IndexedTableJoinable( + new FrameBasedIndexedTable( + frameBasedInlineDataSource, + rightKeyColumns, + DateTimes.nowUtc().toString() + ) + ) + ); + } + + return Optional.empty(); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java index 4eee53fde5dc..a265e15279f6 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java @@ -30,7 +30,8 @@ import java.util.Set; /** - * A {@link JoinableFactory} for {@link InlineDataSource}. It works by building an {@link IndexedTable}. + * A {@link JoinableFactory} for {@link InlineDataSource}. + * It works by building an {@link IndexedTable}. * * It is not valid to pass any other DataSource type to the "build" method. */ @@ -48,7 +49,7 @@ public boolean isDirectlyJoinable(DataSource dataSource) @Override public Optional build(final DataSource dataSource, final JoinConditionAnalysis condition) { - final InlineDataSource inlineDataSource = (InlineDataSource) dataSource; + InlineDataSource inlineDataSource = (InlineDataSource) dataSource; if (condition.canHashJoin()) { final Set rightKeyColumns = condition.getRightEquiConditionKeys(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java new file mode 100644 index 000000000000..a7de99905c39 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -0,0 +1,336 @@ +/* + * 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.segment.join.table; + +import com.google.common.base.Preconditions; +import com.google.common.math.IntMath; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.read.columnar.FrameColumnReader; +import org.apache.druid.frame.read.columnar.FrameColumnReaders; +import org.apache.druid.frame.segment.FrameStorageAdapter; +import org.apache.druid.frame.segment.columnar.FrameQueryableIndex; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +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.guava.Sequences; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.NilColumnValueSelector; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.BaseColumn; +import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +public class FrameBasedIndexedTable implements IndexedTable +{ + private static final Logger LOG = new Logger(FrameBasedIndexedTable.class); + + private final Set keyColumns; + private final RowSignature rowSignature; + private final String version; + private final List keyColumnsIndexes; + private final int numRows; + private final List frameQueryableIndexes = new ArrayList<>(); + private final List cumulativeRowCount = new ArrayList<>(); + + + public FrameBasedIndexedTable( + final FrameBasedInlineDataSource frameBasedInlineDataSource, + final Set keyColumns, + final String version + ) + { + this.keyColumns = keyColumns; + this.version = version; + this.rowSignature = frameBasedInlineDataSource.getRowSignature(); + + int rowCount = 0; + for (FrameSignaturePair frameSignaturePair : frameBasedInlineDataSource.getFrames()) { + Frame frame = frameSignaturePair.getFrame(); + RowSignature frameRowSignature = frameSignaturePair.getRowSignature(); + frameQueryableIndexes.add(new FrameQueryableIndex( + frame, + frameRowSignature, + createColumnReaders(frameRowSignature) + )); + rowCount += frame.numRows(); + cumulativeRowCount.add(rowCount); + } + + this.numRows = rowCount; + + final ArrayList indexBuilders = new ArrayList<>(rowSignature.size()); + final List keyColumnNames = new ArrayList<>(keyColumns.size()); + + for (int i = 0; i < rowSignature.size(); i++) { + final RowBasedIndexBuilder m; + final String columnName = rowSignature.getColumnName(i); + if (keyColumns.contains(columnName)) { + final ColumnType keyType = + rowSignature.getColumnType(i).orElse(IndexedTableJoinMatcher.DEFAULT_KEY_TYPE); + + m = new RowBasedIndexBuilder(keyType); + keyColumnNames.add(columnName); + } else { + m = null; + } + indexBuilders.add(m); + } + + final Sequence cursors = Sequences.concat( + frameBasedInlineDataSource + .getFrames() + .stream() + .map(frameSignaturePair -> { + Frame frame = frameSignaturePair.getFrame(); + RowSignature rowSignature = frameSignaturePair.getRowSignature(); + FrameStorageAdapter frameStorageAdapter = + new FrameStorageAdapter(frame, FrameReader.create(rowSignature), Intervals.ETERNITY); + return frameStorageAdapter.makeCursors( + null, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ); + }) + .collect(Collectors.toList()) + ); + + final Sequence sequence = Sequences.map( + cursors, + cursor -> { + if (cursor == null) { + return 0; + } + int rowNumber = 0; + ColumnSelectorFactory columnSelectorFactory = cursor.getColumnSelectorFactory(); + + // this should really be optimized to use dimension selectors where possible to populate indexes from bitmap + // indexes, but, an optimization for another day + final List selectors = keyColumnNames + .stream() + .map(columnSelectorFactory::makeColumnValueSelector) + .collect(Collectors.toList()); + + while (!cursor.isDone()) { + for (int keyColumnSelectorIndex = 0; keyColumnSelectorIndex < selectors.size(); keyColumnSelectorIndex++) { + final String keyColumnName = keyColumnNames.get(keyColumnSelectorIndex); + final int columnPosition = rowSignature.indexOf(keyColumnName); + final RowBasedIndexBuilder keyColumnIndexBuilder = indexBuilders.get(columnPosition); + keyColumnIndexBuilder.add(selectors.get(keyColumnSelectorIndex).getObject()); + } + + if (rowNumber % 100_000 == 0) { + if (rowNumber == 0) { + LOG.debug("Indexed first row for frame based datasource"); + } else { + LOG.debug("Indexed row %s for frame based datasource", rowNumber); + } + } + rowNumber++; + cursor.advance(); + } + return rowNumber; + } + ); + + Integer totalRows = sequence.accumulate(0, (accumulated, in) -> accumulated += in); + + this.keyColumnsIndexes = indexBuilders.stream() + .map(builder -> builder != null ? builder.build() : null) + .collect(Collectors.toList()); + + LOG.info("Created FrameBasedIndexedTable with %s rows.", totalRows); + } + + @Override + public String version() + { + return version; + } + + @Override + public Set keyColumns() + { + return keyColumns; + } + + @Override + public RowSignature rowSignature() + { + return rowSignature; + } + + @Override + public int numRows() + { + return numRows; + } + + @Override + public Index columnIndex(int column) + { + return RowBasedIndexedTable.getKeyColumnIndex(column, keyColumnsIndexes); + + } + + @Override + public Reader columnReader(int column) + { + + if (!rowSignature.contains(column)) { + throw new IAE("Column[%d] is not a valid column for the frame based datasource", column); + } + + String columnName = rowSignature.getColumnName(column); + final SimpleAscendingOffset offset = new SimpleAscendingOffset(numRows()); + final List> columnValueSelectors = new ArrayList<>(); + final Set closeables = new HashSet<>(); + + for (QueryableIndex frameQueryableIndex : frameQueryableIndexes) { + ColumnHolder columnHolder = frameQueryableIndex.getColumnHolder(columnName); + if (columnHolder == null) { + columnValueSelectors.add(NilColumnValueSelector.instance()); + } else { + BaseColumn baseColumn = columnHolder.getColumn(); + columnValueSelectors.add(baseColumn.makeColumnValueSelector(offset)); + closeables.add(baseColumn); + } + } + + return new Reader() + { + @Nullable + @Override + public Object read(int row) + { + int frameIndex = binSearch(cumulativeRowCount, row); + if (frameIndex == frameQueryableIndexes.size()) { + throw new IndexOutOfBoundsException( + StringUtils.format("Requested row index [%d], Max row count [%d]", row, numRows()) + ); + } + // The offset needs to be changed as well + int adjustedOffset = frameIndex == 0 + ? row + : IntMath.checkedSubtract(row, cumulativeRowCount.get(frameIndex - 1)); + offset.setCurrentOffset(adjustedOffset); + return columnValueSelectors.get(frameIndex).getObject(); + } + + @Override + public void close() throws IOException + { + for (Closeable c : closeables) { + c.close(); + } + } + }; + } + + @Override + public boolean isCacheable() + { + return false; + } + + @Override + public void close() + { + + } + + @Override + public Optional acquireReferences() + { + return Optional.of( + () -> { + } + ); + } + + private List createColumnReaders(RowSignature rowSignature) + { + final List columnReaders = new ArrayList<>(rowSignature.size()); + + for (int columnNumber = 0; columnNumber < rowSignature.size(); columnNumber++) { + ColumnType columnType = Preconditions.checkNotNull( + rowSignature.getColumnType(columnNumber).orElse(null), + "Type for column [%s]", + rowSignature.getColumnName(columnNumber) + ); + columnReaders.add(FrameColumnReaders.create(columnNumber, columnType)); + } + + return columnReaders; + } + + /** + * This method finds out the frame which contains the row indexed "row" from the cumulative array + * This is basically a binary search where we have to find the FIRST element which is STRICTLY GREATER than + * the "row" provided + *

+ * Note: row is the index (therefore it is 0-indexed) + */ + private int binSearch(List arr, int row) + { + int start = 0; + int end = arr.size(); + + while (start < end) { + + // This will be rounded down in case (start + end) is odd. Therefore middle will always be less than + // end and will be equal to or greater than start + int middle = start + (end - start) / 2; + + // If the "middle" satisfies the below predicate, then we can move the end backward because every element after + // middle would be satisfying the predicate + if (arr.get(middle) > row) { + end = middle; + } else { + start = middle + 1; + } + } + + return start; // Note: at this point, end == start + } +} diff --git a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java new file mode 100644 index 000000000000..fbbc089255b5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java @@ -0,0 +1,156 @@ +/* + * 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; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.HeapMemoryAllocator; +import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; + +public class FrameBasedInlineDataSourceSerializerTest +{ + + static { + NullHandling.initializeForTests(); + } + + private static final Interval INTERVAL = Intervals.of("2000/P1Y"); + + private static final RowSignature FOO_INLINE_SIGNATURE = RowSignature.builder() + .addTimeColumn() + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) + .build(); + + private static final InlineDataSource FOO_INLINE = InlineDataSource.fromIterable( + ImmutableList.builder() + .add(new Object[]{INTERVAL.getStartMillis(), "x", 1}) + .add(new Object[]{INTERVAL.getStartMillis(), "x", 2}) + .add(new Object[]{INTERVAL.getStartMillis(), "y", 3}) + .add(new Object[]{INTERVAL.getStartMillis(), "z", 4}) + .build(), + FOO_INLINE_SIGNATURE + ); + + private static final RowSignature BAR_INLINE_SIGNATURE = RowSignature.builder() + .addTimeColumn() + .add("s", ColumnType.STRING) + .add("n", ColumnType.LONG) + .build(); + + + private static final InlineDataSource BAR_INLINE = InlineDataSource.fromIterable( + ImmutableList.builder() + .add(new Object[]{INTERVAL.getStartMillis(), "a", 1}) + .add(new Object[]{INTERVAL.getStartMillis(), "a", 2}) + .add(new Object[]{INTERVAL.getStartMillis(), "b", 3}) + .add(new Object[]{INTERVAL.getStartMillis(), "c", 4}) + .build(), + BAR_INLINE_SIGNATURE + ); + + private static final RowSignature MULTI_VALUE_INLINE_SIGNATURE = RowSignature.builder() + .addTimeColumn() + .add("s", ColumnType.STRING_ARRAY) + .add("n", ColumnType.LONG) + .build(); + + private static final InlineDataSource MULTI_VALUE_INLINE = InlineDataSource.fromIterable( + ImmutableList.builder() + .add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("a", "b"), 1}) + .add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("a", "c"), 2}) + .add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("b"), 3}) + .add(new Object[]{INTERVAL.getStartMillis(), ImmutableList.of("c"), 4}) + .build(), + MULTI_VALUE_INLINE_SIGNATURE + ); + + ObjectMapper objectMapper = new DefaultObjectMapper(); + + @Test + public void serialize() throws JsonProcessingException + { + assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource( + convertToFrameBasedInlineDataSource(FOO_INLINE, FOO_INLINE_SIGNATURE), + FOO_INLINE + ); + assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource( + convertToFrameBasedInlineDataSource(BAR_INLINE, BAR_INLINE_SIGNATURE), + BAR_INLINE + ); + assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource( + convertToFrameBasedInlineDataSource(MULTI_VALUE_INLINE, MULTI_VALUE_INLINE_SIGNATURE), + MULTI_VALUE_INLINE + ); + } + + private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource( + InlineDataSource inlineDataSource, + RowSignature rowSignature + ) + { + Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable( + inlineDataSource.getRows(), + rowSignature + ); + RowSignature modifiedRowSignature = FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature); + Sequence frames = FrameCursorUtils.cursorToFrames( + cursor, + FrameWriters.makeFrameWriterFactory( + FrameType.ROW_BASED, + new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + modifiedRowSignature, + new ArrayList<>() + ) + ); + return new FrameBasedInlineDataSource( + frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).toList(), + modifiedRowSignature + ); + } + + private void assertConversionBetweenFrameBasedAndIterableBasedInlineDataSource( + FrameBasedInlineDataSource frameBasedInlineDataSource, + InlineDataSource inlineDataSource + ) throws JsonProcessingException + { + String s = objectMapper.writeValueAsString(frameBasedInlineDataSource); + DataSource back = objectMapper.readValue(s, DataSource.class); + Assert.assertEquals(inlineDataSource, back); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java new file mode 100644 index 000000000000..1acaceabbd60 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java @@ -0,0 +1,82 @@ +/* + * 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; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + + +public class IterableRowsCursorHelperTest +{ + + List rows = ImmutableList.of( + new Object[]{1, "a"}, + new Object[]{3, "b"}, + new Object[]{2, "b"} + ); + + RowSignature rowSignature = RowSignature.builder() + .add("dim1", ColumnType.LONG) + .add("dim2", ColumnType.STRING) + .build(); + + @Test + public void getCursorFromIterable() + { + Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature); + testCursorMatchesRowSequence(cursor, rowSignature, rows); + } + + @Test + public void getCursorFromSequence() + { + + Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature); + testCursorMatchesRowSequence(cursor, rowSignature, rows); + } + + private void testCursorMatchesRowSequence( + Cursor cursor, + RowSignature expectedRowSignature, + List expectedRows + ) + { + List actualRows = new ArrayList<>(); + while (!cursor.isDone()) { + Object[] row = new Object[expectedRowSignature.size()]; + for (int i = 0; i < expectedRowSignature.size(); ++i) { + ColumnValueSelector columnValueSelector = cursor.getColumnSelectorFactory() + .makeColumnValueSelector(expectedRowSignature.getColumnName(i)); + row[i] = columnValueSelector.getObject(); + } + actualRows.add(row); + cursor.advance(); + } + QueryToolChestTestHelper.assertArrayResultsEquals(expectedRows, Sequences.simple(actualRows)); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java b/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java new file mode 100644 index 000000000000..afe7a95cfdcf --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scan/ConcatCursorTest.java @@ -0,0 +1,287 @@ +/* + * 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.segment.Cursor; +import org.apache.druid.segment.ListCursor; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; + +public class ConcatCursorTest +{ + @Test + public void testConcatCursor() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b")); + Cursor dummyCursor2 = new ListCursor(new ArrayList<>()); + Cursor cursor2 = new ListCursor(ImmutableList.of("c", "d")); + Cursor dummyCursor3 = new ListCursor(new ArrayList<>()); + + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + dummyCursor1, + cursor1, + dummyCursor2, + cursor2, + dummyCursor3 + )); + + List tempList = new ArrayList<>(); + // Initial iteration + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b", "c", "d"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 3; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b", "c"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b", "c", "d"), tempList); + } + + @Test + public void testConcatCursorOfEmptyCursors() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor2 = new ListCursor(new ArrayList<>()); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + dummyCursor1, + dummyCursor2 + )); + Assert.assertTrue(concatCursor.isDone()); + } + + @Test + public void testConcatCursorWhenBeginningCursorIsEmpty() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b")); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + dummyCursor1, + cursor1 + )); + + List tempList = new ArrayList<>(); + + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 1; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + } + + @Test + public void testConcatCursorWhenEndingCursorIsEmpty() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b")); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + cursor1, + dummyCursor1 + )); + + List tempList = new ArrayList<>(); + + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 1; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + } + + @Test + public void testConcatCursorWhenMultipleEmptyCursorsAtBeginning() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor2 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor3 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b")); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + dummyCursor1, + dummyCursor2, + dummyCursor3, + cursor1 + )); + + List tempList = new ArrayList<>(); + + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 1; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + } + + @Test + public void testConcatCursorWhenMultipleEmptyCursorsAtEnd() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor2 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor3 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a", "b")); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + cursor1, + dummyCursor1, + dummyCursor2, + dummyCursor3 + )); + + List tempList = new ArrayList<>(); + + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 1; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + } + + @Test + public void testConcatCursorWhenMultipleEmptyCursorsAtTheMiddle() + { + Cursor dummyCursor1 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor2 = new ListCursor(new ArrayList<>()); + Cursor dummyCursor3 = new ListCursor(new ArrayList<>()); + Cursor cursor1 = new ListCursor(ImmutableList.of("a")); + Cursor cursor2 = new ListCursor(ImmutableList.of("b")); + Cursor concatCursor = new ConcatCursor(ImmutableList.of( + cursor1, + dummyCursor1, + dummyCursor2, + dummyCursor3, + cursor2 + )); + + List tempList = new ArrayList<>(); + + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + + // Check if reset() works after exhausting the cursor + concatCursor.reset(); + tempList.clear(); + for (int i = 0; i < 1; ++i) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a"), tempList); + + // Check if reset() works from the middle + concatCursor.reset(); + tempList.clear(); + while (!concatCursor.isDone()) { + tempList.add(concatCursor.getColumnSelectorFactory().makeColumnValueSelector("ignored").getObject()); + concatCursor.advance(); + } + Assert.assertEquals(ImmutableList.of("a", "b"), tempList); + } +} 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 bdd9b01838c7..63c5b3819216 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 @@ -20,15 +20,24 @@ package org.apache.druid.query.scan; import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.frame.allocation.HeapMemoryAllocator; +import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; 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.FrameBasedInlineDataSource; +import org.apache.druid.query.FrameSignaturePair; import org.apache.druid.query.QueryToolChestTestHelper; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.serde.ComplexMetrics; import org.junit.Assert; import org.junit.Test; @@ -40,12 +49,28 @@ public class ScanQueryQueryToolChestTest { + + static { + NullHandling.initializeForTests(); + ComplexMetrics.registerSerde(NestedDataComplexTypeSerde.TYPE_NAME, NestedDataComplexTypeSerde.INSTANCE); + } + // Expected results for the resultsAsArrays test methods. - private static final List ARRAY_RESULTS = ImmutableList.of( + private static final List ARRAY_RESULTS_1 = ImmutableList.of( new Object[]{null, 3.2}, new Object[]{"x", "y"} ); + private static final List ARRAY_RESULTS_2 = ImmutableList.of( + new Object[]{"str1", 3.2}, + new Object[]{"str2", 3.3} + ); + + private static final List ARRAY_RESULTS_3 = ImmutableList.of( + new Object[]{3.4, "str3"}, + new Object[]{3.5, "str4"} + ); + private final ScanQueryQueryToolChest toolChest = new ScanQueryQueryToolChest( new ScanQueryConfig(), new DefaultGenericQueryMetricsFactory() @@ -121,7 +146,7 @@ public void test_resultsAsArrays_columnsNotSpecifiedListResults() QueryToolChestTestHelper.assertArrayResultsEquals( ImmutableList.of(new Object[]{}, new Object[]{}), - toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) + toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) ); } @@ -137,7 +162,7 @@ public void test_resultsAsArrays_columnsNotSpecifiedCompactedListResults() QueryToolChestTestHelper.assertArrayResultsEquals( ImmutableList.of(new Object[]{}, new Object[]{}), - toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) + toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) ); } @@ -153,8 +178,8 @@ public void test_resultsAsArrays_columnsSpecifiedListResults() .build(); QueryToolChestTestHelper.assertArrayResultsEquals( - ARRAY_RESULTS, - toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) + ARRAY_RESULTS_1, + toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST)) ); } @@ -170,22 +195,185 @@ public void test_resultsAsArrays_columnsSpecifiedCompactedListResults() .build(); QueryToolChestTestHelper.assertArrayResultsEquals( - ARRAY_RESULTS, - toolChest.resultsAsArrays(scanQuery, makeResults(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) + ARRAY_RESULTS_1, + toolChest.resultsAsArrays(scanQuery, makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)) + ); + } + + @Test + public void test_resultsAsFrames_batchingWorksAsExpectedWithDistinctColumnTypes() + { + + final ScanQuery scanQuery = + Druids.newScanQueryBuilder() + .dataSource("foo") + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))) + .columns("foo", "bar", "foo2", "bar2", "foo3", "bar3") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) + .build(); + + List frames = + toolChest.resultsAsFrames( + scanQuery, + Sequences.concat(makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST), results2(), results3()), + new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + true + ).get().toList(); + + + Assert.assertEquals(3, frames.size()); + + RowSignature resultRowSignature = RowSignature.builder() + .add("foo", null) + .add("bar", null) + .add("foo2", null) + .add("bar2", null) + .add("foo3", null) + .add("bar3", null) + .build(); + + Sequence rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence(); + + QueryToolChestTestHelper.assertArrayResultsEquals( + ImmutableList.of( + new Object[]{null, StructuredData.wrap(3.2), null, null, null, null}, + new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null}, + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null}, + new Object[]{null, null, null, null, 3.4, "str3"}, + new Object[]{null, null, null, null, 3.5, "str4"} + ), + rows + ); + } + + @Test + public void test_resultsAsFrames_batchingWorksAsExpectedWithMixedColumnTypes() + { + + final ScanQuery scanQuery = + Druids.newScanQueryBuilder() + .dataSource("foo") + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))) + .columns("foo", "bar", "foo2", "bar2", "foo3", "bar3") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) + .build(); + + List frames = + toolChest.resultsAsFrames( + scanQuery, + Sequences.concat( + results2(), + makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST), + makeResults1(ScanQuery.ResultFormat.RESULT_FORMAT_LIST), + results3(), + results2(), + results2(), + results3() + ), + new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + true + ).get().toList(); + + + Assert.assertEquals(5, frames.size()); + + RowSignature resultRowSignature = RowSignature.builder() + .add("foo", null) + .add("bar", null) + .add("foo2", null) + .add("bar2", null) + .add("foo3", null) + .add("bar3", null) + .build(); + + Sequence rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence(); + + QueryToolChestTestHelper.assertArrayResultsEquals( + ImmutableList.of( + // results2 + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null}, + // results1 + new Object[]{null, StructuredData.wrap(3.2), null, null, null, null}, + new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null}, + // results1 + new Object[]{null, StructuredData.wrap(3.2), null, null, null, null}, + new Object[]{StructuredData.wrap("x"), StructuredData.wrap("y"), null, null, null, null}, + // results3 + new Object[]{null, null, null, null, 3.4, "str3"}, + new Object[]{null, null, null, null, 3.5, "str4"}, + // results2 + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null}, + // results2 + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null}, + // results3 + new Object[]{null, null, null, null, 3.4, "str3"}, + new Object[]{null, null, null, null, 3.5, "str4"} + ), + rows + ); + } + + + @Test + public void test_resultsAsFrames_batchingWorksAsExpectedWithSameColumnTypes() + { + + final ScanQuery scanQuery = + Druids.newScanQueryBuilder() + .dataSource("foo") + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2000/3000")))) + .columns("foo", "bar", "foo2", "bar2", "foo3", "bar3") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_LIST) + .build(); + + List frames = + toolChest.resultsAsFrames( + scanQuery, + Sequences.concat(results2(), results2()), + new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + true + ).get().toList(); + + + Assert.assertEquals(1, frames.size()); + + RowSignature resultRowSignature = RowSignature.builder() + .add("foo", null) + .add("bar", null) + .add("foo2", null) + .add("bar2", null) + .add("foo3", null) + .add("bar3", null) + .build(); + + Sequence rows = new FrameBasedInlineDataSource(frames, resultRowSignature).getRowsAsSequence(); + + QueryToolChestTestHelper.assertArrayResultsEquals( + ImmutableList.of( + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null}, + new Object[]{null, null, "str1", 3.2, null, null}, + new Object[]{null, null, "str2", 3.3, null, null} + ), + rows ); } /** * Returns results that are a single ScanResultValue with two rows, each row having columns "foo" and "bar". */ - private static Sequence makeResults(final ScanQuery.ResultFormat resultFormat) + private static Sequence makeResults1(final ScanQuery.ResultFormat resultFormat) { final List rows = new ArrayList<>(); // Generate rows in the manner of ScanQueryEngine. switch (resultFormat) { case RESULT_FORMAT_LIST: - ARRAY_RESULTS.forEach(arr -> { + ARRAY_RESULTS_1.forEach(arr -> { final Map m = new HashMap<>(); m.put("foo", arr[0]); m.put("bar", arr[1]); @@ -193,7 +381,7 @@ private static Sequence makeResults(final ScanQuery.ResultForma }); break; case RESULT_FORMAT_COMPACTED_LIST: - ARRAY_RESULTS.forEach(arr -> rows.add(Arrays.asList(arr))); + ARRAY_RESULTS_1.forEach(arr -> rows.add(Arrays.asList(arr))); break; default: throw new ISE("Cannot generate resultFormat '%s'", resultFormat); @@ -209,4 +397,66 @@ private static Sequence makeResults(final ScanQuery.ResultForma ) ); } + + /** + * Returns results that are a single ScanResultValue with two rows, each row having columns "foo2" and "bar2". This + * generates results in the format of {@link ScanQuery.ResultFormat#RESULT_FORMAT_LIST} + */ + private static Sequence results2() + { + final List rows = new ArrayList<>(); + + ARRAY_RESULTS_2.forEach(arr -> { + final Map m = new HashMap<>(); + m.put("foo2", arr[0]); + m.put("bar2", arr[1]); + rows.add(m); + }); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("foo2", ColumnType.STRING); + rowSignatureBuilder.add("bar2", ColumnType.DOUBLE); + + return Sequences.simple( + ImmutableList.of( + new ScanResultValue( + null, + ImmutableList.of("foo2", "bar2"), + rows, + rowSignatureBuilder.build() + ) + ) + ); + } + + /** + * Returns results that are a single ScanResultValue with two rows, each row having columns "foo3" and "bar3". This + * generates results in the format of {@link ScanQuery.ResultFormat#RESULT_FORMAT_LIST} + */ + private static Sequence results3() + { + final List rows = new ArrayList<>(); + + ARRAY_RESULTS_3.forEach(arr -> { + final Map m = new HashMap<>(); + m.put("foo3", arr[0]); + m.put("bar3", arr[1]); + rows.add(m); + }); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.add("foo3", ColumnType.DOUBLE); + rowSignatureBuilder.add("bar3", ColumnType.STRING); + + return Sequences.simple( + ImmutableList.of( + new ScanResultValue( + null, + ImmutableList.of("foo3", "bar3"), + rows, + rowSignatureBuilder.build() + ) + ) + ); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java new file mode 100644 index 000000000000..bbb5704a561e --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java @@ -0,0 +1,372 @@ +/* + * 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.segment.join.table; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import it.unimi.dsi.fastutil.ints.IntBidirectionalIterator; +import it.unimi.dsi.fastutil.ints.IntSortedSet; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.FrameType; +import org.apache.druid.frame.allocation.HeapMemoryAllocator; +import org.apache.druid.frame.allocation.SingleMemoryAllocatorFactory; +import org.apache.druid.frame.segment.FrameCursorUtils; +import org.apache.druid.frame.write.FrameWriterFactory; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.query.FrameSignaturePair; +import org.apache.druid.query.IterableRowsCursorHelper; +import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; + +public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest +{ + private static final String STRING_COL_1 = "market"; + private static final String LONG_COL_1 = "longNumericNull"; + private static final String DOUBLE_COL_1 = "doubleNumericNull"; + private static final String FLOAT_COL_1 = "floatNumericNull"; + private static final String STRING_COL_2 = "partial_null_column"; + private static final String MULTI_VALUE_COLUMN = "placementish"; + private static final String NON_INDEXED_STRING_COL = "nonIndexedString"; + private static final String NON_INDEXED_LONG_COL = "nonIndexedNumeric"; + private static final String NON_INDEXED_DOUBLE_COL = "nonIndexedDouble"; + private static final String NON_INDEXED_FLOAT_COL = "nonIndexedFloat"; + private static final String DIM_NOT_EXISTS = "DIM_NOT_EXISTS"; + + private static final List DATASOURCE_ROWS = + ImmutableList.builder() + .add( + new Object[]{ + "spot", + 1L, + null, + 3.1f, + "preferred", + new Object[]{"val1", "val2"}, + "spot", + 1L, + null, + 3.1f + }) + .add(new Object[]{ + "total_market", + 1L, + 1.2d, + 3.2f, + null, + new Object[]{"val1", "val2"}, + "total_market", + 1L, + 1.2d, + 3.2f + }) + .add(new Object[]{ + "spot", + 2L, + 1.3d, + 3.1f, + "preferred", + new Object[]{"val1", "val2"}, + "spot", + 2L, + 1.3d, + 3.1f + }) + .add(new Object[]{ + "upfront", + 1L, + 1.5d, + 3.5f, + "preferred", + new Object[]{"val1", "val2"}, + "upfront", + 1L, + 1.5d, + 3.5f + }) + .add(new Object[]{ + "total_market", + null, + 1.1d, + 3.1f, + null, + new Object[]{"val1", "val2"}, + "total_market", + null, + 1.1d, + 3.1f + }) + .add(new Object[]{ + "upfront", + 2L, + 1.5d, + null, + "preferred", + new Object[]{"val1", "val2"}, + "upfront", + 2L, + 1.5d, + null + }) + .add(new Object[]{ + "upfront", + 4L, + 1.1d, + 3.9f, + "preferred", + new Object[]{"val1", "val2"}, + "upfront", + 4L, + 1.1d, + 3.9f + }) + .add(new Object[]{ + "total_market", + 1L, + 1.7d, + 3.8f, + "preferred", + new Object[]{"val1", "val2"}, + "total_market", + 1L, + 1.7d, + 3.8f + }) + .add(new Object[]{ + "spot", + 5L, + 1.8d, + 3.1f, + null, + new Object[]{"val1", "val2"}, + "spot", + 5L, + 1.8d, + 3.1f + }) + .build(); + + private static final RowSignature ROW_SIGNATURE = + RowSignature.builder() + .add(STRING_COL_1, ColumnType.STRING) + .add(LONG_COL_1, ColumnType.LONG) + .add(DOUBLE_COL_1, ColumnType.DOUBLE) + .add(FLOAT_COL_1, ColumnType.FLOAT) + .add(STRING_COL_2, ColumnType.STRING) + .add(MULTI_VALUE_COLUMN, ColumnType.STRING_ARRAY) + .add(NON_INDEXED_STRING_COL, ColumnType.STRING) + .add(NON_INDEXED_LONG_COL, ColumnType.LONG) + .add(NON_INDEXED_DOUBLE_COL, ColumnType.DOUBLE) + .add(NON_INDEXED_FLOAT_COL, ColumnType.FLOAT) + .build(); + + private static final Set KEY_COLUMNS = ImmutableSet.builder() + .add(STRING_COL_1) + .add(STRING_COL_2) + .add(LONG_COL_1) + .add(DOUBLE_COL_1) + .add(FLOAT_COL_1) + .add(MULTI_VALUE_COLUMN) + .add(DIM_NOT_EXISTS) + .build(); + + + private FrameBasedInlineDataSource dataSource; + private FrameBasedIndexedTable frameBasedIndexedTable; + + @Before + public void setup() + { + Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE); + FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( + FrameType.COLUMNAR, + new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), + ROW_SIGNATURE, + new ArrayList<>() + ); + Frame frame = Iterables.getOnlyElement(FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).toList()); + + dataSource = new FrameBasedInlineDataSource( + ImmutableList.of(new FrameSignaturePair(frame, ROW_SIGNATURE)), + ROW_SIGNATURE + ); + + frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test"); + + } + + @Test + public void testInitShouldGenerateCorrectTable() + { + Assert.assertEquals(9, frameBasedIndexedTable.numRows()); + } + + @Test + public void testStringKeyColumn() + { + final String[] vals = new String[]{"spot", "total_market", "upfront"}; + checkIndexAndReader(STRING_COL_1, vals); + } + + @Test + public void testNullableStringKeyColumn() + { + final String[] vals = new String[]{null, "preferred"}; + checkIndexAndReader(STRING_COL_2, vals); + } + + @Test + public void testMultiValueStringKeyColumn() + { + final Object[] nonMatchingVals = new Object[]{ImmutableList.of("a", "preferred")}; + checkIndexAndReader(MULTI_VALUE_COLUMN, new Object[0], nonMatchingVals); + } + + @Test + public void testLongKeyColumn() + { + final Long[] vals = new Long[]{NullHandling.replaceWithDefault() ? 0L : null, 1L, 2L, 4L, 5L}; + checkIndexAndReader(LONG_COL_1, vals); + } + + @Test + public void testFloatKeyColumn() + { + final Float[] vals = new Float[]{NullHandling.replaceWithDefault() ? 0.0f : null, 3.1f, 3.2f, 3.5f, 3.8f, 3.9f}; + checkIndexAndReader(FLOAT_COL_1, vals); + } + + @Test + public void testDoubleKeyColumn() + { + final Double[] vals = new Double[]{ + NullHandling.replaceWithDefault() ? 0.0 : null, 1.1d, 1.2d, 1.3d, 1.5d, 1.7d, 1.8d + }; + checkIndexAndReader(DOUBLE_COL_1, vals); + } + + @Test + public void testStringNonKeyColumn() + { + checkNonIndexedReader(NON_INDEXED_STRING_COL); + } + + @Test + public void testLongNonKeyColumn() + { + checkNonIndexedReader(NON_INDEXED_LONG_COL); + } + + @Test + public void testFloatNonKeyColumn() + { + checkNonIndexedReader(NON_INDEXED_FLOAT_COL); + } + + @Test + public void testDoubleNonKeyColumn() + { + checkNonIndexedReader(NON_INDEXED_DOUBLE_COL); + } + + @Test + public void testIsCacheable() + { + Assert.assertFalse(frameBasedIndexedTable.isCacheable()); + } + + private void checkIndexAndReader(String columnName, Object[] vals) + { + checkIndexAndReader(columnName, vals, new Object[0]); + } + + private void checkIndexAndReader(String columnName, Object[] vals, Object[] nonmatchingVals) + { + checkColumnReader(columnName); + try (final Closer closer = Closer.create()) { + final int columnIndex = ROW_SIGNATURE.indexOf(columnName); + final IndexedTable.Reader reader = frameBasedIndexedTable.columnReader(columnIndex); + closer.register(reader); + final IndexedTable.Index valueIndex = frameBasedIndexedTable.columnIndex(columnIndex); + + for (Object val : vals) { + final IntSortedSet valIndex = valueIndex.find(val); + if (val == null) { + Assert.assertEquals(0, valIndex.size()); + } else { + Assert.assertTrue(valIndex.size() > 0); + final IntBidirectionalIterator rowIterator = valIndex.iterator(); + while (rowIterator.hasNext()) { + Assert.assertEquals(val, reader.read(rowIterator.nextInt())); + } + } + } + for (Object val : nonmatchingVals) { + final IntSortedSet valIndex = valueIndex.find(val); + Assert.assertEquals(0, valIndex.size()); + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private void checkNonIndexedReader(String columnName) + { + // make sure it doesn't have an index since it isn't a key column + checkColumnReader(columnName); + int columnIndex = ROW_SIGNATURE.indexOf(columnName); + try { + Assert.assertNull(frameBasedIndexedTable.columnIndex(columnIndex)); + } + catch (IAE iae) { + Assert.assertEquals(StringUtils.format("Column[%d] is not a key column", columnIndex), iae.getMessage()); + } + } + + + private void checkColumnReader(String columnName) + { + int numRows = DATASOURCE_ROWS.size(); + + int columnNumber = ROW_SIGNATURE.indexOf(columnName); + IndexedTable.Reader reader = frameBasedIndexedTable.columnReader(columnNumber); + List originalRows = dataSource.getRowsAsSequence().toList(); + for (int i = 0; i < numRows; ++i) { + Object original = originalRows.get(i)[columnNumber]; + Assert.assertEquals(original, reader.read(i)); + } + } +} diff --git a/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java b/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java index e8f803dc7730..102f1b349b3e 100644 --- a/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/JoinableFactoryModule.java @@ -27,10 +27,12 @@ import com.google.inject.multibindings.MapBinder; import com.google.inject.multibindings.Multibinder; import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.segment.join.BroadcastTableJoinableFactory; +import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory; import org.apache.druid.segment.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.LookupJoinableFactory; @@ -50,6 +52,7 @@ public class JoinableFactoryModule implements Module static final Map, Class> FACTORY_MAPPINGS = ImmutableMap.of( InlineDataSource.class, InlineJoinableFactory.class, + FrameBasedInlineDataSource.class, FrameBasedInlineJoinableFactory.class, LookupDataSource.class, LookupJoinableFactory.class, GlobalTableDataSource.class, BroadcastTableJoinableFactory.class ); diff --git a/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java b/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java index 93b7090a0835..a6cc1ce987b9 100644 --- a/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java +++ b/server/src/main/java/org/apache/druid/guice/SegmentWranglerModule.java @@ -26,8 +26,10 @@ import com.google.inject.Scopes; import com.google.inject.multibindings.MapBinder; import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.segment.FrameBasedInlineSegmentWrangler; import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.LookupSegmentWrangler; import org.apache.druid.segment.MapSegmentWrangler; @@ -47,6 +49,7 @@ public class SegmentWranglerModule implements Module static final Map, Class> WRANGLER_MAPPINGS = ImmutableMap.of( InlineDataSource.class, InlineSegmentWrangler.class, + FrameBasedInlineDataSource.class, FrameBasedInlineSegmentWrangler.class, LookupDataSource.class, LookupSegmentWrangler.class ); diff --git a/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java b/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java new file mode 100644 index 000000000000..407bbf750faa --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/FrameBasedInlineSegmentWrangler.java @@ -0,0 +1,55 @@ +/* + * 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.segment; + +import org.apache.druid.frame.read.FrameReader; +import org.apache.druid.frame.segment.FrameSegment; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; + +public class FrameBasedInlineSegmentWrangler implements SegmentWrangler +{ + + private static final String SEGMENT_ID = "inline"; + + @Override + public Iterable getSegmentsForIntervals( + DataSource dataSource, + Iterable intervals + ) + { + final FrameBasedInlineDataSource frameBasedInlineDataSource = (FrameBasedInlineDataSource) dataSource; + + return () -> frameBasedInlineDataSource + .getFrames() + .stream() + .map( + frameSignaturePair -> new FrameSegment( + frameSignaturePair.getFrame(), + FrameReader.create(frameSignaturePair.getRowSignature()), + SegmentId.dummy(SEGMENT_ID) + ) + ) + .iterator(); + + } +} diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java index 61f7817e2fd4..d66055229bc5 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -28,13 +28,18 @@ import org.apache.druid.client.DirectDruidClient; import org.apache.druid.client.cache.Cache; import org.apache.druid.client.cache.CacheConfig; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; 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.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DataSource; import org.apache.druid.query.FluentQueryRunnerBuilder; +import org.apache.druid.query.FrameBasedInlineDataSource; +import org.apache.druid.query.FrameSignaturePair; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.PostProcessingOperator; @@ -65,9 +70,12 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Queue; import java.util.Stack; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; /** @@ -81,6 +89,10 @@ */ public class ClientQuerySegmentWalker implements QuerySegmentWalker { + + private static final Logger log = new Logger(ClientQuerySegmentWalker.class); + private static final int FRAME_SIZE = 8_000_000; + private final ServiceEmitter emitter; private final QuerySegmentWalker clusterClient; private final QuerySegmentWalker localClient; @@ -163,12 +175,19 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable QueryRunner getQueryRunnerForIntervals(Query query, Iterable inlineIfNecessary(child, null, subqueryRowLimitAccumulator, maxSubqueryRows, dryRun)) + .map(child -> inlineIfNecessary( + child, + null, + subqueryRowLimitAccumulator, + subqueryMemoryLimitAccumulator, + cannotMaterializeToFrames, + maxSubqueryRows, + maxSubqueryMemory, + useNestedForUnknownTypeInSubquery, + dryRun + )) .collect(Collectors.toList()) ); } @@ -560,23 +632,145 @@ private DataSource insertSubqueryIds( * If zero, this method will throw an error immediately. * @throws ResourceLimitExceededException if the limit is exceeded */ - private static > InlineDataSource toInlineDataSource( + private static > DataSource toInlineDataSource( final QueryType query, final Sequence results, final QueryToolChest toolChest, final AtomicInteger limitAccumulator, - final int limit + final AtomicLong memoryLimitAccumulator, + final AtomicBoolean cannotMaterializeToFrames, + final int limit, + long memoryLimit, + boolean useNestedForUnknownTypeInSubquery + ) + { + final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit; + + DataSource dataSource; + + switch (ClientQuerySegmentWalkerUtils.getLimitType(memoryLimit, cannotMaterializeToFrames.get())) { + case ROW_LIMIT: + if (limitAccumulator.get() >= rowLimitToUse) { + throw ResourceLimitExceededException.withMessage( + "Cannot issue the query, subqueries generated results beyond maximum[%d] rows", + rowLimitToUse + ); + } + dataSource = materializeResultsAsArray( + query, + results, + toolChest, + limitAccumulator, + limit + ); + break; + case MEMORY_LIMIT: + if (memoryLimitAccumulator.get() >= memoryLimit) { + throw ResourceLimitExceededException.withMessage( + "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes", + memoryLimit + ); + } + Optional maybeDataSource = materializeResultsAsFrames( + query, + results, + toolChest, + limitAccumulator, + memoryLimitAccumulator, + memoryLimit, + useNestedForUnknownTypeInSubquery + ); + if (!maybeDataSource.isPresent()) { + cannotMaterializeToFrames.set(true); + // Check if the previous row limit accumulator has exceeded the memory results + if (memoryLimitAccumulator.get() >= memoryLimit) { + throw ResourceLimitExceededException.withMessage( + "Cannot issue the query, subqueries generated results beyond maximum[%d] bytes", + memoryLimit + ); + } + dataSource = materializeResultsAsArray( + query, + results, + toolChest, + limitAccumulator, + limit + ); + } else { + dataSource = maybeDataSource.get(); + } + break; + default: + throw new IAE("Only row based and memory based limiting is supported"); + } + return dataSource; + } + + /** + * This method materializes the query results as Frames. The method defaults back to materializing as rows in case + * one cannot materialize the results as frames + */ + private static > Optional materializeResultsAsFrames( + final QueryType query, + final Sequence results, + final QueryToolChest toolChest, + final AtomicInteger limitAccumulator, + final AtomicLong memoryLimitAccumulator, + long memoryLimit, + boolean useNestedForUnknownTypeInSubquery ) { - final int limitToUse = limit < 0 ? Integer.MAX_VALUE : limit; + Optional> framesOptional; - if (limitAccumulator.get() >= limitToUse) { - throw ResourceLimitExceededException.withMessage( - "Cannot issue subquery, maximum[%d] reached", - limitToUse + try { + framesOptional = toolChest.resultsAsFrames( + query, + results, + new ArenaMemoryAllocatorFactory(FRAME_SIZE), + useNestedForUnknownTypeInSubquery ); } + catch (Exception e) { + log.debug(e, "Unable to materialize the results as frames due to an unhandleable exception " + + "while conversion. Defaulting to materializing the results as rows"); + return Optional.empty(); + } + + if (!framesOptional.isPresent()) { + log.debug("Unable to materialize the results as frames. Defaulting to materializing the results as rows"); + return Optional.empty(); + } + Sequence frames = framesOptional.get(); + List frameSignaturePairs = new ArrayList<>(); + frames.forEach( + frame -> { + limitAccumulator.addAndGet(frame.getFrame().numRows()); + if (memoryLimitAccumulator.addAndGet(frame.getFrame().numBytes()) >= memoryLimit) { + throw ResourceLimitExceededException.withMessage( + "Subquery generated results beyond maximum[%d] bytes", + memoryLimit + ); + + } + frameSignaturePairs.add(frame); + } + ); + return Optional.of(new FrameBasedInlineDataSource(frameSignaturePairs, toolChest.resultArraySignature(query))); + } + + /** + * This method materializes the query results as {@code List} + */ + private static > DataSource materializeResultsAsArray( + final QueryType query, + final Sequence results, + final QueryToolChest toolChest, + final AtomicInteger limitAccumulator, + final int limit + ) + { + final int rowLimitToUse = limit < 0 ? Integer.MAX_VALUE : limit; final RowSignature signature = toolChest.resultArraySignature(query); final ArrayList resultList = new ArrayList<>(); @@ -584,17 +778,16 @@ private static > InlineDataSource toInlineDataSour toolChest.resultsAsArrays(query, results).accumulate( resultList, (acc, in) -> { - if (limitAccumulator.getAndIncrement() >= limitToUse) { + if (limitAccumulator.getAndIncrement() >= rowLimitToUse) { throw ResourceLimitExceededException.withMessage( - "Subquery generated results beyond maximum[%d]", - limitToUse + "Subquery generated results beyond maximum[%d] rows", + rowLimitToUse ); } acc.add(in); return acc; } ); - return InlineDataSource.fromIterable(resultList, signature); } diff --git a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java new file mode 100644 index 000000000000..f185a5a53266 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalkerUtils.java @@ -0,0 +1,40 @@ +/* + * 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.server; + +public class ClientQuerySegmentWalkerUtils +{ + public enum SubqueryResultLimit + { + ROW_LIMIT, + MEMORY_LIMIT + } + + public static SubqueryResultLimit getLimitType(long memoryLimitBytes, boolean cannotMaterializeToFrames) + { + if (cannotMaterializeToFrames) { + return SubqueryResultLimit.ROW_LIMIT; + } + if (memoryLimitBytes > 0) { + return SubqueryResultLimit.MEMORY_LIMIT; + } + return SubqueryResultLimit.ROW_LIMIT; + } +} diff --git a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java index d6b628221153..9b72670c3fd5 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ServerConfig.java @@ -43,6 +43,9 @@ public class ServerConfig { public static final int DEFAULT_GZIP_INFLATE_BUFFER_SIZE = 4096; + public static final long DEFAULT_MAX_SUBQUERY_BYTES = -1L; + + private static final boolean DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY = false; /** * The ServerConfig is normally created using {@link org.apache.druid.guice.JsonConfigProvider} binding. @@ -57,6 +60,8 @@ public ServerConfig( long defaultQueryTimeout, long maxScatterGatherBytes, int maxSubqueryRows, + long maxSubqueryBytes, + boolean useNestedForUnknownTypeInSubquery, long maxQueryTimeout, int maxRequestHeaderSize, @NotNull Period gracefulShutdownTimeout, @@ -78,6 +83,8 @@ public ServerConfig( this.defaultQueryTimeout = defaultQueryTimeout; this.maxScatterGatherBytes = HumanReadableBytes.valueOf(maxScatterGatherBytes); this.maxSubqueryRows = maxSubqueryRows; + this.maxSubqueryBytes = maxSubqueryBytes; + this.useNestedForUnknownTypeInSubquery = useNestedForUnknownTypeInSubquery; this.maxQueryTimeout = maxQueryTimeout; this.maxRequestHeaderSize = maxRequestHeaderSize; this.gracefulShutdownTimeout = gracefulShutdownTimeout; @@ -125,6 +132,12 @@ public ServerConfig() @Min(1) private int maxSubqueryRows = 100000; + @JsonProperty + private long maxSubqueryBytes = DEFAULT_MAX_SUBQUERY_BYTES; + + @JsonProperty + private boolean useNestedForUnknownTypeInSubquery = DEFAULT_USE_NESTED_FOR_UNKNOWN_TYPE_IN_SUBQUERY; + @JsonProperty @Min(1) private long maxQueryTimeout = Long.MAX_VALUE; @@ -204,6 +217,16 @@ public int getMaxSubqueryRows() return maxSubqueryRows; } + public long getMaxSubqueryBytes() + { + return maxSubqueryBytes; + } + + public boolean isuseNestedForUnknownTypeInSubquery() + { + return useNestedForUnknownTypeInSubquery; + } + public long getMaxQueryTimeout() { return maxQueryTimeout; @@ -280,6 +303,8 @@ public boolean equals(Object o) enableRequestLimit == that.enableRequestLimit && defaultQueryTimeout == that.defaultQueryTimeout && maxSubqueryRows == that.maxSubqueryRows && + maxSubqueryBytes == that.maxSubqueryBytes && + useNestedForUnknownTypeInSubquery == that.useNestedForUnknownTypeInSubquery && maxQueryTimeout == that.maxQueryTimeout && maxRequestHeaderSize == that.maxRequestHeaderSize && inflateBufferSize == that.inflateBufferSize && @@ -307,6 +332,8 @@ public int hashCode() defaultQueryTimeout, maxScatterGatherBytes, maxSubqueryRows, + maxSubqueryBytes, + useNestedForUnknownTypeInSubquery, maxQueryTimeout, maxRequestHeaderSize, gracefulShutdownTimeout, @@ -333,6 +360,8 @@ public String toString() ", defaultQueryTimeout=" + defaultQueryTimeout + ", maxScatterGatherBytes=" + maxScatterGatherBytes + ", maxSubqueryRows=" + maxSubqueryRows + + ", maxSubqueryBytes=" + maxSubqueryBytes + + ", useNestedForUnknownTypeInSubquery=" + useNestedForUnknownTypeInSubquery + ", maxQueryTimeout=" + maxQueryTimeout + ", maxRequestHeaderSize=" + maxRequestHeaderSize + ", gracefulShutdownTimeout=" + gracefulShutdownTimeout + diff --git a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java index 491ee4fd98b7..687ca2ef5485 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/jetty/CliIndexerServerModule.java @@ -153,6 +153,8 @@ public ServerConfig makeAdjustedServerConfig(ServerConfig oldConfig) oldConfig.getDefaultQueryTimeout(), oldConfig.getMaxScatterGatherBytes(), oldConfig.getMaxSubqueryRows(), + oldConfig.getMaxSubqueryBytes(), + oldConfig.isuseNestedForUnknownTypeInSubquery(), oldConfig.getMaxQueryTimeout(), oldConfig.getMaxRequestHeaderSize(), oldConfig.getGracefulShutdownTimeout(), diff --git a/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java b/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java index f6161e6fa570..96745e163a00 100644 --- a/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java +++ b/server/src/test/java/org/apache/druid/initialization/ServerConfigTest.java @@ -52,6 +52,8 @@ public void testSerde() throws Exception defaultConfig.getDefaultQueryTimeout(), defaultConfig.getMaxScatterGatherBytes(), defaultConfig.getMaxSubqueryRows(), + defaultConfig.getMaxSubqueryBytes(), + defaultConfig.isuseNestedForUnknownTypeInSubquery(), defaultConfig.getMaxQueryTimeout(), defaultConfig.getMaxRequestHeaderSize(), defaultConfig.getGracefulShutdownTimeout(), diff --git a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java index 967d9342d78f..a8aaf10da00b 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -34,6 +34,7 @@ import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; +import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinDataSource; @@ -62,6 +63,7 @@ import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.segment.FrameBasedInlineSegmentWrangler; import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.MapSegmentWrangler; import org.apache.druid.segment.ReferenceCountingSegment; @@ -72,6 +74,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableStringArray; +import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory; import org.apache.druid.segment.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; @@ -348,7 +351,7 @@ public void testTimeseriesOnGroupByOnTable() testQuery( query, - ImmutableList.of( + new ArrayList<>(ImmutableList.of( ExpectedQuery.cluster(subquery.withId(DUMMY_QUERY_ID).withSubQueryId("1.1")), ExpectedQuery.local( query.withDataSource( @@ -358,7 +361,7 @@ public void testTimeseriesOnGroupByOnTable() ) ) ) - ), + )), ImmutableList.of(new Object[]{Intervals.ETERNITY.getStartMillis(), 3L}) ); @@ -798,7 +801,35 @@ public void testTimeseriesOnGroupByOnTableErrorTooManyRows() .withId(DUMMY_QUERY_ID); expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Subquery generated results beyond maximum[2]"); + expectedException.expectMessage("Subquery generated results beyond maximum[2] rows"); + + testQuery(query, ImmutableList.of(), ImmutableList.of()); + } + + + @Test + public void testTimeseriesOnGroupByOnTableErrorTooLarge() + { + final GroupByQuery subquery = + GroupByQuery.builder() + .setDataSource(FOO) + .setGranularity(Granularities.ALL) + .setInterval(Collections.singletonList(INTERVAL)) + .setDimensions(DefaultDimensionSpec.of("s")) + .build(); + + final TimeseriesQuery query = + (TimeseriesQuery) Druids.newTimeseriesQueryBuilder() + .dataSource(new QueryDataSource(subquery)) + .granularity(Granularities.ALL) + .intervals(Intervals.ONLY_ETERNITY) + .aggregators(new CountAggregatorFactory("cnt")) + .context(ImmutableMap.of(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "1")) + .build() + .withId(DUMMY_QUERY_ID); + + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Subquery generated results beyond maximum[1] bytes"); testQuery(query, ImmutableList.of(), ImmutableList.of()); } @@ -1316,8 +1347,9 @@ private void initWalker(final Map serverProperties, QuerySchedul final SegmentWrangler segmentWrangler = new MapSegmentWrangler( ImmutableMap., SegmentWrangler>builder() - .put(InlineDataSource.class, new InlineSegmentWrangler()) - .build() + .put(InlineDataSource.class, new InlineSegmentWrangler()) + .put(FrameBasedInlineDataSource.class, new FrameBasedInlineSegmentWrangler()) + .build() ); final JoinableFactory globalFactory = new JoinableFactory() @@ -1336,11 +1368,12 @@ public Optional build(DataSource dataSource, JoinConditionAnalysis con }; final JoinableFactory joinableFactory = new MapJoinableFactory( - ImmutableSet.of(globalFactory, new InlineJoinableFactory()), + ImmutableSet.of(globalFactory, new InlineJoinableFactory(), new FrameBasedInlineJoinableFactory()), ImmutableMap., Class>builder() - .put(InlineJoinableFactory.class, InlineDataSource.class) - .put(globalFactory.getClass(), GlobalTableDataSource.class) - .build() + .put(InlineJoinableFactory.class, InlineDataSource.class) + .put(FrameBasedInlineJoinableFactory.class, FrameBasedInlineDataSource.class) + .put(globalFactory.getClass(), GlobalTableDataSource.class) + .build() ); final JoinableFactoryWrapper joinableFactoryWrapper = new JoinableFactoryWrapper(joinableFactory); @@ -1450,8 +1483,9 @@ private static class ExpectedQuery ExpectedQuery(Query query, ClusterOrLocal how) { + Query modifiedQuery; // Need to blast various parameters that will vary and aren't important to test for. - this.query = query.withOverriddenContext( + modifiedQuery = query.withOverriddenContext( ImmutableMap.builder() .put(DirectDruidClient.QUERY_FAIL_TIME, 0L) .put(QueryContexts.DEFAULT_TIMEOUT_KEY, 0L) @@ -1467,6 +1501,11 @@ private static class ExpectedQuery .build() ); + if (modifiedQuery.getDataSource() instanceof FrameBasedInlineDataSource) { + // Do this recursively for if the query's datasource is a query datasource + } + + this.query = modifiedQuery; this.how = how; } diff --git a/server/src/test/java/org/apache/druid/server/QueryStackTests.java b/server/src/test/java/org/apache/druid/server/QueryStackTests.java index 7011a0283df2..1c23edf3b9a8 100644 --- a/server/src/test/java/org/apache/druid/server/QueryStackTests.java +++ b/server/src/test/java/org/apache/druid/server/QueryStackTests.java @@ -28,6 +28,7 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.Query; @@ -70,6 +71,7 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.join.FrameBasedInlineJoinableFactory; import org.apache.druid.segment.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.JoinableFactoryWrapper; @@ -372,8 +374,9 @@ public static JoinableFactory makeJoinableFactoryFromDefault( ImmutableSet.Builder setBuilder = ImmutableSet.builder(); ImmutableMap.Builder, Class> mapBuilder = ImmutableMap.builder(); - setBuilder.add(new InlineJoinableFactory()); + setBuilder.add(new InlineJoinableFactory(), new FrameBasedInlineJoinableFactory()); mapBuilder.put(InlineJoinableFactory.class, InlineDataSource.class); + mapBuilder.put(FrameBasedInlineJoinableFactory.class, FrameBasedInlineDataSource.class); if (lookupProvider != null) { setBuilder.add(new LookupJoinableFactory(lookupProvider)); mapBuilder.put(LookupJoinableFactory.class, LookupDataSource.class); 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 b93228e076f3..2ba9ce6dc730 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 @@ -239,6 +239,10 @@ public static void setupNullValues() QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); + public static final Map QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT = + DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000") + .build(); + // Add additional context to the given context map for when the // timeseries query has timestamp_floor expression on the timestamp dimension public static Map getTimeseriesContextWithFloorTime( 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 792c25a5e1d7..fcf2c9dbb2bb 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 @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -42,7 +41,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; -import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; @@ -139,52 +137,6 @@ public class CalciteQueryTest extends BaseCalciteQueryTest { - @Test - public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries() - { - cannotVectorize(); - - testQuery( - "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n" - + "FROM (\n" - + " SELECT\n" - + " FLOOR(__time to hour) as \"date\",\n" - + " COUNT(*) as x\n" - + " FROM foo\n" - + " GROUP BY 1\n" - + ")\n" - + "GROUP BY 1", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.HOUR) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) - .build() - ) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setVirtualColumns( - expressionVirtualColumn( - "v0", - "timestamp_format(\"d0\",'yyyy-MM','UTC')", - ColumnType.STRING - ) - ) - .setGranularity(Granularities.ALL) - .addDimension(new DefaultDimensionSpec("v0", "_d0")) - .addAggregator(new LongSumAggregatorFactory("_a0", "a0")) - .build() - ), - ImmutableList.of( - new Object[]{"2000-01", 3L}, - new Object[]{"2001-01", 3L} - ) - ); - } - @Test public void testInformationSchemaSchemata() { @@ -4048,56 +4000,6 @@ public void testColumnIsNull() ); } - @Test - public void testSelfJoin() - { - // Cannot vectorize due to virtual columns. - cannotVectorize(); - - testQuery( - "SELECT COUNT(*) FROM druid.foo x, druid.foo y\n", - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns( - ImmutableList.of( - "__time", - "cnt", - "dim1", - "dim2", - "dim3", - "m1", - "m2", - "unique_dim1" - ) - ) - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - "1", - JoinType.INNER - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{36L} - ) - ); - } - @Test public void testGroupingWithNullInFilter() { @@ -4131,82 +4033,6 @@ public void testGroupingWithNullInFilter() ); } - @Test - public void testTwoExactCountDistincts() - { - testQuery( - PLANNER_CONFIG_NO_HLL, - "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo", - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of( - newScanQueryBuilder() - .dataSource( - join( - new QueryDataSource( - GroupByQuery - .builder() - .setDataSource( - GroupByQuery - .builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs( - new FilteredAggregatorFactory( - new CountAggregatorFactory("a0"), - not(selector("d0", null, null)) - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - new QueryDataSource( - GroupByQuery - .builder() - .setDataSource( - GroupByQuery - .builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs( - new FilteredAggregatorFactory( - new CountAggregatorFactory("a0"), - not(selector("d0", null, null)) - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - "1", - JoinType.INNER - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("a0", "j0.a0") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L} - ) - ); - } - @Test public void testGroupByNothingWithLiterallyFalseFilter() { @@ -4549,58 +4375,6 @@ public void testConfusedView() ); } - @Test - public void testViewAndJoin() - { - notMsqCompatible(); - cannotVectorize(); - Map queryContext = withLeftDirectAccessEnabled(QUERY_CONTEXT_DEFAULT); - testQuery( - "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ", - queryContext, - ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource( - join( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("dim2") - .context(queryContext) - .build() - ), - "j0.", - "(\"dim2\" == \"j0.dim2\")", - JoinType.INNER, - bound("dim2", "a", "a", false, false, null, null) - ), - new QueryDataSource( - newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("dim2") - .context(queryContext) - .build() - ), - "_j0.", - "('a' == \"_j0.dim2\")", - JoinType.INNER - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))) - .granularity(Granularities.ALL) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(withLeftDirectAccessEnabled(QUERY_CONTEXT_DEFAULT)) - .build() - ), - ImmutableList.of( - new Object[]{8L} - ) - ); - } - @Test public void testCountStarWithLikeFilter() { @@ -7181,157 +6955,6 @@ public void testExactCountDistinctUsingSubqueryOnUnionAllTables() ); } - @Test - public void testUseTimeFloorInsteadOfGranularityOnJoinResult() - { - cannotVectorize(); - - testQuery( - "WITH main AS (SELECT * FROM foo LIMIT 2)\n" - + "SELECT TIME_FLOOR(__time, 'PT1H') AS \"time\", dim1, COUNT(*)\n" - + "FROM main\n" - + "WHERE dim1 IN (SELECT dim1 FROM main GROUP BY 1 ORDER BY COUNT(*) DESC LIMIT 5)\n" - + "GROUP BY 1, 2", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - join( - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.ETERNITY)) - .columns("__time", "dim1") - .limit(2) - .build() - ), - new QueryDataSource( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.ETERNITY)) - .columns("dim1") - .limit(2) - .build() - ) - ) - .setInterval(querySegmentSpec(Intervals.ETERNITY)) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setLimitSpec( - new DefaultLimitSpec( - ImmutableList.of( - new OrderByColumnSpec( - "a0", - Direction.DESCENDING, - StringComparators.NUMERIC - ) - ), - 5 - ) - ) - .build() - ), - "j0.", - "(\"dim1\" == \"j0.d0\")", - JoinType.INNER - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - expressionVirtualColumn( - "v0", - "timestamp_floor(\"__time\",'PT1H',null,'UTC')", - ColumnType.LONG - ) - ) - .setDimensions(dimensions( - new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), - new DefaultDimensionSpec("dim1", "d1") - )) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - NullHandling.sqlCompatible() - ? ImmutableList.of(new Object[]{946684800000L, "", 1L}, new Object[]{946771200000L, "10.1", 1L}) - : ImmutableList.of(new Object[]{946771200000L, "10.1", 1L}) - ); - } - - @Test - public void testMinMaxAvgDailyCountWithLimit() - { - // Cannot vectorize due to virtual columns. - cannotVectorize(); - - testQuery( - "SELECT * FROM (" - + " SELECT max(cnt), min(cnt), avg(cnt), TIME_EXTRACT(max(t), 'EPOCH') last_time, count(1) num_days FROM (\n" - + " SELECT TIME_FLOOR(__time, 'P1D') AS t, count(1) cnt\n" - + " FROM \"foo\"\n" - + " GROUP BY 1\n" - + " )" - + ") LIMIT 1\n", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) - .intervals(querySegmentSpec(Filtration.eternity())) - .aggregators(new CountAggregatorFactory("a0")) - .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) - .build() - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs( - useDefault ? - aggregators( - new LongMaxAggregatorFactory("_a0", "a0"), - new LongMinAggregatorFactory("_a1", "a0"), - new LongSumAggregatorFactory("_a2:sum", "a0"), - new CountAggregatorFactory("_a2:count"), - new LongMaxAggregatorFactory("_a3", "d0"), - new CountAggregatorFactory("_a4") - ) : aggregators( - new LongMaxAggregatorFactory("_a0", "a0"), - new LongMinAggregatorFactory("_a1", "a0"), - new LongSumAggregatorFactory("_a2:sum", "a0"), - new FilteredAggregatorFactory( - new CountAggregatorFactory("_a2:count"), - not(selector("a0", null, null)) - ), - new LongMaxAggregatorFactory("_a3", "d0"), - new CountAggregatorFactory("_a4") - ) - ) - .setPostAggregatorSpecs( - ImmutableList.of( - new ArithmeticPostAggregator( - "_a2", - "quotient", - ImmutableList.of( - new FieldAccessPostAggregator(null, "_a2:sum"), - new FieldAccessPostAggregator(null, "_a2:count") - ) - ), - expressionPostAgg("p0", "timestamp_extract(\"_a3\",'EPOCH','UTC')") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of(new Object[]{1L, 1L, 1L, 978480000L, 6L}) - ); - } - @Test public void testAvgDailyCountDistinct() { @@ -7407,116 +7030,6 @@ public void testAvgDailyCountDistinct() ); } - @Test - public void testExactCountDistinctOfSemiJoinResult() - { - // Cannot vectorize due to extraction dimension spec. - cannotVectorize(); - - testQuery( - "SELECT COUNT(*)\n" - + "FROM (\n" - + " SELECT DISTINCT dim2\n" - + " FROM druid.foo\n" - + " WHERE SUBSTRING(dim2, 1, 1) IN (\n" - + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n" - + " ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n" - + ")", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - GroupByQuery.builder() - .setDataSource( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - GroupByQuery - .builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) - .setDimensions( - dimensions( - new ExtractionDimensionSpec( - "dim1", - "d0", - new SubstringDimExtractionFn(0, 1) - ) - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - equalsCondition( - makeExpression("substring(\"dim2\", 0, 1)"), - DruidExpression.ofColumn(ColumnType.STRING, "j0.d0") - ), - JoinType.INNER - ) - ) - .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01"))) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ) - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{2L} - ) - ); - } - - @Test - public void testMaxSubqueryRows() - { - notMsqCompatible(); - expectedException.expect(ResourceLimitExceededException.class); - expectedException.expectMessage("Subquery generated results beyond maximum[2]"); - - testQuery( - PLANNER_CONFIG_DEFAULT, - ImmutableMap.of(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 2), - "SELECT COUNT(*)\n" - + "FROM druid.foo\n" - + "WHERE SUBSTRING(dim2, 1, 1) IN (\n" - + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n" - + ")\n", - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of() - ); - } - - @Test - public void testZeroMaxNumericInFilter() - { - expectedException.expect(UOE.class); - expectedException.expectMessage("[maxNumericInFilters] must be greater than 0"); - - testQuery( - PLANNER_CONFIG_DEFAULT, - ImmutableMap.of(QueryContexts.MAX_NUMERIC_IN_FILTERS, 0), - "SELECT COUNT(*)\n" - + "FROM druid.numfoo\n" - + "WHERE dim6 IN (\n" - + "1,2,3\n" - + ")\n", - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of() - ); - } - @Test public void testHighestMaxNumericInFilter() { @@ -7641,54 +7154,6 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() ); } - @Test - public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() - { - // Cannot vectorize topN operator. - cannotVectorize(); - - testQuery( - "SELECT\n" - + " SUM(cnt),\n" - + " COUNT(*)\n" - + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n" - + "WHERE cnt > 0", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - new QueryDataSource( - new TopNQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .dimension(new DefaultDimensionSpec("dim2", "d0")) - .aggregators(new LongSumAggregatorFactory("a0", "cnt")) - .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) - .threshold(1) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ) - ) - .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs(aggregators( - new LongSumAggregatorFactory("_a0", "a0"), - new CountAggregatorFactory("_a1") - )) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - NullHandling.replaceWithDefault() ? - ImmutableList.of( - new Object[]{3L, 1L} - ) : - ImmutableList.of( - new Object[]{2L, 1L} - ) - ); - } - @Test public void testCompareExactAndApproximateCountDistinctUsingSubquery() { @@ -11306,148 +10771,6 @@ public void testTimeExtractWithTooFewArguments() } } - @Test - public void testUsingSubqueryAsFilterOnTwoColumns() - { - testQuery( - "SELECT __time, cnt, dim1, dim2 FROM druid.foo " - + " WHERE (dim1, dim2) IN (" - + " SELECT dim1, dim2 FROM (" - + " SELECT dim1, dim2, COUNT(*)" - + " FROM druid.foo" - + " WHERE dim2 = 'abc'" - + " GROUP BY dim1, dim2" - + " HAVING COUNT(*) = 1" - + " )" - + " )", - ImmutableList.of( - newScanQueryBuilder() - .dataSource( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimFilter(selector("dim2", "abc", null)) - .setDimensions(dimensions( - new DefaultDimensionSpec("dim1", "d0"), - new DefaultDimensionSpec("dim2", "d1") - )) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setPostAggregatorSpecs( - ImmutableList.of(expressionPostAgg("p0", "'abc'")) - ) - .setHavingSpec(having(selector("a0", "1", null))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - StringUtils.format( - "(%s && %s)", - equalsCondition(makeColumnExpression("dim1"), makeColumnExpression("j0.d0")), - equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.p0")) - ), - JoinType.INNER - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "'abc'", ColumnType.STRING)) - .columns("__time", "cnt", "dim1", "v0") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - ImmutableList.of( - new Object[]{timestamp("2001-01-02"), 1L, "def", "abc"} - ) - ); - } - - @Test - public void testUsingSubqueryAsFilterWithInnerSort() - { - // Regression test for https://github.com/apache/druid/issues/4208 - - testQuery( - "SELECT dim1, dim2 FROM druid.foo\n" - + " WHERE dim2 IN (\n" - + " SELECT dim2\n" - + " FROM druid.foo\n" - + " GROUP BY dim2\n" - + " ORDER BY dim2 DESC\n" - + " )", - ImmutableList.of( - newScanQueryBuilder() - .dataSource( - join( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - equalsCondition(makeColumnExpression("dim2"), makeColumnExpression("j0.d0")), - JoinType.INNER - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("dim1", "dim2") - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - NullHandling.replaceWithDefault() ? - ImmutableList.of( - new Object[]{"", "a"}, - new Object[]{"1", "a"}, - new Object[]{"def", "abc"} - ) : - ImmutableList.of( - new Object[]{"", "a"}, - new Object[]{"2", ""}, - new Object[]{"1", "a"}, - new Object[]{"def", "abc"} - ) - ); - } - - @Test - public void testUsingSubqueryWithLimit() - { - // Cannot vectorize scan query. - cannotVectorize(); - - testQuery( - "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA", - ImmutableList.of( - GroupByQuery.builder() - .setDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG)) - .columns("v0") - .limit(10) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .build() - ), - ImmutableList.of( - new Object[]{6L} - ) - ); - } - @Test public void testUsingSubqueryWithoutLimit() { @@ -13244,104 +12567,6 @@ public void testCountAndAverageByConstantVirtualColumn() ); } - @Test - public void testEmptyGroupWithOffsetDoesntInfiniteLoop() - { - notMsqCompatible(); - testQuery( - "SELECT r0.c, r1.c\n" - + "FROM (\n" - + " SELECT COUNT(*) AS c\n" - + " FROM \"foo\"\n" - + " GROUP BY ()\n" - + " OFFSET 1\n" - + ") AS r0\n" - + "LEFT JOIN (\n" - + " SELECT COUNT(*) AS c\n" - + " FROM \"foo\"\n" - + " GROUP BY ()\n" - + ") AS r1 ON TRUE LIMIT 10", - ImmutableList.of( - Druids.newScanQueryBuilder() - .dataSource( - join( - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs( - aggregators( - new CountAggregatorFactory("a0") - ) - ) - .setLimitSpec(DefaultLimitSpec.builder().offset(1).limit(10).build()) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ), - new QueryDataSource( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .aggregators(new CountAggregatorFactory("a0")) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0.", - "1", - JoinType.LEFT, - null - ) - ) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("a0", "j0.a0") - .limit(10) - .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .legacy(false) - .build() - ), - ImmutableList.of() - ); - } - - @Test - public void testJoinWithTimeDimension() - { - testQuery( - PLANNER_CONFIG_DEFAULT, - QUERY_CONTEXT_DEFAULT, - "SELECT count(*) FROM druid.foo t1 inner join druid.foo t2 on t1.__time = t2.__time", - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(Druids.newTimeseriesQueryBuilder() - .dataSource(JoinDataSource.create( - new TableDataSource(CalciteTests.DATASOURCE1), - new QueryDataSource( - Druids.newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .columns("__time") - .legacy(false) - .context(QUERY_CONTEXT_DEFAULT) - .build()), - "j0.", - "(\"__time\" == \"j0.__time\")", - JoinType.INNER, - null, - ExprMacroTable.nil(), - CalciteTests.createJoinableFactoryWrapper() - )) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(QUERY_CONTEXT_DEFAULT) - .build()), - ImmutableList.of(new Object[]{6L}) - ); - } - @Test public void testExpressionCounts() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java new file mode 100644 index 000000000000..40c04f358073 --- /dev/null +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -0,0 +1,945 @@ +/* + * 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.sql.calcite; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.granularity.PeriodGranularity; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.Druids; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.FilteredAggregatorFactory; +import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongMinAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; +import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.ExtractionDimensionSpec; +import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; +import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.query.scan.ScanQuery; +import org.apache.druid.query.topn.DimensionTopNMetricSpec; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.join.JoinType; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.util.CalciteTests; +import org.joda.time.DateTimeZone; +import org.joda.time.Period; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +/** + * Calcite tests which involve subqueries and materializing the intermediate results on {@link org.apache.druid.server.ClientQuerySegmentWalker} + * The tests are run with two different codepaths: + * 1. Where the memory limit is not set. The intermediate results are materialized as inline rows + * 2. Where the memory limit is set. The intermediate results are materialized as frames + */ +@RunWith(Parameterized.class) +public class CalciteSubqueryTest extends BaseCalciteQueryTest +{ + + public String testName; + public Map queryContext; + + public CalciteSubqueryTest( + String testName, + Map queryContext + ) + { + this.testName = testName; + this.queryContext = queryContext; + } + + @Parameterized.Parameters(name = "{0}") + public static Iterable constructorFeeder() + { + final List constructors = new ArrayList<>(); + constructors.add( + new Object[]{"without memory limit", QUERY_CONTEXT_DEFAULT} + ); + constructors.add( + new Object[]{"with memory limit", QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT} + ); + return constructors; + } + + @Test + public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() + { + // Cannot vectorize topN operator. + cannotVectorize(); + + testQuery( + "SELECT\n" + + " SUM(cnt),\n" + + " COUNT(*)\n" + + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n" + + "WHERE cnt > 0", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) + .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) + .threshold(1) + .build() + ) + ) + .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new LongSumAggregatorFactory("_a0", "a0"), + new CountAggregatorFactory("_a1") + )) + .setContext(queryContext) + .build() + ), + NullHandling.replaceWithDefault() ? + ImmutableList.of( + new Object[]{3L, 1L} + ) : + ImmutableList.of( + new Object[]{2L, 1L} + ) + ); + } + + @Test + public void testExactCountDistinctOfSemiJoinResult() + { + // Cannot vectorize due to extraction dimension spec. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*)\n" + + "FROM (\n" + + " SELECT DISTINCT dim2\n" + + " FROM druid.foo\n" + + " WHERE SUBSTRING(dim2, 1, 1) IN (\n" + + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 <> ''\n" + + " ) AND __time >= '2000-01-01' AND __time < '2002-01-01'\n" + + ")", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(not(selector("dim1", "", null))) + .setDimensions( + dimensions( + new ExtractionDimensionSpec( + "dim1", + "d0", + new SubstringDimExtractionFn(0, 1) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.ofExpression( + ColumnType.STRING, + null, + args -> "substring(\"dim2\", 0, 1)", + Collections.emptyList() + ), + DruidExpression.ofColumn(ColumnType.STRING, "j0.d0") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01"))) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{2L} + ) + ); + } + + @Ignore("Merge buffers exceed the prescribed limit when the results are materialized as frames") + @Test + public void testTwoExactCountDistincts() + { + testQuery( + PLANNER_CONFIG_NO_HLL, + queryContext, + "SELECT COUNT(distinct dim1), COUNT(distinct dim2) FROM druid.foo", + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + not(selector("d0", null, null)) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + new QueryDataSource( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ColumnType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + new FilteredAggregatorFactory( + new CountAggregatorFactory("a0"), + not(selector("d0", null, null)) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a0", "j0.a0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{NullHandling.sqlCompatible() ? 6L : 5L, NullHandling.sqlCompatible() ? 3L : 2L} + ) + ); + } + + @Test + public void testViewAndJoin() + { + cannotVectorize(); + Map queryContextModified = withLeftDirectAccessEnabled(queryContext); + testQuery( + "SELECT COUNT(*) FROM view.cview as a INNER JOIN druid.foo d on d.dim2 = a.dim2 WHERE a.dim1_firstchar <> 'z' ", + queryContextModified, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource( + join( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE3) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .context(queryContextModified) + .build() + ), + "j0.", + "(\"dim2\" == \"j0.dim2\")", + JoinType.INNER, + bound("dim2", "a", "a", false, false, null, null) + ), + new QueryDataSource( + newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2") + .context(queryContextModified) + .build() + ), + "_j0.", + "('a' == \"_j0.dim2\")", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(queryContextModified) + .build() + ), + ImmutableList.of( + new Object[]{8L} + ) + ); + } + + @Test + public void testGroupByWithPostAggregatorReferencingTimeFloorColumnOnTimeseries() + { + cannotVectorize(); + + testQuery( + "SELECT TIME_FORMAT(\"date\", 'yyyy-MM'), SUM(x)\n" + + "FROM (\n" + + " SELECT\n" + + " FLOOR(__time to hour) as \"date\",\n" + + " COUNT(*) as x\n" + + " FROM foo\n" + + " GROUP BY 1\n" + + ")\n" + + "GROUP BY 1", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.HOUR) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) + .build() + ) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "timestamp_format(\"d0\",'yyyy-MM','UTC')", + ColumnType.STRING + ) + ) + .setGranularity(Granularities.ALL) + .addDimension(new DefaultDimensionSpec("v0", "_d0")) + .addAggregator(new LongSumAggregatorFactory("_a0", "a0")) + .build() + ), + ImmutableList.of( + new Object[]{"2000-01", 3L}, + new Object[]{"2001-01", 3L} + ) + ); + } + + @Test + public void testUsingSubqueryAsFilterWithInnerSort() + { + // Regression test for https://github.com/apache/druid/issues/4208 + + testQuery( + "SELECT dim1, dim2 FROM druid.foo\n" + + " WHERE dim2 IN (\n" + + " SELECT dim2\n" + + " FROM druid.foo\n" + + " GROUP BY dim2\n" + + " ORDER BY dim2 DESC\n" + + " )", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim2"), DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim1", "dim2") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + NullHandling.replaceWithDefault() ? + ImmutableList.of( + new Object[]{"", "a"}, + new Object[]{"1", "a"}, + new Object[]{"def", "abc"} + ) : + ImmutableList.of( + new Object[]{"", "a"}, + new Object[]{"2", ""}, + new Object[]{"1", "a"}, + new Object[]{"def", "abc"} + ) + ); + } + + @Test + public void testUsingSubqueryAsFilterOnTwoColumns() + { + testQuery( + "SELECT __time, cnt, dim1, dim2 FROM druid.foo " + + " WHERE (dim1, dim2) IN (" + + " SELECT dim1, dim2 FROM (" + + " SELECT dim1, dim2, COUNT(*)" + + " FROM druid.foo" + + " WHERE dim2 = 'abc'" + + " GROUP BY dim1, dim2" + + " HAVING COUNT(*) = 1" + + " )" + + " )", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter(selector("dim2", "abc", null)) + .setDimensions(dimensions( + new DefaultDimensionSpec("dim1", "d0"), + new DefaultDimensionSpec("dim2", "d1") + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setPostAggregatorSpecs( + ImmutableList.of(expressionPostAgg("p0", "'abc'")) + ) + .setHavingSpec(having(selector("a0", "1", null))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + StringUtils.format( + "(%s && %s)", + equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim1"), DruidExpression.ofColumn(ColumnType.STRING, "j0.d0")), + equalsCondition(DruidExpression.ofColumn(ColumnType.STRING, "dim2"), DruidExpression.ofColumn(ColumnType.STRING, "j0.p0")) + ), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "'abc'", ColumnType.STRING)) + .columns("__time", "cnt", "dim1", "v0") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{timestamp("2001-01-02"), 1L, "def", "abc"} + ) + ); + } + + @Test + public void testMinMaxAvgDailyCountWithLimit() + { + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + testQuery( + "SELECT * FROM (" + + " SELECT max(cnt), min(cnt), avg(cnt), TIME_EXTRACT(max(t), 'EPOCH') last_time, count(1) num_days FROM (\n" + + " SELECT TIME_FLOOR(__time, 'P1D') AS t, count(1) cnt\n" + + " FROM \"foo\"\n" + + " GROUP BY 1\n" + + " )" + + ") LIMIT 1\n", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) + .intervals(querySegmentSpec(Filtration.eternity())) + .aggregators(new CountAggregatorFactory("a0")) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) + .build() + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + useDefault ? + aggregators( + new LongMaxAggregatorFactory("_a0", "a0"), + new LongMinAggregatorFactory("_a1", "a0"), + new LongSumAggregatorFactory("_a2:sum", "a0"), + new CountAggregatorFactory("_a2:count"), + new LongMaxAggregatorFactory("_a3", "d0"), + new CountAggregatorFactory("_a4") + ) : aggregators( + new LongMaxAggregatorFactory("_a0", "a0"), + new LongMinAggregatorFactory("_a1", "a0"), + new LongSumAggregatorFactory("_a2:sum", "a0"), + new FilteredAggregatorFactory( + new CountAggregatorFactory("_a2:count"), + not(selector("a0", null, null)) + ), + new LongMaxAggregatorFactory("_a3", "d0"), + new CountAggregatorFactory("_a4") + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "_a2", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "_a2:sum"), + new FieldAccessPostAggregator(null, "_a2:count") + ) + ), + expressionPostAgg("p0", "timestamp_extract(\"_a3\",'EPOCH','UTC')") + ) + ) + .setContext(queryContext) + .build() + ), + ImmutableList.of(new Object[]{1L, 1L, 1L, 978480000L, 6L}) + ); + } + + @Test + public void testEmptyGroupWithOffsetDoesntInfiniteLoop() + { + testQuery( + "SELECT r0.c, r1.c\n" + + "FROM (\n" + + " SELECT COUNT(*) AS c\n" + + " FROM \"foo\"\n" + + " GROUP BY ()\n" + + " OFFSET 1\n" + + ") AS r0\n" + + "LEFT JOIN (\n" + + " SELECT COUNT(*) AS c\n" + + " FROM \"foo\"\n" + + " GROUP BY ()\n" + + ") AS r1 ON TRUE LIMIT 10", + queryContext, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs( + aggregators( + new CountAggregatorFactory("a0") + ) + ) + .setLimitSpec(DefaultLimitSpec.builder().offset(1).limit(10).build()) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(new CountAggregatorFactory("a0")) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + "1", + JoinType.LEFT, + null + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("a0", "j0.a0") + .limit(10) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .legacy(false) + .build() + ), + ImmutableList.of() + ); + } + + + @Test + public void testMaxSubqueryRows() + { + if ("without memory limit".equals(testName)) { + expectedException.expect(ResourceLimitExceededException.class); + expectedException.expectMessage("Subquery generated results beyond maximum[1]"); + Map modifiedQueryContext = new HashMap<>(queryContext); + modifiedQueryContext.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 1); + + testQuery( + "SELECT\n" + + " SUM(cnt),\n" + + " COUNT(*)\n" + + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 2) \n" + + "WHERE cnt > 0", + modifiedQueryContext, + ImmutableList.of(), + ImmutableList.of() + ); + } else { + // Since the results are materializable as frames, we are able to use the memory limit and donot rely on the + // row limit for the subquery + Map modifiedQueryContext = new HashMap<>(queryContext); + modifiedQueryContext.put(QueryContexts.MAX_SUBQUERY_ROWS_KEY, 1); + + testQuery( + "SELECT\n" + + " SUM(cnt),\n" + + " COUNT(*)\n" + + "FROM (SELECT dim2, SUM(cnt) AS cnt FROM druid.foo GROUP BY dim2 LIMIT 1)\n" + + "WHERE cnt > 0", + modifiedQueryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) + .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) + .threshold(1) + .build() + ) + ) + .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new LongSumAggregatorFactory("_a0", "a0"), + new CountAggregatorFactory("_a1") + )) + .setContext(queryContext) + .build() + ), + NullHandling.replaceWithDefault() ? + ImmutableList.of( + new Object[]{3L, 1L} + ) : + ImmutableList.of( + new Object[]{2L, 1L} + ) + ); + } + } + + @Test + public void testZeroMaxNumericInFilter() + { + expectedException.expect(UOE.class); + expectedException.expectMessage("[maxNumericInFilters] must be greater than 0"); + + Map modifiedQueryContext = new HashMap<>(queryContext); + modifiedQueryContext.put(QueryContexts.MAX_NUMERIC_IN_FILTERS, 0); + + + testQuery( + PLANNER_CONFIG_DEFAULT, + modifiedQueryContext, + "SELECT COUNT(*)\n" + + "FROM druid.numfoo\n" + + "WHERE dim6 IN (\n" + + "1,2,3\n" + + ")\n", + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of(), + ImmutableList.of() + ); + } + + @Test + public void testUseTimeFloorInsteadOfGranularityOnJoinResult() + { + cannotVectorize(); + + testQuery( + "WITH main AS (SELECT * FROM foo LIMIT 2)\n" + + "SELECT TIME_FLOOR(__time, 'PT1H') AS \"time\", dim1, COUNT(*)\n" + + "FROM main\n" + + "WHERE dim1 IN (SELECT dim1 FROM main GROUP BY 1 ORDER BY COUNT(*) DESC LIMIT 5)\n" + + "GROUP BY 1, 2", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("__time", "dim1") + .limit(2) + .build() + ), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.ETERNITY)) + .columns("dim1") + .limit(2) + .build() + ) + ) + .setInterval(querySegmentSpec(Intervals.ETERNITY)) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setLimitSpec( + new DefaultLimitSpec( + ImmutableList.of( + new OrderByColumnSpec( + "a0", + OrderByColumnSpec.Direction.DESCENDING, + StringComparators.NUMERIC + ) + ), + 5 + ) + ) + .build() + ), + "j0.", + "(\"dim1\" == \"j0.d0\")", + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "timestamp_floor(\"__time\",'PT1H',null,'UTC')", + ColumnType.LONG + ) + ) + .setDimensions(dimensions( + new DefaultDimensionSpec("v0", "d0", ColumnType.LONG), + new DefaultDimensionSpec("dim1", "d1") + )) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + NullHandling.sqlCompatible() + ? ImmutableList.of(new Object[]{946684800000L, "", 1L}, new Object[]{946771200000L, "10.1", 1L}) + : ImmutableList.of(new Object[]{946771200000L, "10.1", 1L}) + ); + } + + @Test + public void testJoinWithTimeDimension() + { + testQuery( + PLANNER_CONFIG_DEFAULT, + queryContext, + "SELECT count(*) FROM druid.foo t1 inner join druid.foo t2 on t1.__time = t2.__time", + CalciteTests.REGULAR_USER_AUTH_RESULT, + ImmutableList.of(Druids.newTimeseriesQueryBuilder() + .dataSource(JoinDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + Druids.newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .columns("__time") + .legacy(false) + .context(queryContext) + .build()), + "j0.", + "(\"__time\" == \"j0.__time\")", + JoinType.INNER, + null, + ExprMacroTable.nil(), + CalciteTests.createJoinableFactoryWrapper() + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(queryContext) + .build()), + ImmutableList.of(new Object[]{6L}) + ); + } + + @Test + public void testUsingSubqueryWithLimit() + { + // Cannot vectorize scan query. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "0", ColumnType.LONG)) + .columns("v0") + .limit(10) + .context(queryContext) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{6L} + ) + ); + } + + @Test + public void testSelfJoin() + { + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*) FROM druid.foo x, druid.foo y\n", + queryContext, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns( + ImmutableList.of( + "__time", + "cnt", + "dim1", + "dim2", + "dim3", + "m1", + "m2", + "unique_dim1" + ) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + "1", + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{36L} + ) + ); + } +} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java index dfd1acad0cf7..0c667325b4d1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DecoupledPlanningCalciteQueryTest.java @@ -164,27 +164,6 @@ public void testUnionAllSameTableThreeTimesWithSameMapping() } - @Override - @Ignore - public void testSelfJoin() - { - - } - - @Override - @Ignore - public void testTwoExactCountDistincts() - { - - } - - @Override - @Ignore - public void testViewAndJoin() - { - - } - @Override @Ignore public void testGroupByWithSortOnPostAggregationDefault() @@ -234,34 +213,6 @@ public void testExactCountDistinctUsingSubqueryOnUnionAllTables() } - @Override - @Ignore - public void testUseTimeFloorInsteadOfGranularityOnJoinResult() - { - - } - - @Override - @Ignore - public void testMinMaxAvgDailyCountWithLimit() - { - - } - - @Override - @Ignore - public void testExactCountDistinctOfSemiJoinResult() - { - - } - - @Override - @Ignore - public void testMaxSubqueryRows() - { - - } - @Override @Ignore public void testExactCountDistinctUsingSubqueryWithWherePushDown() @@ -276,27 +227,6 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() } - @Override - @Ignore - public void testUsingSubqueryAsFilterOnTwoColumns() - { - - } - - @Override - @Ignore - public void testUsingSubqueryAsFilterWithInnerSort() - { - - } - - @Override - @Ignore - public void testUsingSubqueryWithLimit() - { - - } - @Override @Ignore public void testPostAggWithTimeseries() @@ -324,20 +254,6 @@ public void testRequireTimeConditionSemiJoinNegative() } - @Override - @Ignore - public void testEmptyGroupWithOffsetDoesntInfiniteLoop() - { - - } - - @Override - @Ignore - public void testJoinWithTimeDimension() - { - - } - @Override @Ignore public void testSubqueryTypeMismatchWithLiterals() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 6bc8e6ca7287..b43a65159567 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -23,6 +23,7 @@ import com.google.common.collect.Ordering; import com.google.common.io.Closeables; import org.apache.druid.query.DataSource; +import org.apache.druid.query.FrameBasedInlineDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.Query; @@ -32,6 +33,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.lookup.LookupExtractorFactoryContainer; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; +import org.apache.druid.segment.FrameBasedInlineSegmentWrangler; import org.apache.druid.segment.InlineSegmentWrangler; import org.apache.druid.segment.LookupSegmentWrangler; import org.apache.druid.segment.MapSegmentWrangler; @@ -129,6 +131,7 @@ public SpecificSegmentsQuerySegmentWalker(final QueryRunnerFactoryConglomerate c new MapSegmentWrangler( ImmutableMap., SegmentWrangler>builder() .put(InlineDataSource.class, new InlineSegmentWrangler()) + .put(FrameBasedInlineDataSource.class, new FrameBasedInlineSegmentWrangler()) .put( LookupDataSource.class, new LookupSegmentWrangler(LOOKUP_EXTRACTOR_FACTORY_CONTAINER_PROVIDER)