diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 53431b94fb19..43a7280c3028 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -52,7 +52,7 @@ jobs: stale-issue-label: stale stale-pr-label: stale exempt-issue-labels: 'Evergreen,Security,Bug,Proposal,Design Review,Improvement,Performance,Refactoring,Apache,Area - Automation/Static Analysis,Area - Batch Indexing,Area - Cache,Area - Deep Storage,Area - Dependencies,Area - Dependency Injection,Area - Dev,Area - Documentation,Area - Extension,Area - Kafka/Kinesis Indexing,Area - Lookups,Area - Metadata,Area - Metrics/Event Emitting,Area - Null Handling,Area - Operations,Area - Query UI,Area - Querying,Area - Router,Area - Segment Balancing/Coordination,Area - Segment Format and Ser/De,Area - SQL,Area - Testing,Area - Web Console,Area - Zookeeper/Curator,Compatibility,Contributions Welcome,Development Blocker,Ease of Use,Error handling,HTTP,Incompatible,Stable API' - exempt-pr-labels: 'Evergreen' + exempt-pr-labels: 'Evergreen,Area - Dependencies' exempt-milestones: true exempt-assignees: true ascending: true diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java new file mode 100644 index 000000000000..c3ebbe5122f9 --- /dev/null +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByDeserializationBenchmark.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.benchmark; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.guice.NestedDataModule; +import org.apache.druid.jackson.AggregatorsModule; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.aggregation.SerializablePairLongString; +import org.apache.druid.query.aggregation.post.ConstantPostAggregator; +import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.groupby.GroupByQuery; +import org.apache.druid.query.groupby.GroupByQueryConfig; +import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; +import org.apache.druid.query.groupby.ResultRow; +import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnType; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.TimeUnit; + +@State(Scope.Benchmark) +@Fork(value = 1) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +public class GroupByDeserializationBenchmark +{ + + static { + NullHandling.initializeForTests(); + NestedDataModule.registerHandlersAndSerde(); + AggregatorsModule.registerComplexMetricsAndSerde(); + } + + @Param({"100", "1000"}) + private int numDimensions; + + @Param({"0", "0.25", "0.5", "0.75", "0.85", "0.95", "0.99", "1.0"}) + private double primitiveToComplexDimensionRatio; + + @Param({"json", "serializablePairLongString"}) + private String complexDimensionType; + + @Param({"true", "false"}) + private boolean backwardCompatibility; + + private GroupByQuery sqlQuery; + private String serializedRow; + private GroupByQueryQueryToolChest groupByQueryQueryToolChest; + private ObjectMapper decoratedMapper; + + @Setup(Level.Trial) + public void setup() throws JsonProcessingException + { + final ObjectMapper undecoratedMapper = TestHelper.makeJsonMapper(); + undecoratedMapper.registerModules(NestedDataModule.getJacksonModulesList()); + undecoratedMapper.registerModule(new AggregatorsModule()); + final Pair sqlQueryAndResultRow = sqlQueryAndResultRow( + numDimensions, + primitiveToComplexDimensionRatio, + complexDimensionType, + undecoratedMapper + ); + sqlQuery = sqlQueryAndResultRow.lhs; + serializedRow = sqlQueryAndResultRow.rhs; + + groupByQueryQueryToolChest = new GroupByQueryQueryToolChest( + null, + () -> new GroupByQueryConfig() + { + @Override + public boolean isIntermediateResultAsMapCompat() + { + return backwardCompatibility; + } + }, + null, + null + ); + + decoratedMapper = groupByQueryQueryToolChest.decorateObjectMapper(undecoratedMapper, sqlQuery); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void deserializeResultRows(Blackhole blackhole) throws JsonProcessingException + { + blackhole.consume(decoratedMapper.readValue(serializedRow, ResultRow.class)); + } + + private static Pair sqlQueryAndResultRow( + final int numDimensions, + final double primitiveToComplexDimensionRatio, + final String complexDimensionType, + final ObjectMapper mapper + ) throws JsonProcessingException + { + final int numPrimitiveDimensions = (int) Math.floor(primitiveToComplexDimensionRatio * numDimensions); + final int numComplexDimensions = numDimensions - numPrimitiveDimensions; + + final List dimensions = new ArrayList<>(); + final List rowList = new ArrayList<>(); + + // Add timestamp + rowList.add(DateTimes.of("2000").getMillis()); + + for (int i = 0; i < numPrimitiveDimensions; ++i) { + dimensions.add( + new DefaultDimensionSpec( + StringUtils.format("primitive%d", i), + StringUtils.format("primitive%d", i), + ColumnType.STRING + ) + ); + rowList.add("foo"); + } + + for (int i = 0; i < numComplexDimensions; ++i) { + dimensions.add( + new DefaultDimensionSpec( + StringUtils.format("complex%d", i), + StringUtils.format("complex%d", i), + ColumnType.ofComplex(complexDimensionType) + ) + ); + + // Serialized version of this object is a valid value for both json and long-string pair dimensions + rowList.add(new SerializablePairLongString(1L, "test")); + } + + // Add aggregator + rowList.add(100); + + // Add post aggregator + rowList.add(10.0); + + GroupByQuery query = GroupByQuery.builder() + .setDataSource("foo") + .setQuerySegmentSpec(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .setDimensions(dimensions) + .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) + .setPostAggregatorSpecs(Collections.singletonList(new ConstantPostAggregator( + "post", + 10 + ))) + .setContext(ImmutableMap.of(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, true)) + .setGranularity(Granularities.DAY) + .build(); + + return Pair.of(query, mapper.writeValueAsString(rowList)); + } +} diff --git a/docs/querying/arrays.md b/docs/querying/arrays.md index a7eebaa32afe..d9a9f3911880 100644 --- a/docs/querying/arrays.md +++ b/docs/querying/arrays.md @@ -71,46 +71,10 @@ The following shows an example `dimensionsSpec` for native ingestion of the data ### SQL-based ingestion -#### `arrayIngestMode` - -Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md) when you include the query context -parameter `arrayIngestMode: array`. - -When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new -tables. - -When `arrayIngestMode` is `mvd`, SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv). -This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type -as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This -is the default behavior when `arrayIngestMode` is not provided in your query context, although the default behavior -may change to `array` in a future release. - -When `arrayIngestMode` is `none`, Druid throws an exception when trying to store any type of arrays. This mode is most -useful when set in the system default query context with `druid.query.default.context.arrayIngestMode = none`, in cases -where the cluster administrator wants SQL query authors to explicitly provide one or the other in their query context. - -The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and -`arrayIngestMode: mvd`. - -| SQL type | Stored type when `arrayIngestMode: array` | Stored type when `arrayIngestMode: mvd` (default) | -|---|---|---| -|`VARCHAR ARRAY`|`ARRAY`|[multi-value `STRING`](multi-value-dimensions.md)| -|`BIGINT ARRAY`|`ARRAY`|not possible (validation error)| -|`DOUBLE ARRAY`|`ARRAY`|not possible (validation error)| - -In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as -[multi-value strings](multi-value-dimensions.md). - -When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead -to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails -validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either -a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally -mixing arrays and multi-value strings in the same column. +Arrays can be inserted with [SQL-based ingestion](../multi-stage-query/index.md). #### Examples -Set [`arrayIngestMode: array`](#arrayingestmode) in your query context to run the following examples. - ```sql REPLACE INTO "array_example" OVERWRITE ALL WITH "ext" AS ( @@ -169,6 +133,35 @@ GROUP BY 1,2,3,4,5 PARTITIONED BY DAY ``` +#### `arrayIngestMode` + +For seamless backwards compatible behavior with Druid versions older than 31, there is an `arrayIngestMode` query context flag. + +When `arrayIngestMode` is `array`, SQL ARRAY types are stored using Druid array columns. This is recommended for new +tables and the default configuration for Druid 31 and newer. + +When `arrayIngestMode` is `mvd` (legacy), SQL `VARCHAR ARRAY` are implicitly wrapped in [`ARRAY_TO_MV`](sql-functions.md#array_to_mv). +This causes them to be stored as [multi-value strings](multi-value-dimensions.md), using the same `STRING` column type +as regular scalar strings. SQL `BIGINT ARRAY` and `DOUBLE ARRAY` cannot be loaded under `arrayIngestMode: mvd`. This +mode is not recommended and will be removed in a future release, but provided for backwards compatibility. + +The following table summarizes the differences in SQL ARRAY handling between `arrayIngestMode: array` and +`arrayIngestMode: mvd`. + +| SQL type | Stored type when `arrayIngestMode: array` (default) | Stored type when `arrayIngestMode: mvd` | +|---|---|---| +|`VARCHAR ARRAY`|`ARRAY`|[multi-value `STRING`](multi-value-dimensions.md)| +|`BIGINT ARRAY`|`ARRAY`|not possible (validation error)| +|`DOUBLE ARRAY`|`ARRAY`|not possible (validation error)| + +In either mode, you can explicitly wrap string arrays in `ARRAY_TO_MV` to cause them to be stored as +[multi-value strings](multi-value-dimensions.md). + +When validating a SQL INSERT or REPLACE statement that contains arrays, Druid checks whether the statement would lead +to mixing string arrays and multi-value strings in the same column. If this condition is detected, the statement fails +validation unless the column is named under the `skipTypeVerification` context parameter. This parameter can be either +a comma-separated list of column names, or a JSON array in string form. This validation is done to prevent accidentally +mixing arrays and multi-value strings in the same column. ## Querying arrays @@ -284,9 +277,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter `"arrayIngestMode": "array"`. Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers. -To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any `arrayIngestMode`. Multi-value dimensions can only contain strings. +To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions. Multi-value dimensions can only contain strings. You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like: diff --git a/docs/querying/multi-value-dimensions.md b/docs/querying/multi-value-dimensions.md index 1ce3a618dac7..e41e8630ebb1 100644 --- a/docs/querying/multi-value-dimensions.md +++ b/docs/querying/multi-value-dimensions.md @@ -507,9 +507,9 @@ Avoid confusing string arrays with [multi-value dimensions](multi-value-dimensio Use care during ingestion to ensure you get the type you want. -To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays and set the context parameter [`"arrayIngestMode": "array"`](arrays.md#arrayingestmode). Arrays may contain strings or numbers. +To get arrays when performing an ingestion using JSON ingestion specs, such as [native batch](../ingestion/native-batch.md) or streaming ingestion such as with [Apache Kafka](../ingestion/kafka-ingestion.md), use dimension type `auto` or enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), write a query that generates arrays. Arrays may contain strings or numbers. -To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion), which ensures you get multi-value dimensions in any [`arrayIngestMode`](arrays.md#arrayingestmode). Multi-value dimensions can only contain strings. +To get multi-value dimensions when performing an ingestion using JSON ingestion specs, use dimension type `string` and do not enable `useSchemaDiscovery`. When performing a [SQL-based ingestion](../multi-stage-query/index.md), wrap arrays in [`ARRAY_TO_MV`](multi-value-dimensions.md#sql-based-ingestion). Multi-value dimensions can only contain strings. You can tell which type you have by checking the `INFORMATION_SCHEMA.COLUMNS` table, using a query like: diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index f5219f6a4b27..b04664ac90e3 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -1069,11 +1069,30 @@ Returns the following: ## LOG10 -`LOG10(expr)` +Calculates the base-10 logarithm of the numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `LOG10()` +* **Function type:** Scalar, numeric + +
Example -Calculates the base-10 of the numeric expression. +The following example applies the LOG10 function to the `max_temperature` column from the `taxi-trips` datasource. + +```sql +SELECT + "max_temperature" AS "max_temperature", + LOG10("max_temperature") AS "log10_max_temp" +FROM "taxi-trips" +LIMIT 1 +``` +Returns the following: + +| `max_temperature` | `log10_max_temp` | +| -- | -- | +| `76` | `1.8808135922807914` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## LOOKUP @@ -1133,11 +1152,26 @@ Returns the minimum value of a set of values. ## MOD -`MOD(x, y)` +Calculates x modulo y, or the remainder of x divided by y. Where x and y are numeric expressions. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `MOD(x, y)` +* **Function type:** Scalar, numeric + +
Example + +The following calculates 78 MOD 10. + +```sql +SELECT MOD(78, 10) as "modulo" +``` +Returns the following: + +| `modulo` | +| -- | +| `8` | +
-Calculates x modulo y, or the remainder of x divided by y. +[Learn more](sql-scalar.md#numeric-functions) ## MV_APPEND @@ -1317,11 +1351,26 @@ Returns the one-based index position of a substring within an expression, option ## POWER -`POWER(expr, power)` +Calculates a numerical expression raised to the specified power. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `POWER(base, exponent)` +* **Function type:** Scalar, numeric -Calculates a numerical expression raised to the specified power. +
Example + +The following example raises 5 to the power of 2. + +```sql +SELECT POWER(5, 2) AS "power" +``` +Returns the following: + +| `power` | +| -- | +| `25` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## RADIANS @@ -1398,11 +1447,31 @@ Returns the rightmost number of characters from an expression. ## ROUND -`ROUND(expr[, digits])` +Calculates the rounded value for a numerical expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `ROUND(expr[, digits])` +* **Function type:** Scalar, numeric -Calculates the rounded value for a numerical expression. +
Example + +The following applies the ROUND function to 0 decimal points on the `pickup_longitude` column from the `taxi-trips` datasource. + +```sql +SELECT + "pickup_longitude" AS "pickup_longitude", + ROUND("pickup_longitude", 0) as "rounded_pickup_longitude" +FROM "taxi-trips" +WHERE "pickup_longitude" IS NOT NULL +LIMIT 1 +``` +Returns the following: + +| `pickup_longitude` | `rounded_pickup_longitude` | +| -- | -- | +| `-73.9377670288086` | `-74` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## ROW_NUMBER @@ -1446,11 +1515,26 @@ Calculates the trigonometric sine of an angle expressed in radians. ## SQRT -`SQRT(expr)` +Calculates the square root of a numeric expression. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `SQRT()` +* **Function type:** Scalar, numeric -Calculates the square root of a numeric expression. +
Example + +The following example calculates the square root of 25. + +```sql +SELECT SQRT(25) AS "square_root" +``` +Returns the following: + +| `square_root` | +| -- | +| `5` | +
+ +[Learn more](sql-scalar.md#numeric-functions) ## STDDEV @@ -1720,20 +1804,41 @@ Trims the leading or trailing characters of an expression. ## TRUNC -`TRUNC(expr[, digits])` +Alias for [`TRUNCATE`](#truncate). -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `TRUNC(expr[, digits])` +* **Function type:** Scalar, numeric -Alias for [`TRUNCATE`](#truncate). +[Learn more](sql-scalar.md#numeric-functions) ## TRUNCATE -`TRUNCATE(expr[, digits])` +Truncates a numerical expression to a specific number of decimal digits. -**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions) +* **Syntax:** `TRUNCATE(expr[, digits])` +* **Function type:** Scalar, numeric -Truncates a numerical expression to a specific number of decimal digits. +
Example +The following applies the TRUNCATE function to 1 decimal place on the `pickup_longitude` column from the `taxi-trips` datasource. + +```sql +SELECT + "pickup_longitude" as "pickup_longitude", + TRUNCATE("pickup_longitude", 1) as "truncate_pickup_longitude" +FROM "taxi-trips" +WHERE "pickup_longitude" IS NOT NULL +LIMIT 1 +``` +Returns the following: + +| `pickup_longitude` | `truncate_pickup_longitude` | +| -- | -- | +| `-73.9377670288086` | `-73.9` | +
+ + +[Learn more](sql-scalar.md#numeric-functions) ## TRY_PARSE_JSON @@ -1783,3 +1888,4 @@ Calculates the sample variance of a set of values. Alias for [`VAR_SAMP`](#var_samp). + diff --git a/docs/querying/sql-window-functions.md b/docs/querying/sql-window-functions.md index d64538779f07..7c2c3aef53e9 100644 --- a/docs/querying/sql-window-functions.md +++ b/docs/querying/sql-window-functions.md @@ -246,11 +246,8 @@ Druid has guardrail logic to prevent you from executing window function queries For example: - You cannot set expressions as bounds for window frames. -- You cannot use two FOLLOWING expressions in the window frame. For example: `ROWS BETWEEN 2 ROWS FOLLOWING and 3 ROWS FOLLOWING`. - You can only use a RANGE frames when both endpoints are unbounded or current row. -If you write a query that violates one of these conditions, Druid throws an error: "The query contains a window frame which may return incorrect results. To disregard this warning, set `windowingStrictValidation` to false in the query context." - ## Window function reference |Function|Notes| diff --git a/extensions-contrib/spectator-histogram/pom.xml b/extensions-contrib/spectator-histogram/pom.xml index 476e562bf4fa..fd6fff411aad 100644 --- a/extensions-contrib/spectator-histogram/pom.xml +++ b/extensions-contrib/spectator-histogram/pom.xml @@ -92,6 +92,11 @@ error_prone_annotations provided + + joda-time + joda-time + provided + org.apache.druid druid-sql @@ -137,5 +142,10 @@ test-jar test + + org.easymock + easymock + test + diff --git a/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java b/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java index 2e54fcf0d45e..8965b595e59f 100644 --- a/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java +++ b/extensions-contrib/spectator-histogram/src/main/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumn.java @@ -19,18 +19,12 @@ package org.apache.druid.spectator.histogram; -import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ComplexColumn; -import org.apache.druid.segment.data.ReadableOffset; - -import javax.annotation.Nullable; public class SpectatorHistogramIndexBasedComplexColumn implements ComplexColumn { private final SpectatorHistogramIndexed index; private final String typeName; - private static final Number ZERO = 0; public SpectatorHistogramIndexBasedComplexColumn(String typeName, SpectatorHistogramIndexed index) { @@ -59,72 +53,11 @@ public Object getRowValue(int rowNum) @Override public int getLength() { - return index.size(); + return -1; } @Override public void close() { } - - @Override - public ColumnValueSelector makeColumnValueSelector(ReadableOffset offset) - { - // Use ColumnValueSelector directly so that we support being queried as a Number using - // longSum or doubleSum aggregators, the NullableNumericBufferAggregator will call isNull. - // This allows us to behave as a Number or SpectatorHistogram object. - // When queried as a Number, we're returning the count of entries in the histogram. - // As such, we can safely return 0 where the histogram is null. - return new ColumnValueSelector() - { - @Override - public boolean isNull() - { - return getObject() == null; - } - - private Number getOrZero() - { - SpectatorHistogram histogram = getObject(); - return histogram != null ? histogram : ZERO; - } - - @Override - public long getLong() - { - return getOrZero().longValue(); - } - - @Override - public float getFloat() - { - return getOrZero().floatValue(); - } - - @Override - public double getDouble() - { - return getOrZero().doubleValue(); - } - - @Nullable - @Override - public SpectatorHistogram getObject() - { - return (SpectatorHistogram) getRowValue(offset.getOffset()); - } - - @Override - public Class classOfObject() - { - return getClazz(); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("column", SpectatorHistogramIndexBasedComplexColumn.this); - } - }; - } } diff --git a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java index 1c30cfc05c36..88e710645d1e 100644 --- a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java +++ b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramAggregatorTest.java @@ -20,8 +20,14 @@ package org.apache.druid.spectator.histogram; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.netflix.spectator.api.histogram.PercentileBuckets; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.impl.NoopInputRowParser; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Druids; @@ -32,6 +38,9 @@ import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.AggregatorUtil; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryRunnerTest; import org.apache.druid.query.groupby.ResultRow; @@ -42,13 +51,17 @@ import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.IncrementalIndexSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.Segment; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -59,6 +72,7 @@ import java.io.File; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -716,6 +730,59 @@ public void testPercentilePostAggregator() throws Exception } } + @Test + public void testBuildingAndCountingHistogramsIncrementalIndex() throws Exception + { + List dimensions = Collections.singletonList("d"); + int n = 10; + DateTime startOfDay = DateTimes.of("2000-01-01"); + List inputRows = new ArrayList<>(n); + for (int i = 1; i <= n; i++) { + String val = String.valueOf(i * 1.0d); + + inputRows.add(new MapBasedInputRow( + startOfDay.plusMinutes(i), + dimensions, + ImmutableMap.of("x", i, "d", val) + )); + } + + IncrementalIndex index = AggregationTestHelper.createIncrementalIndex( + inputRows.iterator(), + new NoopInputRowParser(null), + new AggregatorFactory[]{ + new CountAggregatorFactory("count"), + new SpectatorHistogramAggregatorFactory("histogram", "x") + }, + 0, + Granularities.NONE, + 100, + false + ); + + ImmutableList segments = ImmutableList.of( + new IncrementalIndexSegment(index, SegmentId.dummy("test")), + helper.persistIncrementalIndex(index, null) + ); + + GroupByQuery query = new GroupByQuery.Builder() + .setDataSource("test") + .setGranularity(Granularities.HOUR) + .setInterval("1970/2050") + .setAggregatorSpecs( + new DoubleSumAggregatorFactory("doubleSum", "histogram") + ).build(); + + Sequence seq = helper.runQueryOnSegmentsObjs(segments, query); + + List results = seq.toList(); + Assert.assertEquals(1, results.size()); + // Check timestamp + Assert.assertEquals(startOfDay.getMillis(), results.get(0).get(0)); + // Check doubleSum + Assert.assertEquals(n * segments.size(), (Double) results.get(0).get(1), 0.001); + } + private static void assertResultsMatch(List results, int rowNum, String expectedProduct) { ResultRow row = results.get(rowNum); diff --git a/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.java b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.java new file mode 100644 index 000000000000..643e2e57cc56 --- /dev/null +++ b/extensions-contrib/spectator-histogram/src/test/java/org/apache/druid/spectator/histogram/SpectatorHistogramIndexBasedComplexColumnTest.java @@ -0,0 +1,42 @@ +/* + * 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.spectator.histogram; + +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Test; + +public class SpectatorHistogramIndexBasedComplexColumnTest +{ + @Test + public void testComplexColumn() + { + final SpectatorHistogramIndexed mockIndexed = EasyMock.createMock(SpectatorHistogramIndexed.class); + EasyMock.replay(mockIndexed); + + final String typeName = "type"; + final SpectatorHistogramIndexBasedComplexColumn column = + new SpectatorHistogramIndexBasedComplexColumn("type", mockIndexed); + Assert.assertEquals(typeName, column.getTypeName()); + Assert.assertEquals(-1, column.getLength()); + + EasyMock.verify(mockIndexed); + } +} diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java index e21cae36d0f5..5fbfd3119d03 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/WindowOperatorQueryFrameProcessor.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; +import org.apache.druid.segment.column.NullableTypeStrategy; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; @@ -59,7 +60,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Objects; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -85,6 +85,10 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor private ResultRow outputRow = null; private FrameWriter frameWriter = null; + // List of type strategies to compare the partition columns across rows. + // Type strategies are pushed in the same order as column types in frameReader.signature() + private final NullableTypeStrategy[] typeStrategies; + public WindowOperatorQueryFrameProcessor( WindowOperatorQuery query, ReadableFrameChannel inputChannel, @@ -103,13 +107,18 @@ public WindowOperatorQueryFrameProcessor( this.frameWriterFactory = frameWriterFactory; this.operatorFactoryList = operatorFactoryList; this.jsonMapper = jsonMapper; - this.frameReader = frameReader; this.query = query; this.frameRowsAndCols = new ArrayList<>(); this.resultRowAndCols = new ArrayList<>(); this.objectsOfASingleRac = new ArrayList<>(); this.maxRowsMaterialized = maxRowsMaterializedInWindow; this.partitionColumnNames = partitionColumnNames; + + this.frameReader = frameReader; + this.typeStrategies = new NullableTypeStrategy[frameReader.signature().size()]; + for (int i = 0; i < frameReader.signature().size(); i++) { + typeStrategies[i] = frameReader.signature().getColumnType(i).get().getNullableStrategy(); + } } @Override @@ -499,7 +508,7 @@ private boolean comparePartitionKeys(ResultRow row1, ResultRow row2, List context) - { - - final Map adjustedContext = new HashMap<>(context); - adjustedContext.put(MultiStageQueryContext.CTX_ARRAY_INGEST_MODE, "none"); - - testIngestQuery().setSql( - "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") - .setQueryContext(adjustedContext) - .setExpectedExecutionErrorMatcher(CoreMatchers.allOf( - CoreMatchers.instanceOf(ISE.class), - ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString( - "String arrays can not be ingested when 'arrayIngestMode' is set to 'none'")) - )) - .verifyExecutionError(); - } - - /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to none (default) and the user tries to ingest + * Tests the behaviour of INSERT query when arrayIngestMode is set to default and the user tries to ingest * string arrays */ @MethodSource("data") @@ -172,7 +149,7 @@ public void testReplaceMvdWithStringArray(String contextName, Map contex } /** - * Tests the behaviour of INSERT query when arrayIngestMode is set to mvd (default) and the only array type to be - * ingested is string array + * Tests the behaviour of INSERT query when arrayIngestMode is set to array (default) */ @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") @@ -325,16 +301,32 @@ public void testInsertOnFoo1WithMultiValueToArrayGroupByWithDefaultContext(Strin { RowSignature rowSignature = RowSignature.builder() .add("__time", ColumnType.LONG) - .add("dim3", ColumnType.STRING) + .add("dim3", ColumnType.STRING_ARRAY) .build(); + List expectedRows = new ArrayList<>( + ImmutableList.of( + new Object[]{0L, null}, + new Object[]{0L, new Object[]{"a", "b"}} + ) + ); + if (!useDefault) { + expectedRows.add(new Object[]{0L, new Object[]{""}}); + } + expectedRows.addAll( + ImmutableList.of( + new Object[]{0L, new Object[]{"b", "c"}}, + new Object[]{0L, new Object[]{"d"}} + ) + ); + testIngestQuery().setSql( "INSERT INTO foo1 SELECT MV_TO_ARRAY(dim3) AS dim3 FROM foo GROUP BY 1 PARTITIONED BY ALL TIME") .setExpectedDataSource("foo1") .setExpectedRowSignature(rowSignature) .setQueryContext(context) .setExpectedSegment(ImmutableSet.of(SegmentId.of("foo1", Intervals.ETERNITY, "test", 0))) - .setExpectedResultRows(expectedMultiValueFooRowsToArray()) + .setExpectedResultRows(expectedRows) .verifyResults(); } @@ -603,13 +595,6 @@ public void testInsertArraysAsArrays(String contextName, Map con .verifyResults(); } - @MethodSource("data") - @ParameterizedTest(name = "{index}:with context {0}") - public void testSelectOnArraysWithArrayIngestModeAsNone(String contextName, Map context) - { - testSelectOnArrays(contextName, context, "none"); - } - @MethodSource("data") @ParameterizedTest(name = "{index}:with context {0}") public void testSelectOnArraysWithArrayIngestModeAsMVD(String contextName, Map context) @@ -1128,20 +1113,4 @@ public void testScanExternArrayWithNonConvertibleType(String contextName, Map expectedMultiValueFooRowsToArray() - { - List expectedRows = new ArrayList<>(); - expectedRows.add(new Object[]{0L, null}); - if (!useDefault) { - expectedRows.add(new Object[]{0L, ""}); - } - - expectedRows.addAll(ImmutableList.of( - new Object[]{0L, ImmutableList.of("a", "b")}, - new Object[]{0L, ImmutableList.of("b", "c")}, - new Object[]{0L, "d"} - )); - return expectedRows; - } } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java index afbedb7d7040..5cc84ac6ee61 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java @@ -106,7 +106,7 @@ public void testWindowOnFooWithPartitionByAndInnerGroupBy(String contextName, Ma .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d0") }; @@ -196,7 +196,7 @@ public void testWindowOnFooWithFirstWindowPartitionNextWindowEmpty(String contex .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d1") }; @@ -306,7 +306,7 @@ public void testWindowOnFooWith2WindowsBothWindowsHavingPartitionByInnerGroupBy( .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d1") }; @@ -419,7 +419,7 @@ public void testWindowOnFooWith2WindowsBothPartitionByWithOrderReversed( .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "d0") }; @@ -523,7 +523,7 @@ public void testWindowOnFooWithEmptyOverWithGroupBy(String contextName, Map cont .add("cc", ColumnType.DOUBLE) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new DoubleSumAggregatorFactory("w0", "m1") }; @@ -1322,14 +1305,7 @@ public void testWindowOnFooWithEmptyOverWithUnnest(String contextName, Map cont .add("cc", ColumnType.LONG) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "added") }; @@ -1887,7 +1856,7 @@ public void testSelectWithWikipediaWithPartitionKeyNotInSelect(String contextNam .add("cc", ColumnType.LONG) .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "added") }; @@ -2001,7 +1970,7 @@ public void testGroupByWithWikipedia(String contextName, Map con .build(); - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null); + final WindowFrame theFrame = WindowFrame.unbounded(); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("w0", "d1") }; diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java index 3036c2617622..1b0483d0b5a3 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/sql/resources/SqlMSQStatementResourcePostTest.java @@ -301,15 +301,7 @@ public void testExplain() throws IOException ), SqlStatementResourceTest.makeOkRequest()); Assert.assertEquals( - "{PLAN=[{\"query\":" - + "{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"context\":{\"__resultFormat\":\"object\",\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]," - + " RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]," - + " ATTRIBUTES={\"statementType\":\"SELECT\"}}", + "{PLAN=[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"__resultFormat\":\"object\",\"executionMode\":\"ASYNC\",\"scanSignature\":\"[{\\\"name\\\":\\\"__time\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"cnt\\\",\\\"type\\\":\\\"LONG\\\"},{\\\"name\\\":\\\"dim1\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim2\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"dim3\\\",\\\"type\\\":\\\"STRING\\\"},{\\\"name\\\":\\\"m1\\\",\\\"type\\\":\\\"FLOAT\\\"},{\\\"name\\\":\\\"m2\\\",\\\"type\\\":\\\"DOUBLE\\\"},{\\\"name\\\":\\\"unique_dim1\\\",\\\"type\\\":\\\"COMPLEX\\\"}]\",\"sqlQueryId\":\"queryId\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}], RESOURCES=[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}], ATTRIBUTES={\"statementType\":\"SELECT\"}}", String.valueOf(SqlStatementResourceTest.getResultRowsFromResponse(response).get(0)) ); } diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java index af08acf6ab73..bdd5270321a2 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/util/MultiStageQueryContextTest.java @@ -221,17 +221,12 @@ public void useAutoColumnSchemes_set_returnsCorrectValue() @Test public void arrayIngestMode_unset_returnsDefaultValue() { - Assert.assertEquals(ArrayIngestMode.MVD, MultiStageQueryContext.getArrayIngestMode(QueryContext.empty())); + Assert.assertEquals(ArrayIngestMode.ARRAY, MultiStageQueryContext.getArrayIngestMode(QueryContext.empty())); } @Test public void arrayIngestMode_set_returnsCorrectValue() { - Assert.assertEquals( - ArrayIngestMode.NONE, - MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "none"))) - ); - Assert.assertEquals( ArrayIngestMode.MVD, MultiStageQueryContext.getArrayIngestMode(QueryContext.of(ImmutableMap.of(CTX_ARRAY_INGEST_MODE, "mvd"))) diff --git a/licenses.yaml b/licenses.yaml index 03565a146605..dcdac7bd1879 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -5197,6 +5197,16 @@ license_file_path: licenses/bin/@emotion-weak-memoize.MIT --- +name: "@fontsource/open-sans" +license_category: binary +module: web-console +license_name: SIL Open Font License 1.1 +copyright: Google Inc. +version: 5.0.28 +license_file_path: licenses/bin/@fontsource-open-sans.OFL + +--- + name: "@juggle/resize-observer" license_category: binary module: web-console @@ -5743,16 +5753,6 @@ license_file_path: licenses/bin/follow-redirects.MIT --- -name: "fontsource-open-sans" -license_category: binary -module: web-console -license_name: MIT License -copyright: Lotus -version: 3.0.9 -license_file_path: licenses/bin/fontsource-open-sans.MIT - ---- - name: "form-data" license_category: binary module: web-console diff --git a/licenses/bin/@fontsource-open-sans.OFL b/licenses/bin/@fontsource-open-sans.OFL new file mode 100644 index 000000000000..d5e5ab592f4e --- /dev/null +++ b/licenses/bin/@fontsource-open-sans.OFL @@ -0,0 +1,93 @@ +Copyright 2020 The Open Sans Project Authors (https://github.com/googlefonts/opensans) + +This Font Software is licensed under the SIL Open Font License, Version 1.1. +This license is copied below, and is also available with a FAQ at: +http://scripts.sil.org/OFL + + +----------------------------------------------------------- +SIL OPEN FONT LICENSE Version 1.1 - 26 February 2007 +----------------------------------------------------------- + +PREAMBLE +The goals of the Open Font License (OFL) are to stimulate worldwide +development of collaborative font projects, to support the font creation +efforts of academic and linguistic communities, and to provide a free and +open framework in which fonts may be shared and improved in partnership +with others. + +The OFL allows the licensed fonts to be used, studied, modified and +redistributed freely as long as they are not sold by themselves. The +fonts, including any derivative works, can be bundled, embedded, +redistributed and/or sold with any software provided that any reserved +names are not used by derivative works. The fonts and derivatives, +however, cannot be released under any other type of license. The +requirement for fonts to remain under this license does not apply +to any document created using the fonts or their derivatives. + +DEFINITIONS +"Font Software" refers to the set of files released by the Copyright +Holder(s) under this license and clearly marked as such. This may +include source files, build scripts and documentation. + +"Reserved Font Name" refers to any names specified as such after the +copyright statement(s). + +"Original Version" refers to the collection of Font Software components as +distributed by the Copyright Holder(s). + +"Modified Version" refers to any derivative made by adding to, deleting, +or substituting -- in part or in whole -- any of the components of the +Original Version, by changing formats or by porting the Font Software to a +new environment. + +"Author" refers to any designer, engineer, programmer, technical +writer or other person who contributed to the Font Software. + +PERMISSION & CONDITIONS +Permission is hereby granted, free of charge, to any person obtaining +a copy of the Font Software, to use, study, copy, merge, embed, modify, +redistribute, and sell modified and unmodified copies of the Font +Software, subject to the following conditions: + +1) Neither the Font Software nor any of its individual components, +in Original or Modified Versions, may be sold by itself. + +2) Original or Modified Versions of the Font Software may be bundled, +redistributed and/or sold with any software, provided that each copy +contains the above copyright notice and this license. These can be +included either as stand-alone text files, human-readable headers or +in the appropriate machine-readable metadata fields within text or +binary files as long as those fields can be easily viewed by the user. + +3) No Modified Version of the Font Software may use the Reserved Font +Name(s) unless explicit written permission is granted by the corresponding +Copyright Holder. This restriction only applies to the primary font name as +presented to the users. + +4) The name(s) of the Copyright Holder(s) or the Author(s) of the Font +Software shall not be used to promote, endorse or advertise any +Modified Version, except to acknowledge the contribution(s) of the +Copyright Holder(s) and the Author(s) or with their explicit written +permission. + +5) The Font Software, modified or unmodified, in part or in whole, +must be distributed entirely under this license, and must not be +distributed under any other license. The requirement for fonts to +remain under this license does not apply to any document created +using the Font Software. + +TERMINATION +This license becomes null and void if any of the above conditions are +not met. + +DISCLAIMER +THE FONT SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, +EXPRESS OR IMPLIED, INCLUDING BUT NOT LIMITED TO ANY WARRANTIES OF +MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT +OF COPYRIGHT, PATENT, TRADEMARK, OR OTHER RIGHT. IN NO EVENT SHALL THE +COPYRIGHT HOLDER BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, +INCLUDING ANY GENERAL, SPECIAL, INDIRECT, INCIDENTAL, OR CONSEQUENTIAL +DAMAGES, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING +FROM, OUT OF THE USE OR INABILITY TO USE THE FONT SOFTWARE OR FROM +OTHER DEALINGS IN THE FONT SOFTWARE. diff --git a/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java b/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java index 517235a99f98..ca2708700f08 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequence.java @@ -19,10 +19,9 @@ package org.apache.druid.java.util.common.guava; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import org.apache.druid.java.util.common.RE; +import com.google.common.util.concurrent.AbstractFuture; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryTimeoutException; @@ -63,6 +62,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase { private static final Logger LOG = new Logger(ParallelMergeCombiningSequence.class); + private static final long BLOCK_TIMEOUT = TimeUnit.NANOSECONDS.convert(500, TimeUnit.MILLISECONDS); // these values were chosen carefully via feedback from benchmarks, // see PR https://github.com/apache/druid/pull/8578 for details @@ -84,7 +84,7 @@ public class ParallelMergeCombiningSequence extends YieldingSequenceBase private final long targetTimeNanos; private final Consumer metricsReporter; - private final CancellationGizmo cancellationGizmo; + private final CancellationFuture cancellationFuture; public ParallelMergeCombiningSequence( ForkJoinPool workerPool, @@ -114,14 +114,24 @@ public ParallelMergeCombiningSequence( this.targetTimeNanos = TimeUnit.NANOSECONDS.convert(targetTimeMillis, TimeUnit.MILLISECONDS); this.queueSize = (1 << 15) / batchSize; // each queue can by default hold ~32k rows this.metricsReporter = reporter; - this.cancellationGizmo = new CancellationGizmo(); + this.cancellationFuture = new CancellationFuture(new CancellationGizmo()); } @Override public Yielder toYielder(OutType initValue, YieldingAccumulator accumulator) { if (inputSequences.isEmpty()) { - return Sequences.empty().toYielder(initValue, accumulator); + return Sequences.wrap( + Sequences.empty(), + new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) + { + cancellationFuture.set(true); + } + } + ).toYielder(initValue, accumulator); } // we make final output queue larger than the merging queues so if downstream readers are slower to read there is // less chance of blocking the merge @@ -144,27 +154,43 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat hasTimeout, timeoutAtNanos, metricsAccumulator, - cancellationGizmo + cancellationFuture.cancellationGizmo ); workerPool.execute(mergeCombineAction); - Sequence finalOutSequence = makeOutputSequenceForQueue( - outputQueue, - hasTimeout, - timeoutAtNanos, - cancellationGizmo - ).withBaggage(() -> { - if (metricsReporter != null) { - metricsAccumulator.setTotalWallTime(System.nanoTime() - startTimeNanos); - metricsReporter.accept(metricsAccumulator.build()); - } - }); + + final Sequence finalOutSequence = Sequences.wrap( + makeOutputSequenceForQueue( + outputQueue, + hasTimeout, + timeoutAtNanos, + cancellationFuture.cancellationGizmo + ), + new SequenceWrapper() + { + @Override + public void after(boolean isDone, Throwable thrown) + { + if (isDone) { + cancellationFuture.set(true); + } else { + cancellationFuture.cancel(true); + } + if (metricsReporter != null) { + metricsAccumulator.setTotalWallTime(System.nanoTime() - startTimeNanos); + metricsReporter.accept(metricsAccumulator.build()); + } + } + } + ); return finalOutSequence.toYielder(initValue, accumulator); } - @VisibleForTesting - public CancellationGizmo getCancellationGizmo() + /** + * + */ + public CancellationFuture getCancellationFuture() { - return cancellationGizmo; + return cancellationFuture; } /** @@ -181,8 +207,6 @@ static Sequence makeOutputSequenceForQueue( return new BaseSequence<>( new BaseSequence.IteratorMaker>() { - private boolean shouldCancelOnCleanup = true; - @Override public Iterator make() { @@ -195,7 +219,7 @@ public boolean hasNext() { final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); if (hasTimeout && thisTimeoutNanos < 0) { - throw new QueryTimeoutException(); + throw cancellationGizmo.cancelAndThrow(new QueryTimeoutException()); } if (currentBatch != null && !currentBatch.isTerminalResult() && !currentBatch.isDrained()) { @@ -210,33 +234,32 @@ public boolean hasNext() } } if (currentBatch == null) { - throw new QueryTimeoutException(); + throw cancellationGizmo.cancelAndThrow(new QueryTimeoutException()); } - if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.isCanceled()) { throw cancellationGizmo.getRuntimeException(); } if (currentBatch.isTerminalResult()) { - shouldCancelOnCleanup = false; return false; } return true; } catch (InterruptedException e) { - throw new RE(e); + throw cancellationGizmo.cancelAndThrow(e); } } @Override public T next() { - if (cancellationGizmo.isCancelled()) { + if (cancellationGizmo.isCanceled()) { throw cancellationGizmo.getRuntimeException(); } if (currentBatch == null || currentBatch.isDrained() || currentBatch.isTerminalResult()) { - throw new NoSuchElementException(); + throw cancellationGizmo.cancelAndThrow(new NoSuchElementException()); } return currentBatch.next(); } @@ -246,9 +269,7 @@ public T next() @Override public void cleanup(Iterator iterFromMake) { - if (shouldCancelOnCleanup) { - cancellationGizmo.cancel(new RuntimeException("Already closed")); - } + // nothing to cleanup } } ); @@ -338,7 +359,7 @@ protected void compute() parallelTaskCount ); - QueuePusher> resultsPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + QueuePusher resultsPusher = new QueuePusher<>(out, cancellationGizmo, hasTimeout, timeoutAt); for (Sequence s : sequences) { sequenceCursors.add(new YielderBatchedResultsCursor<>(new SequenceBatcher<>(s, batchSize), orderingFn)); @@ -367,10 +388,10 @@ protected void compute() catch (Throwable t) { closeAllCursors(sequenceCursors); cancellationGizmo.cancel(t); - // Should be the following, but can' change due to lack of - // unit tests. - // out.offer((ParallelMergeCombiningSequence.ResultBatch) ResultBatch.TERMINAL); - out.offer(ResultBatch.TERMINAL); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + out.offer(ResultBatch.terminal()); } } @@ -387,7 +408,7 @@ private void spawnParallelTasks(int parallelMergeTasks) for (List> partition : partitions) { BlockingQueue> outputQueue = new ArrayBlockingQueue<>(queueSize); intermediaryOutputs.add(outputQueue); - QueuePusher> pusher = new QueuePusher<>(outputQueue, hasTimeout, timeoutAt); + QueuePusher pusher = new QueuePusher<>(outputQueue, cancellationGizmo, hasTimeout, timeoutAt); List> partitionCursors = new ArrayList<>(sequences.size()); for (Sequence s : partition) { @@ -415,11 +436,11 @@ private void spawnParallelTasks(int parallelMergeTasks) getPool().execute(task); } - QueuePusher> outputPusher = new QueuePusher<>(out, hasTimeout, timeoutAt); + QueuePusher outputPusher = new QueuePusher<>(out, cancellationGizmo, hasTimeout, timeoutAt); List> intermediaryOutputsCursors = new ArrayList<>(intermediaryOutputs.size()); for (BlockingQueue> queue : intermediaryOutputs) { intermediaryOutputsCursors.add( - new BlockingQueueuBatchedResultsCursor<>(queue, orderingFn, hasTimeout, timeoutAt) + new BlockingQueueuBatchedResultsCursor<>(queue, cancellationGizmo, orderingFn, hasTimeout, timeoutAt) ); } MergeCombineActionMetricsAccumulator finalMergeMetrics = new MergeCombineActionMetricsAccumulator(); @@ -513,7 +534,7 @@ private static class MergeCombineAction extends RecursiveAction private final PriorityQueue> pQueue; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher outputQueue; private final T initialValue; private final int yieldAfter; private final int batchSize; @@ -523,7 +544,7 @@ private static class MergeCombineAction extends RecursiveAction private MergeCombineAction( PriorityQueue> pQueue, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, T initialValue, @@ -550,6 +571,10 @@ private MergeCombineAction( @Override protected void compute() { + if (cancellationGizmo.isCanceled()) { + cleanup(); + return; + } try { long start = System.nanoTime(); long startCpuNanos = JvmUtils.safeGetThreadCpuTime(); @@ -608,7 +633,7 @@ protected void compute() metricsAccumulator.incrementCpuTimeNanos(elapsedCpuNanos); metricsAccumulator.incrementTaskCount(); - if (!pQueue.isEmpty() && !cancellationGizmo.isCancelled()) { + if (!pQueue.isEmpty() && !cancellationGizmo.isCanceled()) { // if there is still work to be done, execute a new task with the current accumulated value to continue // combining where we left off if (!outputBatch.isDrained()) { @@ -650,29 +675,36 @@ protected void compute() metricsAccumulator, cancellationGizmo )); - } else if (cancellationGizmo.isCancelled()) { + } else if (cancellationGizmo.isCanceled()) { // if we got the cancellation signal, go ahead and write terminal value into output queue to help gracefully // allow downstream stuff to stop - LOG.debug("cancelled after %s tasks", metricsAccumulator.getTaskCount()); + LOG.debug("canceled after %s tasks", metricsAccumulator.getTaskCount()); // make sure to close underlying cursors - closeAllCursors(pQueue); - outputQueue.offer(ResultBatch.TERMINAL); + cleanup(); } else { // if priority queue is empty, push the final accumulated value into the output batch and push it out outputBatch.add(currentCombinedValue); metricsAccumulator.incrementOutputRows(batchCounter + 1L); outputQueue.offer(outputBatch); // ... and the terminal value to indicate the blocking queue holding the values is complete - outputQueue.offer(ResultBatch.TERMINAL); + outputQueue.offer(ResultBatch.terminal()); LOG.debug("merge combine complete after %s tasks", metricsAccumulator.getTaskCount()); } } catch (Throwable t) { - closeAllCursors(pQueue); cancellationGizmo.cancel(t); - outputQueue.offer(ResultBatch.TERMINAL); + cleanup(); } } + + private void cleanup() + { + closeAllCursors(pQueue); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + outputQueue.offer(ResultBatch.terminal()); + } } @@ -696,7 +728,7 @@ private static class PrepareMergeCombineInputsAction extends RecursiveAction private final List> partition; private final Ordering orderingFn; private final BinaryOperator combineFn; - private final QueuePusher> outputQueue; + private final QueuePusher outputQueue; private final int yieldAfter; private final int batchSize; private final long targetTimeNanos; @@ -707,7 +739,7 @@ private static class PrepareMergeCombineInputsAction extends RecursiveAction private PrepareMergeCombineInputsAction( List> partition, - QueuePusher> outputQueue, + QueuePusher outputQueue, Ordering orderingFn, BinaryOperator combineFn, int yieldAfter, @@ -744,7 +776,7 @@ protected void compute() cursor.close(); } } - if (cursors.size() > 0) { + if (!cancellationGizmo.isCanceled() && !cursors.isEmpty()) { getPool().execute(new MergeCombineAction( cursors, outputQueue, @@ -758,14 +790,17 @@ protected void compute() cancellationGizmo )); } else { - outputQueue.offer(ResultBatch.TERMINAL); + outputQueue.offer(ResultBatch.terminal()); } metricsAccumulator.setPartitionInitializedTime(System.nanoTime() - startTime); } catch (Throwable t) { closeAllCursors(partition); cancellationGizmo.cancel(t); - outputQueue.offer(ResultBatch.TERMINAL); + // offer terminal result if queue is not full in case out is empty to allow downstream threads waiting on + // stuff to be present to stop blocking immediately. However, if the queue is full, it doesn't matter if we + // write anything because the cancellation signal has been set, which will also terminate processing. + outputQueue.tryOfferTerminal(); } } } @@ -779,12 +814,14 @@ static class QueuePusher implements ForkJoinPool.ManagedBlocker { final boolean hasTimeout; final long timeoutAtNanos; - final BlockingQueue queue; - volatile E item = null; + final BlockingQueue> queue; + final CancellationGizmo gizmo; + volatile ResultBatch item = null; - QueuePusher(BlockingQueue q, boolean hasTimeout, long timeoutAtNanos) + QueuePusher(BlockingQueue> q, CancellationGizmo gizmo, boolean hasTimeout, long timeoutAtNanos) { this.queue = q; + this.gizmo = gizmo; this.hasTimeout = hasTimeout; this.timeoutAtNanos = timeoutAtNanos; } @@ -795,14 +832,16 @@ public boolean block() throws InterruptedException boolean success = false; if (item != null) { if (hasTimeout) { - final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); - if (thisTimeoutNanos < 0) { + final long remainingNanos = timeoutAtNanos - System.nanoTime(); + if (remainingNanos < 0) { item = null; - throw new QueryTimeoutException("QueuePusher timed out offering data"); + throw gizmo.cancelAndThrow(new QueryTimeoutException()); } - success = queue.offer(item, thisTimeoutNanos, TimeUnit.NANOSECONDS); + final long blockTimeoutNanos = Math.min(remainingNanos, BLOCK_TIMEOUT); + success = queue.offer(item, blockTimeoutNanos, TimeUnit.NANOSECONDS); } else { - success = queue.offer(item); + queue.put(item); + success = true; } if (success) { item = null; @@ -817,7 +856,7 @@ public boolean isReleasable() return item == null; } - public void offer(E item) + public void offer(ResultBatch item) { try { this.item = item; @@ -828,6 +867,11 @@ public void offer(E item) throw new RuntimeException("Failed to offer result to output queue", e); } } + + public void tryOfferTerminal() + { + this.queue.offer(ResultBatch.terminal()); + } } /** @@ -837,8 +881,10 @@ public void offer(E item) */ static class ResultBatch { - @SuppressWarnings("rawtypes") - static final ResultBatch TERMINAL = new ResultBatch(); + static ResultBatch terminal() + { + return new ResultBatch<>(); + } @Nullable private final Queue values; @@ -855,19 +901,16 @@ private ResultBatch() public void add(E in) { - assert values != null; values.offer(in); } public E get() { - assert values != null; return values.peek(); } public E next() { - assert values != null; return values.poll(); } @@ -925,6 +968,7 @@ static class SequenceBatcher implements ForkJoinPool.ManagedBlocker Yielder> getBatchYielder() { try { + batchYielder = null; ForkJoinPool.managedBlock(this); return batchYielder; } @@ -1033,8 +1077,8 @@ static class YielderBatchedResultsCursor extends BatchedResultsCursor @Override public void initialize() { - yielder = batcher.getBatchYielder(); - resultBatch = yielder.get(); + yielder = null; + nextBatch(); } @Override @@ -1059,6 +1103,10 @@ public boolean isDone() @Override public boolean block() { + if (yielder == null) { + yielder = batcher.getBatchYielder(); + resultBatch = yielder.get(); + } if (yielder.isDone()) { return true; } @@ -1073,7 +1121,7 @@ public boolean block() @Override public boolean isReleasable() { - return yielder.isDone() || (resultBatch != null && !resultBatch.isDrained()); + return (yielder != null && yielder.isDone()) || (resultBatch != null && !resultBatch.isDrained()); } @Override @@ -1092,11 +1140,13 @@ public void close() throws IOException static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor { final BlockingQueue> queue; + final CancellationGizmo gizmo; final boolean hasTimeout; final long timeoutAtNanos; BlockingQueueuBatchedResultsCursor( BlockingQueue> blockingQueue, + CancellationGizmo cancellationGizmo, Ordering ordering, boolean hasTimeout, long timeoutAtNanos @@ -1104,6 +1154,7 @@ static class BlockingQueueuBatchedResultsCursor extends BatchedResultsCursor< { super(ordering); this.queue = blockingQueue; + this.gizmo = cancellationGizmo; this.hasTimeout = hasTimeout; this.timeoutAtNanos = timeoutAtNanos; } @@ -1142,17 +1193,18 @@ public boolean block() throws InterruptedException { if (resultBatch == null || resultBatch.isDrained()) { if (hasTimeout) { - final long thisTimeoutNanos = timeoutAtNanos - System.nanoTime(); - if (thisTimeoutNanos < 0) { - resultBatch = ResultBatch.TERMINAL; - throw new QueryTimeoutException("BlockingQueue cursor timed out waiting for data"); + final long remainingNanos = timeoutAtNanos - System.nanoTime(); + if (remainingNanos < 0) { + resultBatch = ResultBatch.terminal(); + throw gizmo.cancelAndThrow(new QueryTimeoutException()); } - resultBatch = queue.poll(thisTimeoutNanos, TimeUnit.NANOSECONDS); + final long blockTimeoutNanos = Math.min(remainingNanos, BLOCK_TIMEOUT); + resultBatch = queue.poll(blockTimeoutNanos, TimeUnit.NANOSECONDS); } else { resultBatch = queue.take(); } } - return resultBatch != null; + return resultBatch != null && !resultBatch.isDrained(); } @Override @@ -1164,35 +1216,91 @@ public boolean isReleasable() } // if we can get a result immediately without blocking, also no need to block resultBatch = queue.poll(); - return resultBatch != null; + return resultBatch != null && !resultBatch.isDrained(); } } /** - * Token to allow any {@link RecursiveAction} signal the others and the output sequence that something bad happened - * and processing should cancel, such as a timeout or connection loss. + * Token used to stop internal parallel processing across all tasks in the merge pool. Allows any + * {@link RecursiveAction} signal the others and the output sequence that something bad happened and + * processing should cancel, such as a timeout, error, or connection loss. */ - static class CancellationGizmo + public static class CancellationGizmo { private final AtomicReference throwable = new AtomicReference<>(null); + RuntimeException cancelAndThrow(Throwable t) + { + throwable.compareAndSet(null, t); + return wrapRuntimeException(t); + } + void cancel(Throwable t) { throwable.compareAndSet(null, t); } - boolean isCancelled() + boolean isCanceled() { return throwable.get() != null; } RuntimeException getRuntimeException() { - Throwable ex = throwable.get(); - if (ex instanceof RuntimeException) { - return (RuntimeException) ex; + return wrapRuntimeException(throwable.get()); + } + + private static RuntimeException wrapRuntimeException(Throwable t) + { + if (t instanceof RuntimeException) { + return (RuntimeException) t; } - return new RE(ex); + return new RuntimeException(t); + } + } + + /** + * {@link com.google.common.util.concurrent.ListenableFuture} that allows {@link ParallelMergeCombiningSequence} to be + * registered with {@link org.apache.druid.query.QueryWatcher#registerQueryFuture} to participate in query + * cancellation or anything else that has a need to watch the activity on the merge pool. Wraps a + * {@link CancellationGizmo} to allow for external threads to signal cancellation of parallel processing on the pool + * by triggering {@link CancellationGizmo#cancel(Throwable)} whenever {@link #cancel(boolean)} is called. + * + * This is not used internally by workers on the pool in favor of using the much simpler {@link CancellationGizmo} + * directly instead. + */ + public static class CancellationFuture extends AbstractFuture + { + private final CancellationGizmo cancellationGizmo; + + public CancellationFuture(CancellationGizmo cancellationGizmo) + { + this.cancellationGizmo = cancellationGizmo; + } + + public CancellationGizmo getCancellationGizmo() + { + return cancellationGizmo; + } + + @Override + public boolean set(Boolean value) + { + return super.set(value); + } + + @Override + public boolean setException(Throwable throwable) + { + cancellationGizmo.cancel(throwable); + return super.setException(throwable); + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) + { + cancellationGizmo.cancel(new RuntimeException("Sequence canceled")); + return super.cancel(mayInterruptIfRunning); } } @@ -1308,8 +1416,8 @@ public long getSlowestPartitionInitializedTime() */ static class MergeCombineMetricsAccumulator { - List partitionMetrics; - MergeCombineActionMetricsAccumulator mergeMetrics; + List partitionMetrics = Collections.emptyList(); + MergeCombineActionMetricsAccumulator mergeMetrics = new MergeCombineActionMetricsAccumulator(); private long totalWallTime; @@ -1343,8 +1451,8 @@ MergeCombineMetrics build() // partition long totalPoolTasks = 1 + 1 + partitionMetrics.size(); - long fastestPartInitialized = partitionMetrics.size() > 0 ? Long.MAX_VALUE : mergeMetrics.getPartitionInitializedtime(); - long slowestPartInitialied = partitionMetrics.size() > 0 ? Long.MIN_VALUE : mergeMetrics.getPartitionInitializedtime(); + long fastestPartInitialized = !partitionMetrics.isEmpty() ? Long.MAX_VALUE : mergeMetrics.getPartitionInitializedtime(); + long slowestPartInitialied = !partitionMetrics.isEmpty() ? Long.MIN_VALUE : mergeMetrics.getPartitionInitializedtime(); // accumulate input row count, cpu time, and total number of tasks from each partition for (MergeCombineActionMetricsAccumulator partition : partitionMetrics) { diff --git a/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index bfb1dc79d788..b02de3ce4932 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,8 +20,11 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.JsonToken; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JavaType; import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.JsonSerializer; @@ -122,6 +125,40 @@ public static void writeObjectUsingSerializerProvider( } } + /** + * Reads an object using the {@link JsonParser}. It reuses the provided {@link DeserializationContext} which offers + * better performance that calling {@link JsonParser#readValueAs(Class)} because it avoids re-creating the {@link DeserializationContext} + * for each readValue call + */ + @Nullable + public static T readObjectUsingDeserializationContext( + final JsonParser jp, + final DeserializationContext deserializationContext, + final Class clazz + ) throws IOException + { + if (jp.currentToken() == JsonToken.VALUE_NULL) { + return null; + } + return deserializationContext.readValue(jp, clazz); + } + + /** + * @see #readObjectUsingDeserializationContext(JsonParser, DeserializationContext, Class) + */ + @Nullable + public static Object readObjectUsingDeserializationContext( + final JsonParser jp, + final DeserializationContext deserializationContext, + final JavaType javaType + ) throws IOException + { + if (jp.currentToken() == JsonToken.VALUE_NULL) { + return null; + } + return deserializationContext.readValue(jp, javaType); + } + /** * Convert the given object to an array of bytes. Use when the object is * known serializable so that the Jackson exception can be suppressed. 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 fa394beec43a..978b49226154 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -88,6 +88,11 @@ public final JavaType getBySegmentResultType() * For most queries, this is a no-op, but it can be useful for query types that support more than one result * serialization format. Queries that implement this method must not modify the provided ObjectMapper, but instead * must return a copy. + *

+ * Jackson's default implementation of deserialization is usually optimised and this method should be overriden + * only if there is a functional requirement of so. The method must be benchmarked in isolation, without other portions + * of the query engine executing as modifying this method can alter the performance of queries where deserializing is + * a major portion of the execution. */ public ObjectMapper decorateObjectMapper(final ObjectMapper objectMapper, final QueryType query) { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java index 2034ba21a5be..c4c9a7875ef0 100755 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorUtil.java @@ -28,6 +28,8 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; @@ -35,6 +37,8 @@ import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.Types; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.segment.virtual.ExpressionSelectors; @@ -428,4 +432,26 @@ public static Supplier getSimpleAggregatorCacheKeySupplier( .array(); }); } + + /** + * Whether a simple numeric aggregator should use {@link BaseObjectColumnValueSelector#getObject()}, and coerce the + * result to number, rather than using a primitive method like {@link BaseLongColumnValueSelector#getLong()}. + * + * @param fieldName field name, or null if the aggregator is expression-based + * @param columnSelectorFactory column selector factory + */ + public static boolean shouldUseObjectColumnAggregatorWrapper( + @Nullable final String fieldName, + final ColumnSelectorFactory columnSelectorFactory + ) + { + if (fieldName != null) { + ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); + + // STRING can be coerced to a number. COMPLEX types can be subclasses of Number (or subclasses of some type + // that is coercible to a number.) + return Types.is(capabilities, ValueType.STRING) || Types.is(capabilities, ValueType.COMPLEX); + } + return false; + } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java index c1b4b4090235..6b0d0fb49696 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingAggregator.java @@ -23,7 +23,7 @@ /** * An Aggregator that delegates everything. It is used by Aggregator wrappers e.g. - * {@link StringColumnDoubleAggregatorWrapper} that modify some behavior of a delegate. + * {@link ObjectColumnDoubleAggregatorWrapper} that modify some behavior of a delegate. */ public abstract class DelegatingAggregator implements Aggregator { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java index 9b1aa8086b2b..87718f331088 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DelegatingBufferAggregator.java @@ -26,7 +26,7 @@ /** * A BufferAggregator that delegates everything. It is used by BufferAggregator wrappers e.g. - * {@link StringColumnDoubleBufferAggregatorWrapper} that modify some behavior of a delegate. + * {@link ObjectColumnDoubleBufferAggregatorWrapper} that modify some behavior of a delegate. */ public abstract class DelegatingBufferAggregator implements BufferAggregator { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java index c9928f828f7f..d85d91936d78 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java @@ -24,6 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.segment.BaseNullableColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.column.ColumnType; @@ -38,8 +39,9 @@ * values to be aggregated are null values, or if no values are aggregated at all. If any of the values are non-null, * the result will be the aggregated value of the non-null values. * - * This superclass should only be extended by aggregators that read primitive numbers. It implements logic that is - * not valid for non-numeric selector methods such as {@link ColumnValueSelector#getObject()}. + * Aggregators that use {@link ColumnValueSelector#getObject()} must override + * {@link #useGetObject(ColumnSelectorFactory)}. Otherwise, the logic in this class is not correct for + * non-numeric selectors. * * @see BaseNullableColumnValueSelector#isNull() for why this only works in the numeric case */ @@ -51,16 +53,18 @@ public abstract class NullableNumericAggregatorFactory combiner = makeAggregateCombiner(); + return NullHandling.sqlCompatible() ? new NullableNumericAggregateCombiner<>(combiner) : combiner; } @Override @@ -85,6 +89,23 @@ public final int getMaxIntermediateSizeWithNulls() return getMaxIntermediateSize() + (NullHandling.replaceWithDefault() ? 0 : Byte.BYTES); } + /** + * Returns the selector that should be used by {@link NullableNumericAggregator} and + * {@link NullableNumericBufferAggregator} to determine if the current value is null. + */ + private BaseNullableColumnValueSelector makeNullSelector( + final T selector, + final ColumnSelectorFactory columnSelectorFactory + ) + { + if (useGetObject(columnSelectorFactory)) { + final BaseObjectColumnValueSelector objectSelector = (BaseObjectColumnValueSelector) selector; + return () -> objectSelector.getObject() == null; + } else { + return selector; + } + } + // ---- ABSTRACT METHODS BELOW ------ /** @@ -94,6 +115,17 @@ public final int getMaxIntermediateSizeWithNulls() */ protected abstract T selector(ColumnSelectorFactory columnSelectorFactory); + /** + * Returns whether the selector created by {@link #selector(ColumnSelectorFactory)} for the given + * {@link ColumnSelectorFactory} prefers {@link BaseObjectColumnValueSelector#getObject()}. + * + * For backwards compatibilty with older extensions, this is a non-abstract method. + */ + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) + { + return false; + } + /** * Creates a {@link VectorValueSelector} for the aggregated column. * diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleAggregatorWrapper.java index e970c94a028c..b250eacc78bb 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleAggregatorWrapper.java @@ -28,15 +28,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Double Aggregator that consume double type columns to handle String type. + * This class can be used to wrap Double Aggregator that consume double type columns to handle Object type. */ -public class StringColumnDoubleAggregatorWrapper extends DelegatingAggregator +public class ObjectColumnDoubleAggregatorWrapper extends DelegatingAggregator { private final BaseObjectColumnValueSelector selector; private final double nullValue; private final SettableValueDoubleColumnValueSelector doubleSelector; - public StringColumnDoubleAggregatorWrapper( + public ObjectColumnDoubleAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, double nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java index fb58ad5cc498..f50a6371f93d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnDoubleBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnDoubleBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Double BufferAggregator that consume double type columns to handle String type. + * This class can be used to wrap Double BufferAggregator that consume double type columns to handle Object type. */ -public class StringColumnDoubleBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnDoubleBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final double nullValue; private final SettableValueDoubleColumnValueSelector doubleSelector; - public StringColumnDoubleBufferAggregatorWrapper( + public ObjectColumnDoubleBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, double nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java index bb7cd65c409b..0f738b4bc0d9 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatAggregatorWrapper.java @@ -28,15 +28,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Float Aggregator that consume float type columns to handle String type. + * This class can be used to wrap Float Aggregator that consume float type columns to handle Object type. */ -public class StringColumnFloatAggregatorWrapper extends DelegatingAggregator +public class ObjectColumnFloatAggregatorWrapper extends DelegatingAggregator { private final BaseObjectColumnValueSelector selector; private final float nullValue; private final SettableValueFloatColumnValueSelector floatSelector; - public StringColumnFloatAggregatorWrapper( + public ObjectColumnFloatAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, float nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java index 7c1c5e548810..1c3725e968aa 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnFloatBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnFloatBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Float BufferAggregator that consume float type columns to handle String type. + * This class can be used to wrap Float BufferAggregator that consume float type columns to handle Object type. */ -public class StringColumnFloatBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnFloatBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final float nullValue; private final SettableValueFloatColumnValueSelector floatSelector; - public StringColumnFloatBufferAggregatorWrapper( + public ObjectColumnFloatBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, float nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java index d218ab38cfb3..01b571a4e14d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongAggregatorWrapper.java @@ -28,15 +28,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Long Aggregator that consume long type columns to handle String type. + * This class can be used to wrap Long Aggregator that consume long type columns to handle Object type. */ -public class StringColumnLongAggregatorWrapper extends DelegatingAggregator +public class ObjectColumnLongAggregatorWrapper extends DelegatingAggregator { private final BaseObjectColumnValueSelector selector; private final long nullValue; private final SettableValueLongColumnValueSelector longSelector; - public StringColumnLongAggregatorWrapper( + public ObjectColumnLongAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, long nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java similarity index 93% rename from processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java rename to processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java index ad2e6c2cbc58..831c9a8f26ef 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/StringColumnLongBufferAggregatorWrapper.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/ObjectColumnLongBufferAggregatorWrapper.java @@ -29,15 +29,15 @@ import java.util.function.Function; /** - * This class can be used to wrap Long BufferAggregator that consume long type columns to handle String type. + * This class can be used to wrap Long BufferAggregator that consume long type columns to handle Object type. */ -public class StringColumnLongBufferAggregatorWrapper extends DelegatingBufferAggregator +public class ObjectColumnLongBufferAggregatorWrapper extends DelegatingBufferAggregator { private final BaseObjectColumnValueSelector selector; private final long nullValue; private final SettableValueLongColumnValueSelector longSelector; - public StringColumnLongBufferAggregatorWrapper( + public ObjectColumnLongBufferAggregatorWrapper( BaseObjectColumnValueSelector selector, Function delegateBuilder, long nullValue diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java index 324a10bb4d14..0fa96e226eae 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleDoubleAggregatorFactory.java @@ -31,11 +31,8 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -86,8 +83,8 @@ public SimpleDoubleAggregatorFactory( @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnDoubleAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnDoubleAggregatorWrapper( selector, SimpleDoubleAggregatorFactory.this::buildAggregator, nullValue() @@ -103,8 +100,8 @@ protected BufferAggregator factorizeBuffered( ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnDoubleBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnDoubleBufferAggregatorWrapper( selector, SimpleDoubleAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -131,13 +128,10 @@ protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnS return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java index 7633d39d9723..5268c454ce1b 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleFloatAggregatorFactory.java @@ -31,10 +31,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -76,8 +73,8 @@ public SimpleFloatAggregatorFactory( @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnFloatAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnFloatAggregatorWrapper( selector, SimpleFloatAggregatorFactory.this::buildAggregator, nullValue() @@ -93,8 +90,8 @@ protected BufferAggregator factorizeBuffered( ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnFloatBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnFloatBufferAggregatorWrapper( selector, SimpleFloatAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -121,6 +118,12 @@ protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnS return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) + { + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); + } + @Override public Object deserialize(Object object) { @@ -233,15 +236,6 @@ public boolean canVectorize(ColumnInspector columnInspector) return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) - { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; - } - protected abstract float nullValue(); protected abstract Aggregator buildAggregator(BaseFloatColumnValueSelector selector); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java index 173a9cb229db..c4bc5307ed48 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SimpleLongAggregatorFactory.java @@ -31,10 +31,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.Types; -import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -82,8 +79,8 @@ public SimpleLongAggregatorFactory( @Override protected Aggregator factorize(ColumnSelectorFactory metricFactory, ColumnValueSelector selector) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnLongAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnLongAggregatorWrapper( selector, SimpleLongAggregatorFactory.this::buildAggregator, nullValue() @@ -99,8 +96,8 @@ protected BufferAggregator factorizeBuffered( ColumnValueSelector selector ) { - if (shouldUseStringColumnAggregatorWrapper(metricFactory)) { - return new StringColumnLongBufferAggregatorWrapper( + if (AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, metricFactory)) { + return new ObjectColumnLongBufferAggregatorWrapper( selector, SimpleLongAggregatorFactory.this::buildBufferAggregator, nullValue() @@ -127,6 +124,12 @@ protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnS return AggregatorUtil.makeVectorValueSelector(columnSelectorFactory, fieldName, expression, fieldExpression); } + @Override + protected boolean useGetObject(ColumnSelectorFactory columnSelectorFactory) + { + return AggregatorUtil.shouldUseObjectColumnAggregatorWrapper(fieldName, columnSelectorFactory); + } + @Override public Object deserialize(Object object) { @@ -236,15 +239,6 @@ public boolean canVectorize(ColumnInspector columnInspector) return AggregatorUtil.canVectorize(columnInspector, fieldName, expression, fieldExpression); } - private boolean shouldUseStringColumnAggregatorWrapper(ColumnSelectorFactory columnSelectorFactory) - { - if (fieldName != null) { - ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(fieldName); - return Types.is(capabilities, ValueType.STRING); - } - return false; - } - protected abstract long nullValue(); protected abstract Aggregator buildAggregator(BaseLongColumnValueSelector selector); diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java index 9950695f28ce..dbd6f2869a75 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryConfig.java @@ -293,7 +293,6 @@ public boolean isVectorize() return vectorize; } - @SuppressWarnings("unused") public boolean isIntermediateResultAsMapCompat() { return intermediateResultAsMapCompat; 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 3f2faf3fc562..a45205249687 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 @@ -19,23 +19,14 @@ package org.apache.druid.query.groupby; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.core.ObjectCodec; import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.module.SimpleModule; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Functions; +import com.google.common.base.Supplier; import com.google.common.collect.Lists; import com.google.inject.Inject; -import org.apache.druid.data.input.Row; import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.MemoryAllocatorFactory; @@ -51,7 +42,6 @@ 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.io.Closer; -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; @@ -82,7 +72,6 @@ import javax.annotation.Nullable; import java.io.Closeable; -import java.io.IOException; import java.util.ArrayList; import java.util.BitSet; import java.util.Comparator; @@ -108,6 +97,7 @@ public class GroupByQueryQueryToolChest extends QueryToolChest queryConfigSupplier, GroupByQueryMetricsFactory queryMetricsFactory, @Merging GroupByResourcesReservationPool groupByResourcesReservationPool ) { this.groupingEngine = groupingEngine; + this.queryConfig = queryConfigSupplier.get(); this.queryMetricsFactory = queryMetricsFactory; this.groupByResourcesReservationPool = groupByResourcesReservationPool; } @@ -450,96 +443,7 @@ public TypeReference getResultTypeReference() @Override public ObjectMapper decorateObjectMapper(final ObjectMapper objectMapper, final GroupByQuery query) { - final boolean resultAsArray = query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false); - - // Serializer that writes array- or map-based rows as appropriate, based on the "resultAsArray" setting. - final JsonSerializer serializer = new JsonSerializer() - { - @Override - public void serialize( - final ResultRow resultRow, - final JsonGenerator jg, - final SerializerProvider serializers - ) throws IOException - { - if (resultAsArray) { - JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, resultRow.getArray()); - } else { - JacksonUtils.writeObjectUsingSerializerProvider(jg, serializers, resultRow.toMapBasedRow(query)); - } - } - }; - - // Deserializer that can deserialize either array- or map-based rows. - final JsonDeserializer deserializer = new JsonDeserializer() - { - final Class[] dimensionClasses = createDimensionClasses(query); - boolean containsComplexDimensions = query.getDimensions() - .stream() - .anyMatch( - dimensionSpec -> dimensionSpec.getOutputType().is(ValueType.COMPLEX) - ); - - @Override - public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException - { - if (jp.isExpectedStartObjectToken()) { - final Row row = jp.readValueAs(Row.class); - final ResultRow resultRow = ResultRow.fromLegacyRow(row, query); - if (containsComplexDimensions) { - final List queryDimensions = query.getDimensions(); - for (int i = 0; i < queryDimensions.size(); ++i) { - if (queryDimensions.get(i).getOutputType().is(ValueType.COMPLEX)) { - final int dimensionIndexInResultRow = query.getResultRowDimensionStart() + i; - resultRow.set( - dimensionIndexInResultRow, - objectMapper.convertValue( - resultRow.get(dimensionIndexInResultRow), - dimensionClasses[i] - ) - ); - } - } - } - return resultRow; - } else { - Object[] objectArray = new Object[query.getResultRowSizeWithPostAggregators()]; - - if (!jp.isExpectedStartArrayToken()) { - throw DruidException.defensive("Expected start token, received [%s]", jp.currentToken()); - } - - ObjectCodec codec = jp.getCodec(); - - jp.nextToken(); - - int numObjects = 0; - while (jp.currentToken() != JsonToken.END_ARRAY) { - if (numObjects >= query.getResultRowDimensionStart() && numObjects < query.getResultRowAggregatorStart()) { - objectArray[numObjects] = codec.readValue(jp, dimensionClasses[numObjects - query.getResultRowDimensionStart()]); - } else { - objectArray[numObjects] = codec.readValue(jp, Object.class); - } - jp.nextToken(); - ++numObjects; - } - return ResultRow.of(objectArray); - } - } - }; - - class GroupByResultRowModule extends SimpleModule - { - private GroupByResultRowModule() - { - addSerializer(ResultRow.class, serializer); - addDeserializer(ResultRow.class, deserializer); - } - } - - final ObjectMapper newObjectMapper = objectMapper.copy(); - newObjectMapper.registerModule(new GroupByResultRowModule()); - return newObjectMapper; + return ResultRowObjectMapperDecoratorUtil.decorateObjectMapper(objectMapper, query, queryConfig); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/groupby/ResultRowObjectMapperDecoratorUtil.java b/processing/src/main/java/org/apache/druid/query/groupby/ResultRowObjectMapperDecoratorUtil.java new file mode 100644 index 000000000000..60b55ce74828 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/groupby/ResultRowObjectMapperDecoratorUtil.java @@ -0,0 +1,267 @@ +/* + * 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.groupby; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.core.JsonToken; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.databind.type.TypeFactory; +import org.apache.druid.data.input.Row; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.NullableTypeStrategy; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; +import java.io.IOException; +import java.util.List; + +/** + * Utility class for conditional serde of {@link ResultRow} objects. Depending on the query configuration and the query + * dimensions, this class chooses an optimally performant method for serdeing the result rows while also preserving the + * dimension classes. + * Any modification this class must be benchmarked properly as it runs in a hot-loop and can have significant impact on + * long-running queries. See {@code GroupByDeserializationBenchmark} for existing benchmarks + */ +public class ResultRowObjectMapperDecoratorUtil +{ + /** + * Decorates the provided object mapper so that it can read the result rows generated by the given query and the + * groupByQueryConfig. It never modifies the provided object mapper. It can either return the same mapper undecorated, + * or clones the object mapper before decorating it. + */ + public static ObjectMapper decorateObjectMapper( + final ObjectMapper baseObjectMapper, + final GroupByQuery query, + final GroupByQueryConfig groupByQueryConfig + ) + { + final JsonDeserializer deserializer = getDeserializer(baseObjectMapper, query, groupByQueryConfig); + final JsonSerializer serializer = getSerializer(query, groupByQueryConfig); + if (deserializer == null && serializer == null) { + return baseObjectMapper; + } + + final ObjectMapper decoratedObjectMapper = baseObjectMapper.copy(); + class GroupByResultRowModule extends SimpleModule + { + private GroupByResultRowModule() + { + if (serializer != null) { + addSerializer(ResultRow.class, serializer); + } + if (deserializer != null) { + addDeserializer(ResultRow.class, deserializer); + } + } + } + decoratedObjectMapper.registerModule(new GroupByResultRowModule()); + return decoratedObjectMapper; + } + + /** + * Returns a deserializer required to for the result rows of the provided query. It returns null if no special + * deserialization is required, and type-unaware generic java objects are sufficient. + */ + @Nullable + private static JsonDeserializer getDeserializer( + final ObjectMapper objectMapper, + final GroupByQuery query, + final GroupByQueryConfig groupByQueryConfig + ) + { + final boolean resultAsArray = query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false); + final boolean intermediateCompatMode = groupByQueryConfig.isIntermediateResultAsMapCompat(); + final boolean arrayBasedRows = resultAsArray && !intermediateCompatMode; + final boolean dimensionsRequireConversion = query.getDimensions() + .stream() + .anyMatch( + dimensionSpec -> dimensionRequiresConversion(dimensionSpec.getOutputType()) + ); + + // Most common case - when array based rows are used, and grouping is done on primitive/array/json types + if (arrayBasedRows && !dimensionsRequireConversion) { + // We can assume ResultRow are serialized and deserialized as arrays. No need for special decoration, + // and we can save the overhead of making a copy of the ObjectMapper + return null; + } else if (!arrayBasedRows && !dimensionsRequireConversion) { + // We have to deserialize map based rows, however we don't have to deserialize the dimensions individually + // Returns a deserializer that can deserialize both map and array based rows simultaneously + return new JsonDeserializer() + { + @Override + public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException + { + if (jp.isExpectedStartObjectToken()) { + final Row row = jp.readValueAs(Row.class); + return ResultRow.fromLegacyRow(row, query); + } else { + return ResultRow.of(jp.readValueAs(Object[].class)); + } + } + }; + + } else { + // Dimensions need to be serialized individually because some of them require conversion to specialized types + return new JsonDeserializer() + { + final JavaType[] javaTypes = createJavaTypesForResultRow(query); + + @Override + public ResultRow deserialize(final JsonParser jp, final DeserializationContext ctxt) throws IOException + { + if (jp.isExpectedStartObjectToken()) { + final Row row = jp.readValueAs(Row.class); + final ResultRow resultRow = ResultRow.fromLegacyRow(row, query); + + final List queryDimensions = query.getDimensions(); + for (int i = 0; i < queryDimensions.size(); ++i) { + if (dimensionRequiresConversion(queryDimensions.get(i).getOutputType())) { + final int dimensionIndexInResultRow = query.getResultRowDimensionStart() + i; + resultRow.set( + dimensionIndexInResultRow, + objectMapper.convertValue( + resultRow.get(dimensionIndexInResultRow), + javaTypes[dimensionIndexInResultRow] + ) + ); + } + } + + return resultRow; + } else { + if (!jp.isExpectedStartArrayToken()) { + throw DruidException.defensive("Expected start token, received [%s]", jp.currentToken()); + } + + Object[] objectArray = new Object[query.getResultRowSizeWithPostAggregators()]; + int index = 0; + + while (jp.nextToken() != JsonToken.END_ARRAY) { + objectArray[index] = JacksonUtils.readObjectUsingDeserializationContext(jp, ctxt, javaTypes[index]); + ++index; + } + + return ResultRow.of(objectArray); + } + } + }; + } + } + + /** + * Returns a legacy mode aware serialiazer that serializes the result rows as arrays or maps depending on the query + * configuration + */ + @Nullable + private static JsonSerializer getSerializer( + final GroupByQuery query, + final GroupByQueryConfig groupByQueryConfig + ) + { + final boolean resultAsArray = query.context().getBoolean(GroupByQueryConfig.CTX_KEY_ARRAY_RESULT_ROWS, false); + final boolean intermediateCompatMode = groupByQueryConfig.isIntermediateResultAsMapCompat(); + final boolean arrayBasedRows = resultAsArray && !intermediateCompatMode; + if (arrayBasedRows) { + return null; + } else { + if (resultAsArray) { + return new JsonSerializer() + { + @Override + public void serialize(ResultRow resultRow, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) + throws IOException + { + JacksonUtils.writeObjectUsingSerializerProvider(jsonGenerator, serializerProvider, resultRow.getArray()); + } + }; + + } else { + return new JsonSerializer() + { + @Override + public void serialize(ResultRow resultRow, JsonGenerator jsonGenerator, SerializerProvider serializerProvider) + throws IOException + { + JacksonUtils.writeObjectUsingSerializerProvider( + jsonGenerator, + serializerProvider, + resultRow.toMapBasedRow(query) + ); + } + }; + } + } + } + + /** + * Returns true if the dimension needs to be converted from generic Java objects to the specialized column type. It involves all + * complex types, except for JSON types. JSON types are special in a way that they can work with the generic java objects + * without any conversion + */ + private static boolean dimensionRequiresConversion(final ColumnType dimensionType) + { + return dimensionType.is(ValueType.COMPLEX) && !ColumnType.NESTED_DATA.equals(dimensionType); + } + + /** + * Creates java types for deserializing the result row. For timestamp, aggregators and post-aggregators, it resorts to + * {@code Object.class}. For dimensions requiring conversion (check {@link #dimensionRequiresConversion(ColumnType)}), + * it returns the java type for the associated class of the complex object. + */ + private static JavaType[] createJavaTypesForResultRow(final GroupByQuery groupByQuery) + { + final TypeFactory typeFactory = TypeFactory.defaultInstance(); + final JavaType[] javaTypes = new JavaType[groupByQuery.getResultRowSizeWithPostAggregators()]; + final List dimensions = groupByQuery.getDimensions(); + for (int i = 0; i < groupByQuery.getResultRowSizeWithPostAggregators(); ++i) { + if (i >= groupByQuery.getResultRowDimensionStart() && i < groupByQuery.getResultRowAggregatorStart()) { + DimensionSpec dimension = dimensions.get(i - groupByQuery.getResultRowDimensionStart()); + ColumnType dimensionType = dimensions.get(i - groupByQuery.getResultRowDimensionStart()).getOutputType(); + if (dimensionType.is(ValueType.COMPLEX)) { + //noinspection rawtypes + NullableTypeStrategy nullableTypeStrategy = dimensionType.getNullableStrategy(); + if (!nullableTypeStrategy.groupable()) { + throw DruidException.defensive( + "Ungroupable dimension [%s] with type [%s] found in the query.", + dimension, + dimensionType + ); + } + javaTypes[i] = typeFactory.constructType(nullableTypeStrategy.getClazz()); + } else { + javaTypes[i] = typeFactory.constructType(Object.class); + } + } else { + javaTypes[i] = typeFactory.constructType(Object.class); + } + } + return javaTypes; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index da8a0e046230..0e73d5db6f48 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonValue; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonToken; -import com.fasterxml.jackson.core.ObjectCodec; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.ObjectMapper; @@ -45,6 +44,7 @@ import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.guava.Accumulator; import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.ColumnSelectorPlus; import org.apache.druid.query.DimensionComparisonUtils; @@ -1378,7 +1378,6 @@ public RowBasedKey deserialize( } jp.nextToken(); - final ObjectCodec codec = jp.getCodec(); final int timestampAdjustment = includeTimestamp ? 1 : 0; final int dimsToRead = timestampAdjustment + serdeHelpers.length; int dimsReadSoFar = 0; @@ -1389,15 +1388,19 @@ public RowBasedKey deserialize( jp.currentToken() != JsonToken.END_ARRAY, "Unexpected end of array when deserializing timestamp from the spilled files" ); - objects[dimsReadSoFar] = codec.readValue(jp, Long.class); + objects[dimsReadSoFar] = JacksonUtils.readObjectUsingDeserializationContext(jp, deserializationContext, Long.class); ++dimsReadSoFar; jp.nextToken(); } while (jp.currentToken() != JsonToken.END_ARRAY) { - objects[dimsReadSoFar] = - codec.readValue(jp, serdeHelpers[dimsReadSoFar - timestampAdjustment].getClazz()); + objects[dimsReadSoFar] = JacksonUtils.readObjectUsingDeserializationContext( + jp, + deserializationContext, + serdeHelpers[dimsReadSoFar - timestampAdjustment].getClazz() + ); + ++dimsReadSoFar; jp.nextToken(); diff --git a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java index fca50c25b282..2dd827d323e1 100644 --- a/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java +++ b/processing/src/main/java/org/apache/druid/query/operator/window/WindowFrame.java @@ -21,159 +21,192 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.query.operator.ColumnWithDirection; +import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; + +import javax.annotation.Nullable; -import java.util.Collections; import java.util.List; import java.util.Objects; -import java.util.stream.Collectors; -public class WindowFrame +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "rows", value = WindowFrame.Rows.class), + @JsonSubTypes.Type(name = "groups", value = WindowFrame.Groups.class), +}) +@SubclassesMustOverrideEqualsAndHashCode +public interface WindowFrame { - public static WindowFrame unbounded() + static WindowFrame unbounded() { - return new WindowFrame(PeerType.ROWS, true, 0, true, 0, null); + return rows(null, null); } - @SuppressWarnings("unused") - public enum PeerType + static Rows rows(Integer lowerOffset, Integer upperOffset) { - ROWS, - RANGE + return new WindowFrame.Rows(lowerOffset, upperOffset); } - // Will likely need to add the order by columns to also be able to deal with RANGE peer type. - private final PeerType peerType; - private final boolean lowerUnbounded; - private final int lowerOffset; - private final boolean upperUnbounded; - private final int upperOffset; - private final List orderBy; - - @JsonCreator - public WindowFrame( - @JsonProperty("peerType") PeerType peerType, - @JsonProperty("lowUnbounded") boolean lowerUnbounded, - @JsonProperty("lowOffset") int lowerOffset, - @JsonProperty("uppUnbounded") boolean upperUnbounded, - @JsonProperty("uppOffset") int upperOffset, - @JsonProperty("orderBy") List orderBy - ) + static Groups groups(Integer lowerOffset, Integer upperOffset, List orderByColumns) { - this.peerType = peerType; - this.lowerUnbounded = lowerUnbounded; - this.lowerOffset = lowerOffset; - this.upperUnbounded = upperUnbounded; - this.upperOffset = upperOffset; - this.orderBy = orderBy; + return new WindowFrame.Groups(lowerOffset, upperOffset, orderByColumns); } - @JsonProperty("peerType") - public PeerType getPeerType() + static WindowFrame forOrderBy(String... orderByColumns) { - return peerType; + return groups(null, 0, Lists.newArrayList(orderByColumns)); } - @JsonProperty("lowUnbounded") - public boolean isLowerUnbounded() + abstract class OffsetFrame implements WindowFrame { - return lowerUnbounded; - } + @JsonProperty + public final Integer lowerOffset; + @JsonProperty + public final Integer upperOffset; - @JsonProperty("lowOffset") - public int getLowerOffset() - { - return lowerOffset; - } + @JsonCreator + public OffsetFrame( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + this.lowerOffset = lowerOffset; + this.upperOffset = upperOffset; + } - @JsonProperty("uppUnbounded") - public boolean isUpperUnbounded() - { - return upperUnbounded; - } + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, upperOffset); + } - @JsonProperty("uppOffset") - public int getUpperOffset() - { - return upperOffset; - } + /** + * Calculates the applicable lower offset if the max number of rows is + * known. + */ + public int getLowerOffsetClamped(int maxRows) + { + if (lowerOffset == null) { + return -maxRows; + } + return Math.max(-maxRows, lowerOffset); + } - @JsonProperty("orderBy") - public List getOrderBy() - { - return orderBy; + /** + * Calculates the applicable upper offset if the max number of rows is + * known. + */ + public int getUpperOffsetClamped(int maxRows) + { + if (upperOffset == null) { + return maxRows; + } + return Math.min(maxRows, upperOffset); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + OffsetFrame other = (OffsetFrame) obj; + return Objects.equals(lowerOffset, other.lowerOffset) && Objects.equals(upperOffset, other.upperOffset); + } + + @Override + public abstract String toString(); } - @Override - public boolean equals(Object o) + class Rows extends OffsetFrame { - if (this == o) { - return true; + @JsonCreator + public Rows( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset) + { + super(lowerOffset, upperOffset); } - if (!(o instanceof WindowFrame)) { - return false; + + @Override + public String toString() + { + return "WindowFrame.Rows [" + + "lowerOffset=" + lowerOffset + + ", upperOffset=" + upperOffset + + "]"; } - WindowFrame that = (WindowFrame) o; - return lowerUnbounded == that.lowerUnbounded - && lowerOffset == that.lowerOffset - && upperUnbounded == that.upperUnbounded - && upperOffset == that.upperOffset - && peerType == that.peerType - && Objects.equals(orderBy, that.orderBy); } - @Override - public int hashCode() + class Groups extends OffsetFrame { - return Objects.hash(peerType, lowerUnbounded, lowerOffset, upperUnbounded, upperOffset, orderBy); - } + @JsonProperty + private final ImmutableList orderByColumns; - @Override - public String toString() - { - return "WindowFrame{" + - "peerType=" + peerType + - ", lowerUnbounded=" + lowerUnbounded + - ", lowerOffset=" + lowerOffset + - ", upperUnbounded=" + upperUnbounded + - ", upperOffset=" + upperOffset + - ", orderBy=" + orderBy + - '}'; - } + @JsonCreator + public Groups( + @JsonProperty("lowerOffset") Integer lowerOffset, + @JsonProperty("upperOffset") Integer upperOffset, + @JsonProperty("orderByColumns") List orderByColumns) + { + super(lowerOffset, upperOffset); + this.orderByColumns = ImmutableList.copyOf(orderByColumns); + } - public static WindowFrame forOrderBy(ColumnWithDirection... orderBy) - { - return new WindowFrame(PeerType.RANGE, true, 0, false, 0, Lists.newArrayList(orderBy)); - } + public List getOrderByColumns() + { + return orderByColumns; + } - public List getOrderByColNames() - { - if (orderBy == null) { - return Collections.emptyList(); + @Override + public int hashCode() + { + return Objects.hash(lowerOffset, orderByColumns, upperOffset); } - return orderBy.stream().map(ColumnWithDirection::getColumn).collect(Collectors.toList()); - } - /** - * Calculates the applicable lower offset if the max number of rows is known. - */ - public int getLowerOffsetClamped(int maxRows) - { - if (lowerUnbounded) { - return -maxRows; + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Groups other = (Groups) obj; + return Objects.equals(lowerOffset, other.lowerOffset) + && Objects.equals(orderByColumns, other.orderByColumns) + && Objects.equals(upperOffset, other.upperOffset); + } + + @Override + public String toString() + { + return "WindowFrame.Groups [" + + "lowerOffset=" + lowerOffset + + ", upperOffset=" + upperOffset + + ", orderByColumns=" + orderByColumns + "]"; } - return Math.max(-maxRows, lowerOffset); } - /** - * Calculates the applicable upper offset if the max number of rows is known. - */ - public int getUpperOffsetClamped(int maxRows) + @SuppressWarnings("unchecked") + @Nullable + default T unwrap(Class clazz) { - if (upperUnbounded) { - return maxRows; + if (clazz.isInstance(this)) { + return (T) this; } - return Math.min(maxRows, upperOffset); + return null; } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java index 839528730506..7130fafd8671 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java @@ -21,6 +21,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.aggregation.Aggregator; @@ -28,6 +29,9 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.operator.window.WindowFrame; +import org.apache.druid.query.operator.window.WindowFrame.Groups; +import org.apache.druid.query.operator.window.WindowFrame.OffsetFrame; +import org.apache.druid.query.operator.window.WindowFrame.Rows; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.ColumnSelectorFactory; @@ -106,22 +110,38 @@ public void appendTo(AppendableRowsAndColumns rac) public static Iterable buildIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) { int numRows = rac.numRows(); - if (frame.getLowerOffsetClamped(numRows) == -numRows && frame.getUpperOffsetClamped(numRows) == numRows) { - return buildUnboundedIteratorFor(rac, frame); - } else if (frame.getPeerType() == WindowFrame.PeerType.RANGE) { - return buildGroupIteratorFor(rac, frame); - } else { - return buildRowIteratorFor(rac, frame); + if (isEffectivelyUnbounded(frame, numRows)) { + return buildUnboundedIteratorFor(rac); } + Rows rowsFrame = frame.unwrap(WindowFrame.Rows.class); + if (rowsFrame != null) { + return buildRowIteratorFor(rac, rowsFrame); + } + Groups groupsFrame = frame.unwrap(WindowFrame.Groups.class); + if (groupsFrame != null) { + return buildGroupIteratorFor(rac, groupsFrame); + } + throw DruidException.defensive("Unable to handle WindowFrame [%s]!", frame); } - private static Iterable buildUnboundedIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static boolean isEffectivelyUnbounded(WindowFrame frame, int numRows) { - int[] groupBoundaries = new int[]{0, rac.numRows()}; - return new GroupIteratorForWindowFrame(frame, groupBoundaries); + OffsetFrame offsetFrame = frame.unwrap(WindowFrame.OffsetFrame.class); + if (offsetFrame.getLowerOffsetClamped(numRows) == -numRows + && offsetFrame.getUpperOffsetClamped(numRows) == numRows) { + // regardless the actual mode; all rows will be inside the frame! + return true; + } + return false; + } + + private static Iterable buildUnboundedIteratorFor(AppendableRowsAndColumns rac) + { + int[] groupBoundaries = new int[] {0, rac.numRows()}; + return new GroupIteratorForWindowFrame(WindowFrame.rows(null, null), groupBoundaries); } - private static Iterable buildRowIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static Iterable buildRowIteratorFor(AppendableRowsAndColumns rac, WindowFrame.Rows frame) { int[] groupBoundaries = new int[rac.numRows() + 1]; for (int j = 0; j < groupBoundaries.length; j++) { @@ -130,9 +150,9 @@ private static Iterable buildRowIteratorFor(AppendableRowsAndColumn return new GroupIteratorForWindowFrame(frame, groupBoundaries); } - private static Iterable buildGroupIteratorFor(AppendableRowsAndColumns rac, WindowFrame frame) + private static Iterable buildGroupIteratorFor(AppendableRowsAndColumns rac, WindowFrame.Groups frame) { - int[] groupBoundaries = ClusteredGroupPartitioner.fromRAC(rac).computeBoundaries(frame.getOrderByColNames()); + int[] groupBoundaries = ClusteredGroupPartitioner.fromRAC(rac).computeBoundaries(frame.getOrderByColumns()); return new GroupIteratorForWindowFrame(frame, groupBoundaries); } @@ -145,7 +165,7 @@ static class GroupIteratorForWindowFrame implements Iterable // upper exclusive private final int upperOffset; - public GroupIteratorForWindowFrame(WindowFrame frame, int[] groupBoundaries) + public GroupIteratorForWindowFrame(WindowFrame.OffsetFrame frame, int[] groupBoundaries) { this.groupBoundaries = groupBoundaries; numGroups = groupBoundaries.length - 1; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 50841fefbd9a..3d8621c4643e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -448,6 +448,18 @@ public List getColumnTypes() return columnTypes; } + /** + * Prior to PR https://github.com/apache/druid/pull/16659 (Druid 31) data servers require + * the "legacy" parameter to be set to a non-null value. For compatibility with older data + * servers during rolling updates, we need to write out "false". + */ + @Deprecated + @JsonProperty("legacy") + public Boolean isLegacy() + { + return false; + } + @Override public Ordering getResultOrdering() { diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java index 6275a8fea9fd..4974c0f8833a 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredStorageAdapter.java @@ -134,4 +134,10 @@ public Metadata getMetadata() { return baseStorageAdapter.getMetadata(); } + + @Override + public boolean isFromTombstone() + { + return baseStorageAdapter.isFromTombstone(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java index 59923c81db54..eba801e224e2 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestStorageAdapter.java @@ -254,6 +254,12 @@ public Metadata getMetadata() return baseAdapter.getMetadata(); } + @Override + public boolean isFromTombstone() + { + return baseAdapter.isFromTombstone(); + } + public VirtualColumn getUnnestColumn() { return unnestColumn; diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java index ee8a70aea84c..a478169633b6 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinEngine.java @@ -169,7 +169,7 @@ public ColumnSelectorFactory getColumnSelectorFactory() @Override public void advance() { - advanceUninterruptibly(); + advance(true); BaseQuery.checkInterrupted(); } @@ -188,6 +188,11 @@ private void matchCurrentPosition() @Override public void advanceUninterruptibly() + { + advance(false); + } + + private void advance(boolean interruptibly) { joinColumnSelectorFactory.advanceRowId(); @@ -209,7 +214,11 @@ public void advanceUninterruptibly() do { // No more right-hand side matches; advance the left-hand side. - leftCursor.advanceUninterruptibly(); + if (interruptibly) { + leftCursor.advance(); + } else { + leftCursor.advanceUninterruptibly(); + } // Update joinMatcher state to match new cursor position. matchCurrentPosition(); diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 547fb7be28f1..4319fdc52b57 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -227,6 +227,7 @@ public CursorMaker asCursorMaker(CursorBuildSpec spec) final Filter combinedFilter = baseFilterAnd(spec.getFilter()); + if (clauses.isEmpty()) { // HashJoinEngine isn't vectorized yet. // However, we can still vectorize if there are no clauses, since that means all we need to do is apply @@ -326,6 +327,12 @@ public VectorCursor makeVectorCursor() }; } + @Override + public boolean isFromTombstone() + { + return baseAdapter.isFromTombstone(); + } + /** * Returns whether "column" will be selected from "baseAdapter". This is true if it is not shadowed by any joinables * (i.e. if it does not start with any of their prefixes). diff --git a/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java b/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java index ca34c364dca8..5b76afb90229 100644 --- a/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java +++ b/processing/src/test/java/org/apache/druid/java/util/common/guava/ParallelMergeCombiningSequenceTest.java @@ -143,7 +143,7 @@ public void testOrderedResultBatchFromSequenceBacktoYielderOnSequence() throws I if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.terminal()); rawYielder.close(); cursor.close(); @@ -211,16 +211,18 @@ public void testOrderedResultBatchFromSequenceToBlockingQueueCursor() throws IOE if (!currentBatch.isDrained()) { outputQueue.offer(currentBatch); } - outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.TERMINAL); + outputQueue.offer(ParallelMergeCombiningSequence.ResultBatch.terminal()); rawYielder.close(); cursor.close(); rawYielder = Yielders.each(rawSequence); + ParallelMergeCombiningSequence.CancellationGizmo gizmo = new ParallelMergeCombiningSequence.CancellationGizmo(); ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor queueCursor = new ParallelMergeCombiningSequence.BlockingQueueuBatchedResultsCursor<>( outputQueue, + gizmo, INT_PAIR_ORDERING, false, -1L @@ -551,14 +553,14 @@ public void testTimeoutExceptionDueToStalledInput() } @Test - public void testTimeoutExceptionDueToStalledReader() + public void testTimeoutExceptionDueToSlowReader() { - final int someSize = 2048; + final int someSize = 50_000; List> input = new ArrayList<>(); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); - input.add(nonBlockingSequence(someSize)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); Throwable t = Assert.assertThrows(QueryTimeoutException.class, () -> assertException(input, 8, 64, 1000, 1500)); Assert.assertEquals("Query did not complete within configured timeout period. " + @@ -567,6 +569,110 @@ public void testTimeoutExceptionDueToStalledReader() Assert.assertTrue(pool.isQuiescent()); } + @Test + public void testTimeoutExceptionDueToStoppedReader() throws InterruptedException + { + final int someSize = 150_000; + List reporters = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + List> input = new ArrayList<>(); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + + TestingReporter reporter = new TestingReporter(); + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + input, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, + true, + 1000, + 0, + TEST_POOL_SIZE, + 512, + 128, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS, + reporter + ); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + reporter.future = parallelMergeCombineSequence.getCancellationFuture(); + reporter.yielder = parallelMergeCombineYielder; + reporter.yielder = parallelMergeCombineYielder.next(null); + Assert.assertFalse(parallelMergeCombineYielder.isDone()); + reporters.add(reporter); + } + + // sleep until timeout + Thread.sleep(1000); + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + Assert.assertFalse(pool.hasQueuedSubmissions()); + for (TestingReporter reporter : reporters) { + Assert.assertThrows(QueryTimeoutException.class, () -> reporter.yielder.next(null)); + Assert.assertTrue(reporter.future.isCancelled()); + Assert.assertTrue(reporter.future.getCancellationGizmo().isCanceled()); + } + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + } + + @Test + public void testManyBigSequencesAllAtOnce() throws IOException + { + final int someSize = 50_000; + List reporters = new ArrayList<>(); + for (int i = 0; i < 100; i++) { + List> input = new ArrayList<>(); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + input.add(nonBlockingSequence(someSize, true)); + + TestingReporter reporter = new TestingReporter(); + final ParallelMergeCombiningSequence parallelMergeCombineSequence = new ParallelMergeCombiningSequence<>( + pool, + input, + INT_PAIR_ORDERING, + INT_PAIR_MERGE_FN, + true, + 30 * 1000, + 0, + TEST_POOL_SIZE, + 512, + 128, + ParallelMergeCombiningSequence.DEFAULT_TASK_TARGET_RUN_TIME_MILLIS, + reporter + ); + Yielder parallelMergeCombineYielder = Yielders.each(parallelMergeCombineSequence); + reporter.future = parallelMergeCombineSequence.getCancellationFuture(); + reporter.yielder = parallelMergeCombineYielder; + parallelMergeCombineYielder.next(null); + Assert.assertFalse(parallelMergeCombineYielder.isDone()); + reporters.add(reporter); + } + + for (TestingReporter testingReporter : reporters) { + Yielder parallelMergeCombineYielder = testingReporter.yielder; + while (!parallelMergeCombineYielder.isDone()) { + parallelMergeCombineYielder = parallelMergeCombineYielder.next(parallelMergeCombineYielder.get()); + } + Assert.assertTrue(parallelMergeCombineYielder.isDone()); + parallelMergeCombineYielder.close(); + Assert.assertTrue(testingReporter.future.isDone()); + } + + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); + Assert.assertTrue(pool.isQuiescent()); + Assert.assertEquals(0, pool.getRunningThreadCount()); + Assert.assertFalse(pool.hasQueuedSubmissions()); + Assert.assertEquals(0, pool.getActiveThreadCount()); + for (TestingReporter reporter : reporters) { + Assert.assertTrue(reporter.done); + } + } + @Test public void testGracefulCloseOfYielderCancelsPool() throws IOException { @@ -666,7 +772,9 @@ private void assertResultWithCustomPool( parallelMergeCombineYielder.close(); // cancellation trigger should not be set if sequence was fully yielded and close is called // (though shouldn't actually matter even if it was...) - Assert.assertFalse(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isDone()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); } private void assertResult( @@ -713,13 +821,15 @@ private void assertResult( Assert.assertTrue(combiningYielder.isDone()); Assert.assertTrue(parallelMergeCombineYielder.isDone()); - Assert.assertTrue(pool.awaitQuiescence(1, TimeUnit.SECONDS)); + Assert.assertTrue(pool.awaitQuiescence(5, TimeUnit.SECONDS)); Assert.assertTrue(pool.isQuiescent()); combiningYielder.close(); parallelMergeCombineYielder.close(); // cancellation trigger should not be set if sequence was fully yielded and close is called // (though shouldn't actually matter even if it was...) - Assert.assertFalse(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); + Assert.assertFalse(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isDone()); } private void assertResultWithEarlyClose( @@ -773,20 +883,21 @@ private void assertResultWithEarlyClose( } } // trying to next the yielder creates sadness for you - final String expectedExceptionMsg = "Already closed"; + final String expectedExceptionMsg = "Sequence canceled"; Assert.assertEquals(combiningYielder.get(), parallelMergeCombineYielder.get()); final Yielder finalYielder = parallelMergeCombineYielder; Throwable t = Assert.assertThrows(RuntimeException.class, () -> finalYielder.next(finalYielder.get())); Assert.assertEquals(expectedExceptionMsg, t.getMessage()); // cancellation gizmo of sequence should be cancelled, and also should contain our expected message - Assert.assertTrue(parallelMergeCombineSequence.getCancellationGizmo().isCancelled()); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().isCanceled()); Assert.assertEquals( expectedExceptionMsg, - parallelMergeCombineSequence.getCancellationGizmo().getRuntimeException().getMessage() + parallelMergeCombineSequence.getCancellationFuture().getCancellationGizmo().getRuntimeException().getMessage() ); + Assert.assertTrue(parallelMergeCombineSequence.getCancellationFuture().isCancelled()); - Assert.assertTrue(pool.awaitQuiescence(1, TimeUnit.SECONDS)); + Assert.assertTrue(pool.awaitQuiescence(10, TimeUnit.SECONDS)); Assert.assertTrue(pool.isQuiescent()); Assert.assertFalse(combiningYielder.isDone()); @@ -1082,4 +1193,19 @@ private static IntPair makeIntPair(int mergeKey) { return new IntPair(mergeKey, ThreadLocalRandom.current().nextInt(1, 100)); } + + static class TestingReporter implements Consumer + { + ParallelMergeCombiningSequence.CancellationFuture future; + Yielder yielder; + volatile ParallelMergeCombiningSequence.MergeCombineMetrics metrics; + volatile boolean done = false; + + @Override + public void accept(ParallelMergeCombiningSequence.MergeCombineMetrics mergeCombineMetrics) + { + metrics = mergeCombineMetrics; + done = true; + } + } } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java index e0dc85a4450d..21c8aeacc254 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMaxAggregationTest.java @@ -63,7 +63,7 @@ public void setup() selector = new TestDoubleColumnSelectorImpl(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); VectorValueSelector vectorValueSelector = EasyMock.createMock(VectorValueSelector.class); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java index 5b4041b84225..40f72719f7ff 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/DoubleMinAggregationTest.java @@ -63,7 +63,7 @@ public void setup() selector = new TestDoubleColumnSelectorImpl(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java index d063658d76a2..535efb02faf7 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMaxAggregationTest.java @@ -62,7 +62,7 @@ public void setup() selector = new TestLongColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java index f651ce074b93..129f81d245a5 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/LongMinAggregationTest.java @@ -62,7 +62,7 @@ public void setup() selector = new TestLongColumnSelector(values); colSelectorFactory = EasyMock.createMock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector("nilly")).andReturn(selector); - EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("nilly")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); VectorValueSelector vectorValueSelector = EasyMock.createMock(VectorValueSelector.class); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java index 7279ca938bd8..d9aefd5f55e2 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChestTest.java @@ -792,7 +792,19 @@ public void testResultSerdeIntermediateResultAsMapCompat() throws Exception .setGranularity(QueryRunnerTestHelper.DAY_GRAN) .build(); - final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(null, null, null); + final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( + null, + () -> new GroupByQueryConfig() + { + @Override + public boolean isIntermediateResultAsMapCompat() + { + return true; + } + }, + null, + null + ); final ObjectMapper objectMapper = TestHelper.makeJsonMapper(); final ObjectMapper arraysObjectMapper = toolChest.decorateObjectMapper( diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java index a5dbb49bca51..475848edbdc7 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByQueryRunnerTest.java @@ -363,6 +363,7 @@ public static GroupByQueryRunnerFactory makeQueryRunnerFactory( ); final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest( groupingEngine, + () -> config, DefaultGroupByQueryMetricsFactory.instance(), groupByResourcesReservationPool ); diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java new file mode 100644 index 000000000000..855f4694f430 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFrameTest.java @@ -0,0 +1,70 @@ +/* + * 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.operator.window; + +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.query.operator.window.WindowFrame.OffsetFrame; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class WindowFrameTest +{ + @Test + public void testEqualsRows() + { + EqualsVerifier.forClass(WindowFrame.Rows.class) + .usingGetClass() + .verify(); + } + + @Test + public void testEqualsGroups() + { + EqualsVerifier.forClass(WindowFrame.Groups.class) + .usingGetClass() + .verify(); + } + + @Test + public void testOffsetFrameUnbounded() + { + OffsetFrame of = new WindowFrame.Rows(null, null); + assertEquals(-100, of.getLowerOffsetClamped(100)); + assertEquals(100, of.getUpperOffsetClamped(100)); + } + + @Test + public void testOffsetFrameNormal() + { + OffsetFrame of = new WindowFrame.Rows(-1, 2); + assertEquals(-1, of.getLowerOffsetClamped(100)); + assertEquals(2, of.getUpperOffsetClamped(100)); + } + + @Test + public void testOffsetFrameUnbounded2() + { + OffsetFrame of = new WindowFrame.Rows(-200, 200); + assertEquals(-100, of.getLowerOffsetClamped(100)); + assertEquals(100, of.getUpperOffsetClamped(100)); + } + +} diff --git a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java index 5af321b53c88..9bae78bc2ccf 100644 --- a/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java +++ b/processing/src/test/java/org/apache/druid/query/operator/window/WindowFramedAggregateProcessorTest.java @@ -46,7 +46,7 @@ public class WindowFramedAggregateProcessorTest @Test public void testIsPassThruWhenRACReturnsSemanticInterface() { - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null); + final WindowFrame theFrame = WindowFrame.rows(null, 0); final AggregatorFactory[] theAggs = { new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -78,7 +78,7 @@ public T as(Class clazz) @Test public void testDoesStuffWhenNoSemanticInterfacesAvailable() { - final WindowFrame theFrame = new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null); + final WindowFrame theFrame = WindowFrame.rows(null, 0); final AggregatorFactory[] theAggs = { new LongSumAggregatorFactory("sum", "intCol") }; diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java index d5b11f7a612a..41ceb315a04f 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FramedOnHeapAggregatableTest.java @@ -25,10 +25,8 @@ 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.operator.ColumnWithDirection; import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.operator.window.WindowFrame; -import org.apache.druid.query.operator.window.WindowFrame.PeerType; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; @@ -65,7 +63,7 @@ public void testWindowedAggregationWindowSmallerThanRowsNoOffsets() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 0, null), + WindowFrame.rows(0, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -91,7 +89,7 @@ public void testWindowedAggregationWindowSmallerThanRows() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -1, false, 2, null), + WindowFrame.rows(-1, 2), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -117,7 +115,7 @@ public void testWindowedAggregationWindowSmallerThanRowsOnlyUpper() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 2, null), + WindowFrame.rows(0, 2), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -143,7 +141,7 @@ public void testWindowedAggregationWindowSmallerThanRowsOnlyLower() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -2, false, 0, null), + WindowFrame.rows(-2, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -169,7 +167,7 @@ public void testWindowedAggregationWindowLargerThanRows() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 7, null), + WindowFrame.rows(-5, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -197,7 +195,7 @@ public void testWindowedAggregationLowerLargerThanRows() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 1, null), + WindowFrame.rows(-5, 1), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -225,7 +223,7 @@ public void testWindowedAggregationLowerLargerThanRowsNoUpper() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 0, null), + WindowFrame.rows(-5, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -253,7 +251,7 @@ public void testWindowedAggregationUpperLargerThanRows() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -1, false, 7, null), + WindowFrame.rows(-1, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -281,7 +279,7 @@ public void testWindowedAggregationUpperLargerThanRowsNoLower() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7, null), + WindowFrame.rows(0, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -309,7 +307,7 @@ public void testWindowedAggregationWindowLargerThanRowsOnlyUpper() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, false, 7, null), + WindowFrame.rows(0, 7), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -337,7 +335,7 @@ public void testWindowedAggregationWindowLargerThanRowsOnlyLower() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, -5, false, 0, null), + WindowFrame.rows(-5, 0), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new DoubleMaxAggregatorFactory("maxFromInt", "intCol"), @@ -371,7 +369,7 @@ public void testUnboundedWindowedAggregation() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, true, 0, null), + WindowFrame.unbounded(), new AggregatorFactory[]{ new LongSumAggregatorFactory("sumFromLong", "intCol"), new LongSumAggregatorFactory("sumFromDouble", "doubleCol"), @@ -409,7 +407,7 @@ public void testCumulativeAggregation() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, true, 0, false, 0, null), + WindowFrame.rows(null, 0), new AggregatorFactory[]{ new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -443,7 +441,7 @@ public void testReverseCumulativeAggregation() FramedOnHeapAggregatable agger = FramedOnHeapAggregatable.fromRAC(rac); final RowsAndColumns results = agger.aggregateAll( - new WindowFrame(WindowFrame.PeerType.ROWS, false, 0, true, 0, null), + WindowFrame.rows(0, null), new AggregatorFactory[]{ new LongMaxAggregatorFactory("cummMax", "intCol"), new DoubleSumAggregatorFactory("cummSum", "doubleCol") @@ -465,7 +463,7 @@ public void testReverseCumulativeAggregation() @Test public void testRangeOrderBy() { - WindowFrame frame = WindowFrame.forOrderBy(ColumnWithDirection.ascending("c1")); + WindowFrame frame = WindowFrame.forOrderBy("c1"); int[] c1Vals = new int[] {0, 0, 0, 1, 1, 1, 2, 2, 2, 2}; int[] c2Vals = new int[] {1, 1, 2, 1, 1, 2, 1, 1, 1, 2}; int[] resVals = new int[] {4, 4, 4, 8, 8, 8, 13, 13, 13, 13}; @@ -476,14 +474,7 @@ public void testRangeOrderBy() @Test public void testRangeB1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 0, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 0, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 1, 1, 3, 4, 5}; @@ -495,14 +486,7 @@ public void testRangeB1() @Test public void testRangeA1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - 0, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(0, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 1, 1, 3, 4, 5}; @@ -514,14 +498,7 @@ public void testRangeA1() @Test public void testRangeB1A1() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 1, 2, 3, 4, 5}; int[] c2Vals = new int[] {0, 1, 2, 3, 4, 5}; @@ -534,14 +511,7 @@ public void testRangeB1A1() @Test public void testRangeB1A1_2() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 1, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 1, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 0, 1, 2, 3, 3, 4, 4, 5}; int[] c2Vals = new int[] {0, 0, 1, 2, 2, 1, 2, 2, 5}; @@ -553,14 +523,7 @@ public void testRangeB1A1_2() @Test public void testRangeB1A2() { - WindowFrame frame = new WindowFrame( - PeerType.RANGE, - false, - -1, - false, - 2, - Collections.singletonList(ColumnWithDirection.ascending("c1")) - ); + WindowFrame frame = WindowFrame.groups(-1, 2, Collections.singletonList("c1")); int[] c1Vals = new int[] {0, 0, 0, 1, 1, 1, 2, 2, 2, 2, 3, 3, 3}; int[] c2Vals = new int[] {1, 1, 2, 1, 1, 2, 1, 1, 1, 2, 1, 1, 1}; diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java index 1bc1ff18a58d..e5069743a9ed 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQuerySpecTest.java @@ -47,12 +47,7 @@ public void testSerialization() throws Exception + "\"limit\":3}"; String current = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"columns\":[\"market\",\"quality\",\"index\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"columns\":[\"market\",\"quality\",\"index\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery query = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), @@ -80,13 +75,7 @@ public void testSerialization() throws Exception public void testSerializationWithTimeOrder() throws Exception { String originalJson = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"order\":\"ascending\"," - + "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"order\":\"ascending\",\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery expectedQuery = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), @@ -118,13 +107,7 @@ public void testSerializationWithTimeOrder() throws Exception public void testSerializationWithOrderBy() throws Exception { String originalJson = - "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"}," - + "\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]}," - + "\"resultFormat\":\"list\"," - + "\"limit\":3," - + "\"orderBy\":[{\"columnName\":\"quality\",\"order\":\"ascending\"}]," - + "\"columns\":[\"market\",\"quality\",\"index\",\"__time\"]," - + "\"granularity\":{\"type\":\"all\"}}"; + "{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"testing\"},\"intervals\":{\"type\":\"LegacySegmentSpec\",\"intervals\":[\"2011-01-12T00:00:00.000Z/2011-01-14T00:00:00.000Z\"]},\"resultFormat\":\"list\",\"limit\":3,\"orderBy\":[{\"columnName\":\"quality\",\"order\":\"ascending\"}],\"columns\":[\"market\",\"quality\",\"index\",\"__time\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}"; ScanQuery expectedQuery = new ScanQuery( new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), diff --git a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java index 1804125c0ce6..ce4a417e2bf3 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/ScanQueryTest.java @@ -19,6 +19,8 @@ package org.apache.druid.query.scan; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.DateTimes; @@ -33,6 +35,8 @@ import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.segment.CursorBuildSpec; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.query.Query; +import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -52,6 +56,7 @@ public class ScanQueryTest extends InitializedNullHandlingTest { + private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static QuerySegmentSpec intervalSpec; private static ScanResultValue s1; private static ScanResultValue s2; @@ -101,6 +106,20 @@ public static void setup() ); } + @Test + public void testSerdeAndLegacyBackwardsCompat() throws JsonProcessingException + { + ScanQuery query = Druids.newScanQueryBuilder() + .columns(ImmutableList.of("__time", "quality")) + .dataSource("source") + .intervals(intervalSpec) + .build(); + Assert.assertFalse(query.isLegacy()); + String json = JSON_MAPPER.writeValueAsString(query); + Assert.assertTrue(json.contains("\"legacy\":false")); + Assert.assertEquals(query, JSON_MAPPER.readValue(json, Query.class)); + } + @Test(expected = IllegalArgumentException.class) public void testAscendingScanQueryWithInvalidColumns() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index c6edcf1b7cd6..6c99bd1a39e1 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -341,6 +341,7 @@ public static List readCursor(final CursorMaker cursorMaker, final Lis .collect(Collectors.toList()); final List rows = new ArrayList<>(); + boolean interruptible = false; // test both advance() and advanceUninterruptibly() while (!cursor.isDone()) { final Object[] row = new Object[columns.size()]; @@ -350,7 +351,13 @@ public static List readCursor(final CursorMaker cursorMaker, final Lis } rows.add(row); - cursor.advance(); + if (interruptible) { + cursor.advance(); + } else { + cursor.advanceUninterruptibly(); + } + + interruptible = !interruptible; } return rows; diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 5fa34d6699d8..e4027bcd3574 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -384,7 +384,7 @@ private Sequence merge(List> sequencesByInterval) BinaryOperator mergeFn = toolChest.createMergeFn(query); final QueryContext queryContext = query.context(); if (parallelMergeConfig.useParallelMergePool() && queryContext.getEnableParallelMerges() && mergeFn != null) { - return new ParallelMergeCombiningSequence<>( + final ParallelMergeCombiningSequence parallelSequence = new ParallelMergeCombiningSequence<>( pool, sequencesByInterval, query.getResultOrdering(), @@ -414,6 +414,8 @@ private Sequence merge(List> sequencesByInterval) } } ); + scheduler.registerQueryFuture(query, parallelSequence.getCancellationFuture()); + return parallelSequence; } else { return Sequences .simple(sequencesByInterval) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java index a8bae9698638..7d66eebcad1b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/ApproxCountDistinctSqlAggregator.java @@ -80,6 +80,7 @@ public Aggregation toDruidAggregation( ); } + @NativelySupportsDistinct private static class ApproxCountDistinctSqlAggFunction extends SqlAggFunction { ApproxCountDistinctSqlAggFunction(SqlAggFunction delegate) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java new file mode 100644 index 000000000000..19bbaf8a0f26 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/NativelySupportsDistinct.java @@ -0,0 +1,36 @@ +/* + * 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.aggregation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * This annotation is to distinguish {@link org.apache.calcite.sql.SqlAggFunction} + * which supports the distinct aggregation natively + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE}) +public @interface NativelySupportsDistinct +{ + +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java index a5e62f5e2a9b..d20999d3afc4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArrayConcatSqlAggregator.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -142,6 +143,7 @@ public Aggregation toDruidAggregation( } } + @NativelySupportsDistinct private static class ArrayConcatAggFunction extends SqlAggFunction { ArrayConcatAggFunction() diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java index efb84dca6251..1045a79870bb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/ArraySqlAggregator.java @@ -41,6 +41,7 @@ import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -165,6 +166,7 @@ public RelDataType inferReturnType(SqlOperatorBinding sqlOperatorBinding) } } + @NativelySupportsDistinct private static class ArrayAggFunction extends SqlAggFunction { private static final ArrayAggReturnTypeInference RETURN_TYPE_INFERENCE = new ArrayAggReturnTypeInference(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index a78b3a7a4797..49469decf996 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -47,6 +47,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -226,6 +227,7 @@ public RelDataType inferReturnType(SqlOperatorBinding sqlOperatorBinding) } } + @NativelySupportsDistinct private static class StringAggFunction extends SqlAggFunction { private static final StringAggReturnTypeInference RETURN_TYPE_INFERENCE = new StringAggReturnTypeInference(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java index 18638b32afd8..16d3541e96c6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidSqlValidator.java @@ -28,6 +28,7 @@ import org.apache.calcite.rel.type.RelRecordType; import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.runtime.CalciteException; +import org.apache.calcite.sql.SqlAggFunction; import org.apache.calcite.sql.SqlCall; import org.apache.calcite.sql.SqlIdentifier; import org.apache.calcite.sql.SqlInsert; @@ -65,6 +66,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.expression.builtin.ScalarInArrayOperatorConversion; import org.apache.druid.sql.calcite.parser.DruidSqlIngest; import org.apache.druid.sql.calcite.parser.DruidSqlInsert; @@ -760,8 +762,10 @@ public void validateCall(SqlCall call, SqlValidatorScope scope) throw buildCalciteContextException( StringUtils.format( "The query contains window functions; To run these window functions, specify [%s] in query context.", - PlannerContext.CTX_ENABLE_WINDOW_FNS), - call); + PlannerContext.CTX_ENABLE_WINDOW_FNS + ), + call + ); } } if (call.getKind() == SqlKind.NULLS_FIRST) { @@ -776,9 +780,38 @@ public void validateCall(SqlCall call, SqlValidatorScope scope) throw buildCalciteContextException("ASCENDING ordering with NULLS LAST is not supported!", call); } } + if (plannerContext.getPlannerConfig().isUseApproximateCountDistinct() && isSqlCallDistinct(call)) { + if (call.getOperator().getKind() != SqlKind.COUNT && call.getOperator() instanceof SqlAggFunction) { + if (!call.getOperator().getClass().isAnnotationPresent(NativelySupportsDistinct.class)) { + throw buildCalciteContextException( + StringUtils.format( + "Aggregation [%s] with DISTINCT is not supported when useApproximateCountDistinct is enabled. Run with disabling it.", + call.getOperator().getName() + ), + call + ); + } + } + } super.validateCall(call, scope); } + @Override + protected void validateWindowClause(SqlSelect select) + { + SqlNodeList windows = select.getWindowList(); + for (SqlNode sqlNode : windows) { + if (SqlUtil.containsAgg(sqlNode)) { + throw buildCalciteContextException( + "Aggregation inside window is currently not supported with syntax WINDOW W AS . " + + "Try providing window definition directly without alias", + sqlNode + ); + } + } + super.validateWindowClause(select); + } + @Override protected SqlNode performUnconditionalRewrites(SqlNode node, final boolean underFrom) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java index 39c180530f0c..8e1fc3ee2750 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java @@ -449,19 +449,22 @@ public WindowFrame getWindowFrame() if (group.lowerBound.isUnbounded() && group.upperBound.isUnbounded()) { return WindowFrame.unbounded(); } - return new WindowFrame( - group.isRows ? WindowFrame.PeerType.ROWS : WindowFrame.PeerType.RANGE, - group.lowerBound.isUnbounded(), - figureOutOffset(group.lowerBound), - group.upperBound.isUnbounded(), - figureOutOffset(group.upperBound), - group.isRows ? null : getOrdering() - ); + if (group.isRows) { + return WindowFrame.rows(getBoundAsInteger(group.lowerBound), getBoundAsInteger(group.upperBound)); + } else { + /* Right now we support GROUPS based framing in the native layer; + * but the SQL layer doesn't accept that as of now. + */ + return WindowFrame.groups(getBoundAsInteger(group.lowerBound), getBoundAsInteger(group.upperBound), getOrderingColumNames()); + } } - private int figureOutOffset(RexWindowBound bound) + private Integer getBoundAsInteger(RexWindowBound bound) { - if (bound.isUnbounded() || bound.isCurrentRow()) { + if (bound.isUnbounded()) { + return null; + } + if (bound.isCurrentRow()) { return 0; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java index fecabd00ec39..f0632006d106 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/GroupByRules.java @@ -22,11 +22,13 @@ import org.apache.calcite.rel.core.AggregateCall; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlKind; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.aggregation.Aggregation; +import org.apache.druid.sql.calcite.aggregation.NativelySupportsDistinct; import org.apache.druid.sql.calcite.aggregation.SqlAggregator; import org.apache.druid.sql.calcite.expression.Expressions; import org.apache.druid.sql.calcite.filtration.Filtration; @@ -69,6 +71,16 @@ public static Aggregation translateAggregateCall( return null; } + if (call.isDistinct() && call.getAggregation().getKind() != SqlKind.COUNT) { + if (!call.getAggregation().getClass().isAnnotationPresent(NativelySupportsDistinct.class)) { + plannerContext.setPlanningError( + "Aggregation [%s] with DISTINCT is not supported when useApproximateCountDistinct is enabled. Run with disabling it.", + call.getAggregation().getName() + ); + return null; + } + } + final DimFilter filter; if (call.filterArg >= 0) { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java index 4be552569926..7964bca842c9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteExplainQueryTest.java @@ -40,9 +40,6 @@ public void testExplainCountStarOnView() skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT COUNT(*) FROM view.aview WHERE dim1_firstchar <> 'z'"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"selector\",\"dimension\":\"dim2\",\"value\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"z\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - : "DruidQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"substring(\\\"dim1\\\", 0, 1)\",\"outputType\":\"STRING\"}],\"filter\":{\"type\":\"and\",\"fields\":[{\"type\":\"equals\",\"column\":\"dim2\",\"matchValueType\":\"STRING\",\"matchValue\":\"a\"},{\"type\":\"not\",\"field\":{\"type\":\"equals\",\"column\":\"v0\",\"matchValueType\":\"STRING\",\"matchValue\":\"z\"}}]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"; final String explanation = NullHandling.replaceWithDefault() ? "[{" + "\"query\":{\"queryType\":\"timeseries\"," @@ -70,15 +67,6 @@ public void testExplainCountStarOnView() final String resources = "[{\"name\":\"aview\",\"type\":\"VIEW\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{legacyExplanation, resources, attributes} - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -127,17 +115,6 @@ public void testExplainExactCountDistinctOfSemiJoinResult() + " SELECT SUBSTRING(dim1, 1, 1) FROM druid.foo WHERE dim1 IS NOT NULL\n" + " )\n" + ")"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? - "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - : - "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" - + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"null\",\"column\":\"dim1\"}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"; final String explanation = NullHandling.replaceWithDefault() ? "[" + "{\"query\":{\"queryType\":\"groupBy\"," @@ -160,87 +137,20 @@ public void testExplainExactCountDistinctOfSemiJoinResult() ImmutableList.of(), ImmutableList.of(new Object[]{explanation, resources, attributes}) ); - - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{legacyExplanation, resources, attributes}) - ); } - // This testcase has been added here and not in CalciteSelectQueryTest since this checks if the overrides are working - // properly when displaying the output of "EXPLAIN PLAN FOR ..." queries @Test - public void testExplainSelectStarWithOverrides() + public void testExplainSelectStar() { - Map useRegularExplainContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); - useRegularExplainContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); - - Map legacyExplainContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); - legacyExplainContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, false); - - // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); - String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]" - + "}]"; + String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; - String explanationWithContext = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]" - + "}]"; String sql = "EXPLAIN PLAN FOR SELECT * FROM druid.foo"; String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - // Test when default config and no overrides testQuery(sql, ImmutableList.of(), ImmutableList.of(new Object[]{explanation, resources, attributes})); - - // Test when default config and useNativeQueryExplain is overridden in the context - testQuery( - sql, - legacyExplainContext, - ImmutableList.of(), - ImmutableList.of(new Object[]{legacyExplanationWithContext, resources, attributes}) - ); - - // Test when useNativeQueryExplain enabled by default and no overrides - testQuery( - PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, - sql, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{explanation, resources, attributes}) - ); - - // Test when useNativeQueryExplain enabled by default but is overriden in the context - testQuery( - PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, - useRegularExplainContext, - sql, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of(new Object[]{explanationWithContext, resources, attributes}) - ); } @Test @@ -252,46 +162,11 @@ public void testExplainMultipleTopLevelUnionAllQueries() final String query = "EXPLAIN PLAN FOR SELECT dim1 FROM druid.foo\n" + "UNION ALL (SELECT dim1 FROM druid.foo WHERE dim1 = '42'\n" + "UNION ALL SELECT dim1 FROM druid.foo WHERE dim1 = '44')"; - final String legacyExplanation = NullHandling.replaceWithDefault() - ? "DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - : "DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"; final String explanation = NullHandling.replaceWithDefault() - ? "[" - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}]" - : "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; + ? "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]" + : "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{legacyExplanation, resources, attributes} - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -319,23 +194,7 @@ public void testExplainSelectMvfilterExpressions() defaultExprContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); defaultExprContext.put(PlannerConfig.CTX_KEY_FORCE_EXPRESSION_VIRTUAL_COLUMNS, true); - final String expectedPlanWithDefaultExpressions = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"filter((x) -> array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"filter((x) -> !array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\",\"v1\"]," - + "" - + "\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]" - + "}]"; + final String expectedPlanWithDefaultExpressions = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"filter((x) -> array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"filter((x) -> !array_contains(array('true','false'), x), \\\"dim1\\\")\",\"outputType\":\"STRING\"}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"forceExpressionVirtualColumns\":true,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]}]"; final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String expectedAttributes = "{\"statementType\":\"SELECT\"}"; testQuery( @@ -346,23 +205,7 @@ public void testExplainSelectMvfilterExpressions() ); // Test plan as mv-filtered virtual columns - final String expectedPlanWithMvfiltered = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"mv-filtered\",\"name\":\"v0\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":true}," - + "{\"type\":\"mv-filtered\",\"name\":\"v1\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":false}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\",\"v1\"]," - + "" - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]" - + "}]"; + final String expectedPlanWithMvfiltered = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"mv-filtered\",\"name\":\"v0\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":true},{\"type\":\"mv-filtered\",\"name\":\"v1\",\"delegate\":{\"type\":\"default\",\"dimension\":\"dim1\",\"outputName\":\"dim1\",\"outputType\":\"STRING\"},\"values\":[\"true\",\"false\"],\"isAllowList\":false}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"EXPR$1\"}]}]"; final Map mvFilteredContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); mvFilteredContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); @@ -387,22 +230,7 @@ public void testExplainSelectTimestampExpression() final Map queryContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); queryContext.put(PlannerConfig.CTX_KEY_USE_NATIVE_QUERY_EXPLAIN, true); - final String expectedPlan = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[" - + "{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"dim1\\\",null,'UTC')\",\"outputType\":\"LONG\"}" - + "]," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"v0\"]," - + "" - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"}]" - + "}]"; + final String expectedPlan = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"dim1\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"columns\":[\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"EXPR$0\"}]}]"; final String expectedResources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String expectedAttributes = "{\"statementType\":\"SELECT\"}"; // Verify the query plan diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index d369c23bc1ac..ba7b19311ce9 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -57,7 +57,6 @@ import org.junit.jupiter.api.Test; import java.io.File; -import java.io.IOException; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -742,48 +741,9 @@ public void testExplainPlanInsertWithClusteredBy() throws JsonProcessingExceptio .columnTypes(ColumnType.LONG, ColumnType.STRING, ColumnType.FLOAT, ColumnType.DOUBLE) .build(); - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{__time:LONG, v0:FLOAT, dim1:STRING, v1:DOUBLE}])\n"; - - - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - // Test correctness of the query when only the CLUSTERED BY clause is present final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," - + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," - + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"}," - + "{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"DOUBLE\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -835,51 +795,8 @@ public void testExplainPlanInsertWithAsSubQueryClusteredBy() + "PARTITIONED BY ALL\n" + "CLUSTERED BY 2, 3"; - final String legacyExplanation = "DruidQueryRel(" - + "query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\"," - + "\"inputSource\":{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" - + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," - + "\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," - + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}]," - + " signature=[{__time:LONG, namespace:STRING, country:STRING}])\n"; - - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - // Test correctness of the query when only the CLUSTERED BY clause is present - final String explanation = "[{\"query\":{\"queryType\":\"scan\"," + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\"," - + "\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n" - + "{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"}," - + "\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," - + "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"}," - + "{\"queryColumn\":\"country\",\"outputColumn\":\"country\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"{\\\" \\\": 1681794225551, \\\"namespace\\\": \\\"day1\\\", \\\"country\\\": \\\"one\\\"}\\n{\\\"__time\\\": 1681794225558, \\\"namespace\\\": \\\"day2\\\", \\\"country\\\": \\\"two\\\"}\"},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"namespace\",\"order\":\"ascending\"},{\"columnName\":\"country\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"country\",\"namespace\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"namespace\",\"type\":\"STRING\"},{\"name\":\"country\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"namespace\",\"outputColumn\":\"namespace\"},{\"queryColumn\":\"country\",\"outputColumn\":\"country\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -940,49 +857,9 @@ public void testExplainPlanInsertJoinQuery() + "PARTITIONED BY HOUR\n" + "CLUSTERED BY 1, 2, 3, regionName"; - final String legacyExplanation = "DruidJoinQueryRel(condition=[=($3, $6)], joinType=[left], query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\"," - + "\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," - + "{\"columnName\":\"Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"Capital\",\"isRobot\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," - + "\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{v0:LONG, isRobot:STRING, Capital:STRING, regionName:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"}," - + "\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"cityName\",\"countryIsoCode\",\"isRobot\",\"regionName\",\"timestamp\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{isRobot:STRING, timestamp:STRING, cityName:STRING, countryIsoCode:STRING, regionName:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," - + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}], signature=[{Capital:STRING, ISO2:STRING}])\n"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - sql, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); // Test correctness of the query when only the CLUSTERED BY clause is present - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"}" - + ",\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"}," - + "{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true}," - + "\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\"," - + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"}," - + "{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"}," - + "{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://boo.gz\"]},\"inputFormat\":{\"type\":\"json\"},\"signature\":[{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"timestamp\",\"type\":\"STRING\"},{\"name\":\"cityName\",\"type\":\"STRING\"},{\"name\":\"countryIsoCode\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}]},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"https://foo.tsv\"]},\"inputFormat\":{\"type\":\"tsv\",\"delimiter\":\"\\t\",\"findColumnsFromHeader\":true},\"signature\":[{\"name\":\"Country\",\"type\":\"STRING\"},{\"name\":\"Capital\",\"type\":\"STRING\"},{\"name\":\"ISO3\",\"type\":\"STRING\"},{\"name\":\"ISO2\",\"type\":\"STRING\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"Capital\",\"ISO2\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false}},\"rightPrefix\":\"j0.\",\"condition\":\"(\\\"countryIsoCode\\\" == \\\"j0.ISO2\\\")\",\"joinType\":\"LEFT\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"timestamp_parse(\\\"timestamp\\\",null,'UTC')\",\"outputType\":\"LONG\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"isRobot\",\"order\":\"ascending\"},{\"columnName\":\"j0.Capital\",\"order\":\"ascending\"},{\"columnName\":\"regionName\",\"order\":\"ascending\"}],\"columns\":[\"isRobot\",\"j0.Capital\",\"regionName\",\"v0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"v0\",\"type\":\"LONG\"},{\"name\":\"isRobot\",\"type\":\"STRING\"},{\"name\":\"j0.Capital\",\"type\":\"STRING\"},{\"name\":\"regionName\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"v0\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"isRobot\",\"outputColumn\":\"isRobotAlias\"},{\"queryColumn\":\"j0.Capital\",\"outputColumn\":\"countryCapital\"},{\"queryColumn\":\"regionName\",\"outputColumn\":\"regionName\"}]}]"; testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, @@ -1274,7 +1151,7 @@ public void testInsertWithoutPartitionedBy() } @Test - public void testExplainInsertFromExternal() throws IOException + public void testExplainInsertFromExternal() { // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); @@ -1284,62 +1161,12 @@ public void testExplainInsertFromExternal() throws IOException externSql(externalDataSource) ); - ObjectMapper queryJsonMapper = queryFramework().queryJsonMapper(); - final ScanQuery expectedQuery = newScanQueryBuilder() - .dataSource(externalDataSource) - .intervals(querySegmentSpec(Filtration.eternity())) - .columns("x", "y", "z") - .context( - queryJsonMapper.readValue( - "{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}", - JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT - ) - ) - .columnTypes(STRING, STRING, LONG) - .build(); - - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{x:STRING, y:STRING, z:LONG}])\n"; - final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"}," - + "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - query, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - - testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, ImmutableMap.of("sqlQueryId", "dummy"), @@ -1397,51 +1224,12 @@ public void testExplainPlanForInsertWithClusteredBy() throws JsonProcessingExcep .columnTypes(LONG, STRING, FLOAT, DOUBLE) .build(); - - final String legacyExplanation = - "DruidQueryRel(query=[" - + queryJsonMapper.writeValueAsString(expectedQuery) - + "], signature=[{__time:LONG, v0:FLOAT, dim1:STRING, v1:DOUBLE}])\n"; - final String explanation = - "[" - + "{\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"}," - + "{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}]," - + "\"resultFormat\":\"compactedList\"," - + "\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"}," - + "{\"name\":\"v1\",\"type\":\"DOUBLE\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"}," - + "{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]" - + "}]"; + "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"virtualColumns\":[{\"type\":\"expression\",\"name\":\"v0\",\"expression\":\"floor(\\\"m1\\\")\",\"outputType\":\"FLOAT\"},{\"type\":\"expression\",\"name\":\"v1\",\"expression\":\"ceil(\\\"m2\\\")\",\"outputType\":\"DOUBLE\"}],\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"v0\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"v1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"dim1\",\"v0\",\"v1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"STRING\",\"FLOAT\",\"DOUBLE\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"v0\",\"type\":\"FLOAT\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"v1\",\"type\":\"DOUBLE\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"v0\",\"outputColumn\":\"floor_m1\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"v1\",\"outputColumn\":\"ceil_m2\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"floor_m1\",\"dim1\",\"CEIL(\\\"m2\\\")\"]}"; - // Use testQuery for EXPLAIN (not testIngestionQuery). - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - ImmutableMap.of("sqlQueryId", "dummy"), - Collections.emptyList(), - query, - CalciteTests.SUPER_USER_AUTH_RESULT, - ImmutableList.of(), - new DefaultResultsVerifier( - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ), - null - ) - ); - testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, ImmutableMap.of("sqlQueryId", "dummy"), 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 29751279339f..84b437c20e08 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 @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; +import org.apache.calcite.rel.RelNode; import org.apache.calcite.runtime.CalciteContextException; import org.apache.druid.common.config.NullHandling; import org.apache.druid.error.DruidException; @@ -149,6 +150,7 @@ import java.util.stream.Collectors; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assumptions.assumeFalse; @@ -15501,6 +15503,20 @@ public void testWindowingErrorWithoutFeatureFlag() assertThat(e, invalidSqlIs("The query contains window functions; To run these window functions, specify [enableWindowing] in query context. (line [1], column [13])")); } + @Test + public void testDistinctSumNotSupportedWithApproximation() + { + DruidException e = assertThrows( + DruidException.class, + () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerConfig.CTX_KEY_USE_APPROXIMATE_COUNT_DISTINCT, true)) + .sql("SELECT sum(distinct m1) from druid.foo") + .run() + ); + + assertThat(e, invalidSqlContains("Aggregation [SUM] with DISTINCT is not supported")); + } + @Test public void testUnSupportedNullsFirst() { @@ -15578,6 +15594,23 @@ public void testDistinctNotSupportedWithWindow() assertThat(e, invalidSqlContains("DISTINCT is not supported for window functions")); } + @Test + public void testUnSupportedAggInSelectWindow() + { + assertEquals( + "1.37.0", + RelNode.class.getPackage().getImplementationVersion(), + "Calcite version changed; check if CALCITE-6500 is fixed and update:\n * method DruidSqlValidator#validateWindowClause" + ); + + DruidException e = assertThrows(DruidException.class, () -> testBuilder() + .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) + .sql("SELECT dim1, ROW_NUMBER() OVER W from druid.foo WINDOW W as (ORDER BY max(length(dim1)))") + .run()); + + assertThat(e, invalidSqlContains("not supported with syntax WINDOW W AS ")); + } + @Test public void testInGroupByLimitOutGroupByOrderBy() { @@ -15886,7 +15919,7 @@ public void testWindowingWithOrderBy() OperatorFactoryBuilders.naivePartitionOperator(), OperatorFactoryBuilders.windowOperators( OperatorFactoryBuilders.framedAggregateProcessor( - WindowFrame.forOrderBy(ColumnWithDirection.ascending("d0")), + WindowFrame.forOrderBy("d0"), new LongSumAggregatorFactory("w0", "a0") ) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index 49db00c2519e..721eec21900b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -699,20 +699,7 @@ public void testExplainReplaceFromExternal() throws IOException + queryJsonMapper.writeValueAsString(expectedQuery) + "], signature=[{x:STRING, y:STRING, z:LONG}])\n"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"}," - + "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\"," - + "\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; - + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"inline\",\"data\":\"a,b,1\\nc,d,2\\n\"},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"all\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"},\"replaceTimeChunks\":\"all\"}"; @@ -790,7 +777,7 @@ public void testExplainReplaceTimeChunksWithPartitioningAndClustering() throws I + "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"orderBy\":[{\"columnName\":\"dim1\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"DAY\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"DAY\",\"clusteredBy\":[\"dim1\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; @@ -875,24 +862,7 @@ public void testExplainReplaceWithLimitAndClusteredByOrdinals() throws IOExcepti + queryJsonMapper.writeValueAsString(expectedQuery) + "], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[" - + "{\"query\":{\"queryType\":\"scan\",\"dataSource\":" - + "{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\"," - + "\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\",\"limit\":10," - + "\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"}," - + "{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}]," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\"," - + "\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\"," - + "\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"]," - + "\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"}," - + "{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}," - + "{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"}," - + "{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"limit\":10,\"orderBy\":[{\"columnName\":\"__time\",\"order\":\"ascending\"},{\"columnName\":\"dim1\",\"order\":\"ascending\"},{\"columnName\":\"dim3\",\"order\":\"ascending\"},{\"columnName\":\"dim2\",\"order\":\"ascending\"}],\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"\\\"HOUR\\\"\",\"sqlQueryId\":\"dummy\",\"sqlReplaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"dst\",\"type\":\"DATASOURCE\"},{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"REPLACE\",\"targetDataSource\":\"dst\",\"partitionedBy\":\"HOUR\"," + "\"clusteredBy\":[\"__time\",\"dim1\",\"dim3\",\"dim2\"],\"replaceTimeChunks\":\"2000-01-01T00:00:00.000Z/2000-01-02T00:00:00.000Z\"}"; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index efe04f99a630..3c47eb20491e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -595,35 +595,10 @@ public void testExplainSelectConstantExpression() // Skip vectorization since otherwise the "context" will change for each subtest. skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT 1 + 1"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"EXPR$0\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]" - + "}]"; - final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"}}], signature=[{EXPR$0:LONG}])\n"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"inline\",\"columnNames\":[\"EXPR$0\"],\"columnTypes\":[\"LONG\"],\"rows\":[[2]]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"EXPR$0\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"EXPR$0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"EXPR$0\",\"outputColumn\":\"EXPR$0\"}]}]"; final String resources = "[]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, @@ -1330,33 +1305,10 @@ public void testExplainSelectStar() skipVectorize(); final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo"; - final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n"; - final String explanation = "[{" - + "\"query\":{\"queryType\":\"scan\"," - + "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"}," - + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," - + "\"resultFormat\":\"compactedList\"," - + "\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"]," - + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," - + "\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"LONG\",\"LONG\",\"STRING\",\"STRING\",\"STRING\",\"FLOAT\",\"DOUBLE\",\"COMPLEX\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX\"}],\"columnMappings\":[{\"queryColumn\":\"__time\",\"outputColumn\":\"__time\"},{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"},{\"queryColumn\":\"dim2\",\"outputColumn\":\"dim2\"},{\"queryColumn\":\"dim3\",\"outputColumn\":\"dim3\"},{\"queryColumn\":\"cnt\",\"outputColumn\":\"cnt\"},{\"queryColumn\":\"m1\",\"outputColumn\":\"m1\"},{\"queryColumn\":\"m2\",\"outputColumn\":\"m2\"},{\"queryColumn\":\"unique_dim1\",\"outputColumn\":\"unique_dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; - testQuery( - PLANNER_CONFIG_LEGACY_QUERY_EXPLAIN, - query, - CalciteTests.REGULAR_USER_AUTH_RESULT, - ImmutableList.of(), - ImmutableList.of( - new Object[]{ - legacyExplanation, - resources, - attributes - } - ) - ); testQuery( PLANNER_CONFIG_NATIVE_QUERY_EXPLAIN, query, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java index 9fdd73fb9c74..165b4aa3f631 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java @@ -45,8 +45,10 @@ import java.io.File; import java.net.URL; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; import java.util.Objects; import static org.junit.Assert.assertEquals; @@ -66,6 +68,11 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest private static final ObjectMapper YAML_JACKSON = new DefaultObjectMapper(new YAMLFactory(), "tests"); + private static final Map DEFAULT_QUERY_CONTEXT = ImmutableMap.of( + PlannerContext.CTX_ENABLE_WINDOW_FNS, true, + QueryContexts.ENABLE_DEBUG, true + ); + public static Object[] parametersForWindowQueryTest() throws Exception { final URL windowFolderUrl = ClassLoader.getSystemResource("calcite/tests/window"); @@ -184,34 +191,17 @@ private void maybeDumpActualResults(List results) throws Exception log.info("Actual results:\n%s", sb.toString()); } } - } - - @MethodSource("parametersForWindowQueryTest") - @ParameterizedTest(name = "{0}") - @SuppressWarnings("unchecked") - public void windowQueryTest(String filename) throws Exception - { - TestCase testCase = new TestCase(filename); - - assumeTrue(testCase.getType() != TestType.failingTest); - if (testCase.getType() == TestType.operatorValidation) { - testBuilder() - .skipVectorize(true) - .sql(testCase.getSql()) - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) - .addCustomVerification(QueryVerification.ofResults(testCase)) - .run(); + public Map getQueryContext() + { + return input.queryContext == null ? Collections.emptyMap() : input.queryContext; } } @MethodSource("parametersForWindowQueryTest") @ParameterizedTest(name = "{0}") @SuppressWarnings("unchecked") - public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) throws Exception + public void windowQueryTest(String filename) throws Exception { TestCase testCase = new TestCase(filename); @@ -221,10 +211,11 @@ public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) th testBuilder() .skipVectorize(true) .sql(testCase.getSql()) - .queryContext(ImmutableMap.of(QueryContexts.ENABLE_DEBUG, true, - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000" - ) + .queryContext( + ImmutableMap.builder() + .putAll(DEFAULT_QUERY_CONTEXT) + .putAll(testCase.getQueryContext()) + .build() ) .addCustomVerification(QueryVerification.ofResults(testCase)) .run(); @@ -241,10 +232,7 @@ public void testWithArrayConcat() + "where countryName in ('Austria', 'Republic of Korea') " + "and (cityName in ('Vienna', 'Seoul') or cityName is null)\n" + "group by countryName, cityName, channel") - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) + .queryContext(DEFAULT_QUERY_CONTEXT) .expectedResults( ResultMatchMode.RELAX_NULLS, ImmutableList.of( @@ -277,9 +265,13 @@ enum TestType failingTest, operatorValidation } + @JsonProperty public TestType type; + @JsonProperty + public Map queryContext; + @JsonProperty public String sql; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java index 24076d1cdbf9..4a2f09450871 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/DrillWindowQueryTest.java @@ -4426,7 +4426,7 @@ public void test_last_val_lastValFn_39() windowQueryTest(); } - @NotYetSupported(Modes.NOT_ENOUGH_RULES) + @NotYetSupported(Modes.DISTINCT_AGGREGATE_NOT_SUPPORTED) @DrillTest("nestedAggs/emtyOvrCls_7") @Test public void test_nestedAggs_emtyOvrCls_7() @@ -7274,7 +7274,7 @@ public void test_nestedAggs_emtyOvrCls_13() windowQueryTest(); } - @NotYetSupported(Modes.RESULT_MISMATCH) + @NotYetSupported(Modes.DISTINCT_AGGREGATE_NOT_SUPPORTED) @DrillTest("nestedAggs/emtyOvrCls_8") @Test public void test_nestedAggs_emtyOvrCls_8() @@ -7750,4 +7750,25 @@ public void test_empty_and_non_empty_over_wikipedia_query_3() { windowQueryTest(); } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_1") + @Test + public void test_partition_by_array_wikipedia_query_1() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_2") + @Test + public void test_partition_by_array_wikipedia_query_2() + { + windowQueryTest(); + } + + @DrillTest("druid_queries/partition_by_array/wikipedia_query_3") + @Test + public void test_partition_by_array_wikipedia_query_3() + { + windowQueryTest(); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java index badbbdc0e66f..29200730fee3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/IngestTableFunctionTest.java @@ -313,19 +313,7 @@ public void testExplainHttpFn() " format => 'csv'))\n" + " EXTEND (x VARCHAR, y VARCHAR, z BIGINT)\n" + "PARTITIONED BY ALL TIME"; - final String explanation = "[{" + - "\"query\":{\"queryType\":\"scan\"," + - "\"dataSource\":{\"type\":\"external\"," + - "\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}}," + - "\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]}," + - "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," + - "\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"]," + - "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\"," + - "\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\"," + - "\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}," + - "\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"}}," + - "\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]," + - "\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; + final String explanation = "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"external\",\"inputSource\":{\"type\":\"http\",\"uris\":[\"http://foo.com/bar.csv\"],\"httpAuthenticationUsername\":\"bob\",\"httpAuthenticationPassword\":{\"type\":\"default\",\"password\":\"secret\"}},\"inputFormat\":{\"type\":\"csv\",\"columns\":[\"x\",\"y\",\"z\"]},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}]},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"x\",\"y\",\"z\"],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlInsertSegmentGranularity\":\"{\\\"type\\\":\\\"all\\\"}\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"columnTypes\":[\"STRING\",\"STRING\",\"LONG\"],\"granularity\":{\"type\":\"all\"},\"legacy\":false},\"signature\":[{\"name\":\"x\",\"type\":\"STRING\"},{\"name\":\"y\",\"type\":\"STRING\"},{\"name\":\"z\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"x\",\"outputColumn\":\"x\"},{\"queryColumn\":\"y\",\"outputColumn\":\"y\"},{\"queryColumn\":\"z\",\"outputColumn\":\"z\"}]}]"; final String resources = "[{\"name\":\"EXTERNAL\",\"type\":\"EXTERNAL\"},{\"name\":\"dst\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"INSERT\",\"targetDataSource\":\"dst\",\"partitionedBy\":{\"type\":\"all\"}}"; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java index eaa97be231f9..e5442a2bda24 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/NotYetSupported.java @@ -77,7 +77,7 @@ enum Modes { // @formatter:off - NOT_ENOUGH_RULES(DruidException.class, "not enough rules"), + DISTINCT_AGGREGATE_NOT_SUPPORTED(DruidException.class, "DISTINCT is not supported"), ERROR_HANDLING(AssertionError.class, "targetPersona: is <[A-Z]+> and category: is <[A-Z_]+> and errorCode: is"), EXPRESSION_NOT_GROUPED(DruidException.class, "Expression '[a-z]+' is not being grouped"), NULLS_FIRST_LAST(DruidException.class, "NULLS (FIRST|LAST)"), diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq index 11f1a7eb80ee..05f67289ab5f 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/join.iq @@ -104,7 +104,8 @@ select v.*,e.* from v inner join e on (e.cityName = v.cityName); "columnTypes" : [ "LONG", "STRING", "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan LogicalProject(cityName=[$0], cnt=[$1], cityName0=[$2], cnt0=[$3]) diff --git a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq index f14d26cd58a9..7d4ed9909b82 100644 --- a/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq +++ b/sql/src/test/quidem/org.apache.druid.quidem.SqlQuidemTest/simple.iq @@ -42,7 +42,8 @@ select cityName, countryName from wikipedia where cityName='New York' limit 1; "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan LogicalSort(fetch=[1]) diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq index 993680ae4c21..07245be4ca16 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testCommaJoinLeftFunction.iq @@ -63,7 +63,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq index aab3d8b50e5a..794195713eda 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq index 3bce28a44e98..188abdd4e4c4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_disabled@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq index 74ef149e685c..37a87f4a74c2 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq index 4a5a77ef1bc0..b1bca799e36d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@all_enabled@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq index e9e396fcf9d4..dbc1d7eaf96c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=default.iq @@ -82,7 +82,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -107,7 +108,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq index d6f26f85c927..0f5d1b8fe123 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@default@NullHandling=sql.iq @@ -83,7 +83,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq index 6cfc2e1f12e0..fe58d1e4ebe4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq index fc38514a401e..5fe30d6b657a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-on-value-column_disabled@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq index 5ff1078fc87b..57accf2fbc31 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq index d6d1bd8bd601..4883e3b9894c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites-disabled@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq index d3400229b731..072fac7ae752 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq index e95642a5a235..f874137e09f5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@filter-rewrites@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq index d907fb0f3eac..b3b0e2a346cb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=default.iq @@ -85,7 +85,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -110,7 +111,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq index 6e02805ed7ca..8980b7c4160c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources@join-to-filter@NullHandling=sql.iq @@ -86,7 +86,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -112,7 +113,8 @@ DruidAggregate(group=[{0}], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq index 19e256b68f88..b834b63636ab 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq index cc8a2f42832f..2449b9427743 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_disabled@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq index 16f70e47b8ee..b898d2f1c0d4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq index d479bc1cc150..fb1f9aa91513 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@all_enabled@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq index d595c7f9a5bf..41c9b0393aa3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=default.iq @@ -60,7 +60,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -80,6 +81,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq index 54e573201090..b0a97e2b2991 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@default@NullHandling=sql.iq @@ -60,7 +60,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -80,6 +81,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq index e5ead14de40e..3b58962a1420 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq index ff43508cb0f5..2f250bce5b9f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-on-value-column_disabled@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq index 8e8e12dda6e6..0c6f38cfafb6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq index 83ad89882c7f..1dbe2ef067fc 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites-disabled@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq index bd90718c84f2..8783d7b3734c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq index d817f0eaad29..bdc925467635 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@filter-rewrites@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq index 8e7c0c15b167..517a5e8f1480 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=default.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq index ea3d3cdc18f9..9593087abadc 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinCastLeft@join-to-filter@NullHandling=sql.iq @@ -63,7 +63,8 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "FLOAT", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(m1=[$0], k=[$2], v=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "FLOAT" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq index 94510babc6d9..098749d62259 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_disabled.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq index 03a371936a8c..19d8ce995296 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@all_enabled.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq index a0eee78264f2..a5092ed5c4c7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@default.iq @@ -63,7 +63,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -83,6 +84,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq index fa647b06367f..fc27dfddef62 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-on-value-column_disabled.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq index ca5fbac7643d..b16cd786e187 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites-disabled.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq index fc69d073ebfa..3fc0468e7a39 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@filter-rewrites.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq index 6b14408362ab..8df1ddf12e03 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinLeftFunction@join-to-filter.iq @@ -66,7 +66,8 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], k=[$3], v=[$4], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 30dd4d9e21cb..e0132c50e97f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index 58c79d95df54..e95f2f14e852 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq index df3eec1a32a9..b7fe7b9911f4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 8c7a76b721af..f5cf5cd53aaf 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq index af233e77cc25..a1a9eb3a74ed 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -75,7 +75,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,7 +101,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -122,6 +124,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq index 2ebaee2e4994..4f1a6c50e7d6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq @@ -76,7 +76,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,6 +126,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index ac4309e9a58f..e499257505ee 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index 258c9dd8427e..047a0d9f3d61 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index dd9bedccdf3d..2ed19caa6dd5 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 4e4a97596c8f..6e1a7ec7603c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index f32757809fc6..7ba1e5e14d7a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 2d27fe88b449..d5e130f03a5f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 526f45cea0a5..609ae044013b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index fa3b2f99623c..1753c081a4b3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq index ee44745b7cfc..20e69c705a24 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index ddc0d96e7e62..2d034cb92009 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq index 7ae5addaf83f..9d96525fbeaf 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 6f76974ff437..b5533f936323 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq index 3b03d8444587..1c782f5f3945 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -75,7 +75,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,7 +101,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -122,6 +124,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq index 4f7c30602d5a..62cc25dc99d1 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq @@ -76,7 +76,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,6 +126,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 6d5e53608862..e2d47b9b8158 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index 169a3c25edb8..27133219625a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 3ac3494003e0..db70d749d5b7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index a1b6329e34c2..f2d6e9b0a0bf 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index c249c4927def..d17043dbd492 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 0d647e31f09f..ef9aff735e4a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 1a4a44d61c4b..eebc5217765f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index bbdacff71d32..8578eedb0671 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq index d0e9639d66f8..da6ada101d42 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_disabled.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq index 9704809f8f17..30482302b2e0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@all_enabled.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq index 08b61c0a5a42..ec13b17cbc02 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@default.iq @@ -104,6 +104,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq index 46eeb29104c5..ad63ef44426a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-on-value-column_disabled.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq index 997434994c3f..48890391a6b8 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites-disabled.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq index 6311f5644408..1fe9c554541a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@filter-rewrites.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq index 9eeea292be79..0ad9125b1510 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookup@join-to-filter.iq @@ -107,6 +107,7 @@ DruidProject(dim1=[$0], dim2=[$1], v=[$3], v0=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq index d650be35d63b..0784836b05de 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_disabled.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq index 6d28747396b3..00ec67ad0cba 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@all_enabled.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq index 282084738197..c9c45c4e75eb 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@default.iq @@ -70,7 +70,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -86,6 +87,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq index d0bd1b3f545b..8c077ff99362 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-on-value-column_disabled.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq index bfc74f5a1888..b4157fdc8942 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites-disabled.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq index 69fbe42c908d..92ba66af86ae 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@filter-rewrites.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq index 00329a655e0a..a47285f0c41a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testInnerJoinQueryOfLookupRemovable@join-to-filter.iq @@ -73,7 +73,8 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -89,6 +90,7 @@ DruidProject(dim1=[$0], dim2=[$1], sk=[$3], druid=[logical]) "columnTypes" : [ "STRING", "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq index 8c8353e582bd..c08b970f1617 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq index f2fa3adc9933..07124f8cac04 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime@NullHandling=sql.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq index 10c2b232d065..2f78608ebffd 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=default.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq index 4287ae123461..2a03f6e25160 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTimeWithNoAggregateMultipleValues@NullHandling=sql.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq index ec22a1c73df7..f2275d4db00f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=default.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq index 381f1ff4c5c4..077841f3fe90 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinOnTimeseriesWithFloorOnTime@NullHandling=sql.iq @@ -73,7 +73,8 @@ DruidAggregate(group=[{0, 1}], EXPR$2=[ANY_VALUE($2, $3, $4)], druid=[logical]) "columnTypes" : [ "LONG", "STRING", "FLOAT", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq index ef493a09cf04..b29ee72ab4f1 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=default.iq @@ -62,7 +62,8 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq index 85fd47af3285..1f2ac13cefef 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testJoinWithInputRefCondition@NullHandling=sql.iq @@ -70,7 +70,8 @@ DruidAggregate(group=[{}], EXPR$0=[COUNT() FILTER $0], druid=[logical]) "columnTypes" : [ "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 2fd819422471..5df8aeeaff31 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index 38c72482fb6f..d43b57114441 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq index c8d4c066bb75..76955adfbeb9 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index a482f98c76d5..05f624ac37a4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq index 94c655fec20d..4d2b6332c5b4 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=default.iq @@ -75,7 +75,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,7 +101,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -122,6 +124,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq index ccbcfff78f3b..31c46ccc3b29 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@default@NullHandling=sql.iq @@ -76,7 +76,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,6 +126,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index d46a06eba337..05a49528a40e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index 8242b6ae8510..07d5d256b617 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 1bd8d6341b01..61e0354cd15b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index f774ff7fec6e..d764e7d5e3c7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index aa6268bf9d32..2103374735da 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index b1cc848462ed..ae6924c1b875 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index bc4a7a0a9255..b86beedd9ab7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index e61c0c0fd254..904f6fec8d8f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 7860b81b5fff..bfdea514aa3e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index e7d334a00c1c..4e6d825da40c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq index 990e3516e882..dd9ac1e021d9 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index ef3fdd2fb7ec..ea1bcd75f44d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq index 290528701d04..10f1c84c330e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=default.iq @@ -75,7 +75,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -106,7 +107,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -128,6 +130,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq index ba4342c92d28..ebe0fbc3f718 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@default@NullHandling=sql.iq @@ -76,7 +76,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -108,7 +109,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -130,6 +132,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 1e68ddc963c5..ea5fdff6a88f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index a3326834777e..dcd84462bbec 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index 221c16c161f1..d870dc637b84 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 3ec24f4656f7..11442bae8a5c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index 625caf2321b6..560f7c30f74d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 1b8c22da4eda..4f254f1976ba 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 54a4b2e286ee..ff2df59c5139 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -109,7 +110,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -131,6 +133,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index 56c560bffc46..5d8f487a7806 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -111,7 +112,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -133,6 +135,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq index 2a0418aa9ded..d13c86ab3957 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq index 052684a77619..aa4220f4f427 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq index 50bdfbff93b2..aec23daced0a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq index 28786182e82a..844b53dccc3d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@all_enabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq index 0b7305d3c782..4c84575901ee 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=default.iq @@ -75,7 +75,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -100,7 +101,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -122,6 +124,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq index f452d2f32919..ccbdf92c4575 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@default@NullHandling=sql.iq @@ -76,7 +76,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -124,6 +126,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq index 74950e46f4f5..c22c33b3ea47 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq index daa23575474a..68f6a9250e67 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-on-value-column_disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq index e5e92502c1f0..aa6d4b632acd 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq index 8a74da0eaa51..9c1a8d237960 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites-disabled@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq index 951b7c878473..993c14f6869a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq index 64682aa187b4..76ee90cd8770 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@filter-rewrites@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq index 1779bb4f12df..cf7cdcc09f3e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=default.iq @@ -78,7 +78,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -103,7 +104,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -125,6 +127,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq index cb0c4b5aa323..ecd48b65d915 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess@join-to-filter@NullHandling=sql.iq @@ -79,7 +79,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -105,7 +106,8 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -127,6 +129,7 @@ DruidProject(dim1=[CAST('10.1':VARCHAR):VARCHAR], __time=[$1], druid=[logical]) "columnTypes" : [ "LONG", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq index 391d1b758ded..2fb49c488d87 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_disabled.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq index c092b8b1699f..f8c20e42d6ab 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@all_enabled.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq index 2f95ad4db95a..a365f6899bf3 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@default.iq @@ -91,6 +91,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq index e1746ea9f7da..398b3cc3a85e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-on-value-column_disabled.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq index ee1e57ad0e1e..5c86e793f30a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites-disabled.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq index 3b6855b65783..e1be3203710c 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@filter-rewrites.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq index e72fd71ea960..590296d49043 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testLeftJoinSubqueryWithSelectorFilter@join-to-filter.iq @@ -94,6 +94,7 @@ DruidJoin(condition=[AND(=($0, $1), =('abc', $1))], joinType=[inner]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq index 4e684853916b..dcff8e213aee 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq index df068fe78fa0..cb04db285061 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_disabled@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq index d180546744e5..e45394da7f8f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq index 064b58eaf540..105c345ee9c0 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@all_enabled@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq index b022186b3aba..ad6509bd826e 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=default.iq @@ -65,7 +65,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq index 1cc0aa9a19d1..6dffa2cc78df 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@default@NullHandling=sql.iq @@ -65,7 +65,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq index 870ff8404926..346a65b95040 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq index d3db0f7d7a3e..1bd6bffa1f2d 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-on-value-column_disabled@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq index 18b98ac0b4e5..9b72fc597c0a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq index 11cd61bc8588..b27b5b8cb573 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites-disabled@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq index 64189b47997e..c2c40dc99523 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq index a26f7d991315..c03d202cdaee 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@filter-rewrites@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq index 603c31aec412..b08872faf7d7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=default.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq index 8a87da90dd7a..40229ea1ec11 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testUsingSubqueryWithExtractionFns@join-to-filter@NullHandling=sql.iq @@ -68,7 +68,8 @@ DruidAggregate(group=[{0}], EXPR$1=[COUNT()], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq index 5fc2e7fb7963..8c460cad242a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_disabled.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq index bdbb0d2e62fe..ed9c27dc2d58 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@all_enabled.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq index dcceabcd2b19..4a3b1ddb5cf6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@default.iq @@ -67,7 +67,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -99,7 +100,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -115,6 +117,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq index b210ffd96ac5..5bc156562d7f 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-on-value-column_disabled.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq index 66357bec5f25..8b2cffb58c22 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites-disabled.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq index ef5ad5c6a333..d196685bbe49 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@filter-rewrites.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq index 3f279d8f893c..d29ec916a0fd 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterJoinExpression@join-to-filter.iq @@ -70,7 +70,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -102,7 +103,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -118,6 +120,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq index dd97fb01134f..af6c617d6d69 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_disabled.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq index 3a4d7482e4c1..e82f708d2bb1 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@all_enabled.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq index e42214277c5d..5c154e61c299 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@default.iq @@ -75,7 +75,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -113,7 +114,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -145,7 +147,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -173,7 +176,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -189,6 +193,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq index f1fd8213bafa..8e9600922765 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-on-value-column_disabled.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq index 438e9166fa01..a258e98d01f7 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites-disabled.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq index 5b19876eb77f..63ea77bfbbe9 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@filter-rewrites.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq index 5ac8bae8bfb0..8f9b506aa1a6 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteJoinQueryTest/testVirtualColumnOnMVFilterMultiJoinExpression@join-to-filter.iq @@ -78,7 +78,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -116,7 +117,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "right" : { @@ -148,7 +150,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "j0.", @@ -176,7 +179,8 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "rightPrefix" : "_j0.", @@ -192,6 +196,7 @@ DruidProject(dim3=[$0], dim30=[$2], druid=[logical]) "columnTypes" : [ "STRING", "STRING" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq index 85c1b392c49a..b5d2c682643b 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testGroupByWithLiteralInSubqueryGrouping.iq @@ -30,8 +30,8 @@ SELECT +-------+----+ | t1 | t2 | +-------+----+ -| dummy | b | | dummy | | +| dummy | b | +-------+----+ (2 rows) diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq index 2ed1880fef86..8bc6869aff89 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=default.iq @@ -225,6 +225,7 @@ DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq index f0b4a554df67..0f899348f837 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJoin@NullHandling=sql.iq @@ -236,6 +236,7 @@ DruidProject(dim2=[$0], EXPR$1=[$1], EXPR$2=[$3], EXPR$3=[$5], druid=[logical]) "columnTypes" : [ "LONG", "LONG", "STRING", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } !nativePlan diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq index 22304fccf7b7..8b2ca6626211 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=default.iq @@ -142,7 +142,8 @@ DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "intervals" : { diff --git a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq index 824ea018611d..ae521d6b098a 100644 --- a/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq +++ b/sql/src/test/quidem/org.apache.druid.sql.calcite.DecoupledPlanningCalciteQueryTest/testWindowingWithScanAndSort@NullHandling=sql.iq @@ -153,7 +153,8 @@ DruidProject(ranking=[$2], trend_score=[$1], druid=[logical]) "columnTypes" : [ "LONG", "DOUBLE", "LONG" ], "granularity" : { "type" : "all" - } + }, + "legacy" : false } }, "intervals" : { diff --git a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest index 0c9d88b5041f..a2f82ff2905d 100644 --- a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest @@ -15,13 +15,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + frame: { type: "rows" } aggregations: - { "type": "doubleSum", "name": "w1", "fieldName": "_d1" } - type: "naiveSort" @@ -33,13 +27,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + frame: { type: "rows" } aggregations: - { "type": "doubleSum", "name": "w0", "fieldName": "_d0" } expectedResults: diff --git a/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest b/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest index 16dbe924fdb3..e65e27e8794c 100644 --- a/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/aggregateConstant.sqlTest @@ -13,7 +13,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { type: "count", name: "w0" } diff --git a/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest index d5a324c9e2d8..aa0a4a2a0198 100644 --- a/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/defaultBoundCurrentRow.sqlTest @@ -19,60 +19,43 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + upperOffset: 0 aggregations: - { type: "count", name: "w0" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -1 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: -1 + upperOffset: 0 aggregations: - { type: "count", name: "w1" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: 0 + upperOffset: 0 aggregations: - { type: "count", name: "w2" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: false - uppOffset: 1 - orderBy: null + type: "rows" + lowerOffset: 0 + upperOffset: 1 aggregations: - { type: "count", name: "w3" } - type: "window" processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: 0 - uppUnbounded: true - uppOffset: 0 - orderBy: null + type: "rows" + lowerOffset: 0 aggregations: - { type: "count", name: "w4" } diff --git a/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest index 7c9dae4aad39..2b6f7f7fddbb 100644 --- a/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/no_grouping.sqlTest @@ -1,9 +1,9 @@ -type: "failingTest" +type: "operatorValidation" sql: | SELECT m1, - COUNT(m1) OVER () cc + SUM(m1) OVER () cc FROM druid.foo expectedOperators: @@ -12,18 +12,16 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - - type: "filtered" - aggregator: {"type":"count","name":"w0"} - filter: - type: not - field: {"type":"null","column":"m1"} - name: null + - type: doubleSum + name: w0 + fieldName: m1 + expectedResults: - - [1.0,6] - - [2.0,6] - - [3.0,6] - - [4.0,6] - - [5.0,6] - - [6.0,6] + - [1.0,21.0] + - [2.0,21.0] + - [3.0,21.0] + - [4.0,21.0] + - [5.0,21.0] + - [6.0,21.0] diff --git a/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest b/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest index 7a579c3fc331..4d78b197e312 100644 --- a/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/no_grouping2.sqlTest @@ -12,7 +12,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: rows } aggregations: - type: "doubleSum" name: "w0" diff --git a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest index 0e66ed874601..1e4de22dfca5 100644 --- a/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/rank_handling.sqlTest @@ -1,5 +1,8 @@ type: "operatorValidation" +queryContext: + maxSubqueryBytes: 100000 + sql: | SELECT __time diff --git a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest index 9ca9f88e850b..84bd5ca71af0 100644 --- a/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/simpleSum.sqlTest @@ -14,12 +14,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "RANGE" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: [ {column: "d0", direction: ASC} ] + type: groups + upperOffset: 0 + orderByColumns: [ "d0" ] aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest index 0a86a691e26a..9b5aa6e1365e 100644 --- a/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/virtualColumns.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { type: "doubleMin", name: "w0", fieldName: "_v0" } - { type: "longMin", name: "w1", fieldName: "v1" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest index cc59868482a2..b4ef8006ea9d 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaAggregationsMultipleOrdering.sqlTest @@ -16,12 +16,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -3 - uppUnbounded: false - uppOffset: 2 - orderBy: null + type: rows + lowerOffset: -3 + upperOffset: 2 aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } - { type: "naiveSort", columns: [ { column: "d1", direction: "ASC" }, { column: "a0", direction: "ASC"} ]} diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest index 9368f00e9b49..ebcc060eaa52 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaCumulativeOrdered.sqlTest @@ -39,14 +39,9 @@ expectedOperators: - { "type": "cumeDist", "group": [ "a0" ], "outputColumn": "w9" } - type: "framedAgg" frame: - peerType: "RANGE" - lowUnbounded: true - lowOffset: 0 - uppUnbounded: false - uppOffset: 0 - orderBy: - - column: a0 - direction: ASC + type: groups + upperOffset: 0 + orderByColumns: [ a0 ] aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest index c25f1ff03528..87873d44c485 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaFramedAggregations.sqlTest @@ -15,11 +15,9 @@ expectedOperators: processor: type: "framedAgg" frame: - peerType: "ROWS" - lowUnbounded: false - lowOffset: -3 - uppUnbounded: false - uppOffset: 2 + type: "rows" + lowerOffset: -3 + upperOffset: 2 orderBy: null aggregations: - { type: "longSum", name: "w0", fieldName: "a0" } diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest index 1e75e69b97bc..3843519aa790 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartition.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } - type: "window" diff --git a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest index d310f6a8f1ce..4939057621e5 100644 --- a/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/wikipediaSimplePartitionInitialSort.sqlTest @@ -28,7 +28,7 @@ expectedOperators: - { "type": "last", "inputColumn": "a0", "outputColumn": "w2" } - { "type": "percentile", "outputColumn": "w3", "numBuckets": 3 } - type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: "rows" } aggregations: - { "type": "longSum", "name": "w0", "fieldName": "a0" } - type: "window" diff --git a/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest b/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest index c96b979c0dac..7c7fd03c3c81 100644 --- a/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest +++ b/sql/src/test/resources/calcite/tests/window/windowed_long_null.sqlTest @@ -19,7 +19,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "ROWS", lowUnbounded: true, lowOffset: 0, uppUnbounded: true, uppOffset: 0 } + frame: { type: rows } aggregations: - { type: "longMin", name: "w0", fieldName: "l2" } - type: "naiveSort" @@ -31,7 +31,7 @@ expectedOperators: - type: "window" processor: type: "framedAgg" - frame: { peerType: "RANGE", lowUnbounded: true, lowOffset: 0, uppUnbounded: false, uppOffset: 0, orderBy: [{ column: l1, direction: ASC }] } + frame: { type: groups, upperOffset: 0, orderByColumns: [ l1 ] } aggregations: - { type: "longMin", name: "w1", fieldName: "l2" } diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e new file mode 100644 index 000000000000..26c251a35fb5 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Republic of Korea null #en.wikipedia 2 +Republic of Korea null #ja.wikipedia 3 +Republic of Korea null #ko.wikipedia 4 +Republic of Korea Seoul #ko.wikipedia 1 +Austria Vienna #de.wikipedia 1 +Austria Vienna #es.wikipedia 2 +Austria Vienna #tr.wikipedia 3 +Republic of Korea Jeonju #ko.wikipedia 4 +Republic of Korea Suwon-si #ko.wikipedia 1 +Austria Horsching #de.wikipedia 1 +Republic of Korea Seongnam-si #ko.wikipedia 1 +Republic of Korea Yongsan-dong #ko.wikipedia 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q new file mode 100644 index 000000000000..b10b52af389f --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_1.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,2,length(cityName)] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e new file mode 100644 index 000000000000..a1b116035c18 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Austria Horsching #de.wikipedia 2 +Austria Vienna #de.wikipedia 3 +Austria Vienna #es.wikipedia 4 +Austria Vienna #tr.wikipedia 5 +Republic of Korea null #en.wikipedia 6 +Republic of Korea null #ja.wikipedia 7 +Republic of Korea null #ko.wikipedia 8 +Republic of Korea Jeonju #ko.wikipedia 9 +Republic of Korea Seongnam-si #ko.wikipedia 10 +Republic of Korea Seoul #ko.wikipedia 11 +Republic of Korea Suwon-si #ko.wikipedia 12 +Republic of Korea Yongsan-dong #ko.wikipedia 13 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q new file mode 100644 index 000000000000..99245d7f9530 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_2.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,2,3] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e new file mode 100644 index 000000000000..ebd91f9f8933 --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.e @@ -0,0 +1,13 @@ +Austria null #de.wikipedia 1 +Austria Vienna #de.wikipedia 1 +Austria Vienna #es.wikipedia 2 +Austria Vienna #tr.wikipedia 3 +Austria Horsching #de.wikipedia 1 +Republic of Korea null #en.wikipedia 1 +Republic of Korea null #ja.wikipedia 2 +Republic of Korea null #ko.wikipedia 3 +Republic of Korea Seoul #ko.wikipedia 1 +Republic of Korea Jeonju #ko.wikipedia 1 +Republic of Korea Suwon-si #ko.wikipedia 1 +Republic of Korea Seongnam-si #ko.wikipedia 1 +Republic of Korea Yongsan-dong #ko.wikipedia 1 diff --git a/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q new file mode 100644 index 000000000000..9241f2ee94ef --- /dev/null +++ b/sql/src/test/resources/drill/window/queries/druid_queries/partition_by_array/wikipedia_query_3.q @@ -0,0 +1,6 @@ +select +countryName, cityName, channel, +row_number() over (partition by array[1,length(countryName),length(cityName)] order by countryName) as c +from wikipedia +where countryName in ('Austria', 'Republic of Korea') +group by countryName, cityName, channel diff --git a/web-console/package-lock.json b/web-console/package-lock.json index c7c7d271ce76..412f728d56d6 100644 --- a/web-console/package-lock.json +++ b/web-console/package-lock.json @@ -17,6 +17,7 @@ "@druid-toolkit/query": "^0.22.20", "@druid-toolkit/visuals-core": "^0.3.3", "@druid-toolkit/visuals-react": "^0.3.3", + "@fontsource/open-sans": "^5.0.28", "ace-builds": "~1.4.14", "axios": "^1.6.7", "chronoshift": "^0.10.0", @@ -31,7 +32,6 @@ "date-fns": "^2.28.0", "echarts": "^5.4.3", "file-saver": "^2.0.5", - "fontsource-open-sans": "^3.0.9", "hjson": "^3.2.2", "json-bigint-native": "^1.2.0", "lodash.debounce": "^4.0.8", @@ -1219,6 +1219,11 @@ "node": "^12.22.0 || ^14.17.0 || >=16.0.0" } }, + "node_modules/@fontsource/open-sans": { + "version": "5.0.28", + "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.0.28.tgz", + "integrity": "sha512-hBvJHY76pJT/JynGUB5EXWhnzjYfLdcMn655J5p1v9lTT9HdQSy+keq2KPVXO2Htlg998BBa3p6u/jlrZ6w0kg==" + }, "node_modules/@humanwhocodes/config-array": { "version": "0.11.8", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.8.tgz", @@ -7455,12 +7460,6 @@ } } }, - "node_modules/fontsource-open-sans": { - "version": "3.0.9", - "resolved": "https://registry.npmjs.org/fontsource-open-sans/-/fontsource-open-sans-3.0.9.tgz", - "integrity": "sha512-NEgZAgtpvz7iYZ/IlojOOy6hRLaLmxnxzEKYXnq0KU/X8lb9cD7d0XImykhfksK1/JZ6xyKQKjYq4PycHQZK9w==", - "deprecated": "Package relocated. Please install and migrate to @fontsource/open-sans." - }, "node_modules/for-each": { "version": "0.3.3", "resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz", @@ -19281,6 +19280,11 @@ "integrity": "sha512-lxJ9R5ygVm8ZWgYdUweoq5ownDlJ4upvoWmO4eLxBYHdMo+vZ/Rx0EN6MbKWDJOSUGrqJy2Gt+Dyv/VKml0fjg==", "dev": true }, + "@fontsource/open-sans": { + "version": "5.0.28", + "resolved": "https://registry.npmjs.org/@fontsource/open-sans/-/open-sans-5.0.28.tgz", + "integrity": "sha512-hBvJHY76pJT/JynGUB5EXWhnzjYfLdcMn655J5p1v9lTT9HdQSy+keq2KPVXO2Htlg998BBa3p6u/jlrZ6w0kg==" + }, "@humanwhocodes/config-array": { "version": "0.11.8", "resolved": "https://registry.npmjs.org/@humanwhocodes/config-array/-/config-array-0.11.8.tgz", @@ -24137,11 +24141,6 @@ "resolved": "https://registry.npmjs.org/follow-redirects/-/follow-redirects-1.15.6.tgz", "integrity": "sha512-wWN62YITEaOpSK584EZXJafH1AGpO8RVgElfkuXbTOrPX4fIfOyEpW/CsiNd8JdYrAoOvafRTOEnvsO++qCqFA==" }, - "fontsource-open-sans": { - "version": "3.0.9", - "resolved": "https://registry.npmjs.org/fontsource-open-sans/-/fontsource-open-sans-3.0.9.tgz", - "integrity": "sha512-NEgZAgtpvz7iYZ/IlojOOy6hRLaLmxnxzEKYXnq0KU/X8lb9cD7d0XImykhfksK1/JZ6xyKQKjYq4PycHQZK9w==" - }, "for-each": { "version": "0.3.3", "resolved": "https://registry.npmjs.org/for-each/-/for-each-0.3.3.tgz", diff --git a/web-console/package.json b/web-console/package.json index b45b1bdb8655..0c9370f88083 100644 --- a/web-console/package.json +++ b/web-console/package.json @@ -45,7 +45,7 @@ "prettify": "prettier --write '{src,e2e-tests}/**/*.{ts,tsx,scss}' './*.js'", "prettify-check": "prettier --check '{src,e2e-tests}/**/*.{ts,tsx,scss}' './*.js'", "generate-licenses-file": "license-checker --production --json --out licenses.json", - "check-licenses": "license-checker --production --onlyAllow 'Apache-1.1;Apache-2.0;BSD-2-Clause;BSD-3-Clause;0BSD;MIT;ISC;CC0-1.0' --summary", + "check-licenses": "license-checker --production --onlyAllow 'Apache-1.1;Apache-2.0;BSD-2-Clause;BSD-3-Clause;0BSD;MIT;ISC;CC0-1.0;OFL-1.1' --summary", "start": "webpack serve" }, "engines": { @@ -71,6 +71,7 @@ "@druid-toolkit/query": "^0.22.20", "@druid-toolkit/visuals-core": "^0.3.3", "@druid-toolkit/visuals-react": "^0.3.3", + "@fontsource/open-sans": "^5.0.28", "ace-builds": "~1.4.14", "axios": "^1.6.7", "chronoshift": "^0.10.0", @@ -85,7 +86,6 @@ "date-fns": "^2.28.0", "echarts": "^5.4.3", "file-saver": "^2.0.5", - "fontsource-open-sans": "^3.0.9", "hjson": "^3.2.2", "json-bigint-native": "^1.2.0", "lodash.debounce": "^4.0.8", diff --git a/web-console/script/licenses b/web-console/script/licenses index 1dc820b94c1b..90f1420282ae 100755 --- a/web-console/script/licenses +++ b/web-console/script/licenses @@ -139,6 +139,11 @@ checker.init( licenseExt = '0BSD'; break; + case 'OFL-1.1': + properLicenseName = 'SIL Open Font License 1.1'; + licenseExt = 'OFL'; + break; + default: throw new Error(`Unknown license '${licenses}' in ${p}`); } diff --git a/web-console/src/components/segment-timeline/segment-timeline.tsx b/web-console/src/components/segment-timeline/segment-timeline.tsx index 7527b8017637..0ea19555bfc9 100644 --- a/web-console/src/components/segment-timeline/segment-timeline.tsx +++ b/web-console/src/components/segment-timeline/segment-timeline.tsx @@ -24,7 +24,7 @@ import type { ItemPredicate, ItemRenderer } from '@blueprintjs/select'; import { Select } from '@blueprintjs/select'; import type { AxisScale } from 'd3-axis'; import { scaleLinear, scaleUtc } from 'd3-scale'; -import { enUS } from 'date-fns/locale'; +import enUS from 'date-fns/locale/en-US'; import React from 'react'; import type { Capabilities } from '../../helpers'; diff --git a/web-console/src/entry.scss b/web-console/src/entry.scss index dd2b13e49782..3893d7ccbeea 100644 --- a/web-console/src/entry.scss +++ b/web-console/src/entry.scss @@ -17,7 +17,8 @@ */ @import 'normalize.css/normalize'; -@import 'fontsource-open-sans/index.css'; +@import '@fontsource/open-sans/400.css'; // Normal +@import '@fontsource/open-sans/700.css'; // Bold @import './blueprint-overrides'; @import '@blueprintjs/core/src/blueprint'; @import '@blueprintjs/datetime/src/blueprint-datetime'; diff --git a/web-console/webpack.config.js b/web-console/webpack.config.js index 618a929aaea8..d6518623a478 100644 --- a/web-console/webpack.config.js +++ b/web-console/webpack.config.js @@ -24,6 +24,8 @@ const webpack = require('webpack'); const { version } = require('./package.json'); +const supportedLocales = ['en-US']; + function friendlyErrorFormatter(e) { return `${e.severity}: ${e.content} [TS${e.code}]\n at (${e.file}:${e.line}:${e.character})`; } @@ -47,6 +49,13 @@ module.exports = env => { 'global': {}, 'NODE_ENV': JSON.stringify(mode), }), + + // Prune date-fns locales to only those that are supported + // https://date-fns.org/v2.30.0/docs/webpack + new webpack.ContextReplacementPlugin( + /^date-fns[/\\]locale$/, + new RegExp(`\\.[/\\\\](${supportedLocales.join('|')})[/\\\\]index\\.js$`), + ), ]; return { @@ -137,7 +146,7 @@ module.exports = env => { // have access to them at this point. None of the components that use svg icons // via CSS are themselves being used by the web console, so we can safely omit the icons. // - // TODO: Re-evaluate after upgrading to Blueprint v5 + // TODO: Re-evaluate after upgrading to Blueprint v6 'svg-icon($_icon, $_path)': () => new SassString('transparent'), }, },