diff --git a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java index cb52e1b2a3b..8532574880a 100644 --- a/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java +++ b/extensions-contrib/materialized-view-selection/src/test/java/org/apache/druid/query/materializedview/MaterializedViewQueryTest.java @@ -53,6 +53,10 @@ public class MaterializedViewQueryTest private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private DataSourceOptimizer optimizer; + static { + NullHandling.initializeForTests(); + } + @Before public void setUp() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 6b6f8ff356e..ac9ca855a63 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -1798,7 +1798,7 @@ public class MSQSelectTest extends MSQTestBase .setExpectedValidationErrorMatcher( new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") .expectMessageIs( - "Query planning failed for unknown reason, our best guess is this " + "Query could not be planned. A possible reason is " + "[LATEST and EARLIEST aggregators implicitly depend on the __time column, " + "but the table queried doesn't contain a __time column. " + "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]" diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index c537ef08fc5..357ec61c6cd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -696,7 +696,7 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand .ofCategory(DruidException.Category.INVALID_INPUT) .build( exception, - "Query planning failed for unknown reason, our best guess is this [%s]", + "Query could not be planned. A possible reason is [%s]", errorMessage ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java index ef19c559a8a..25e06ca8825 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java @@ -42,22 +42,26 @@ import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; import org.apache.calcite.sql.SqlKind; import org.apache.calcite.util.ImmutableBitSet; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.query.DataSource; +import org.apache.druid.query.FilteredDataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.UnnestDataSource; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; import org.apache.druid.query.aggregation.LongMinAggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; import org.apache.druid.query.aggregation.SimpleLongAggregatorFactory; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; @@ -773,6 +777,17 @@ public class DruidQuery return VirtualColumns.create(columns); } + public static List getAllFiltersUnderDataSource(DataSource d, List dimFilterList) + { + if (d instanceof FilteredDataSource) { + dimFilterList.add(((FilteredDataSource) d).getFilter()); + } + for (DataSource ds : d.getChildren()) { + dimFilterList.addAll(getAllFiltersUnderDataSource(ds, dimFilterList)); + } + return dimFilterList; + } + /** * Returns a pair of DataSource and Filtration object created on the query filter. In case the, data source is * a join datasource, the datasource may be altered and left filter of join datasource may @@ -786,8 +801,44 @@ public class DruidQuery JoinableFactoryWrapper joinableFactoryWrapper ) { - if (!canUseIntervalFiltering(dataSource)) { + if (dataSource instanceof UnnestDataSource) { + // UnnestDataSource can have another unnest data source + // join datasource, filtered data source, etc as base + Pair pair = getFiltration( + ((UnnestDataSource) dataSource).getBase(), + filter, + virtualColumnRegistry, + joinableFactoryWrapper + ); + return Pair.of(dataSource, pair.rhs); + } else if (!canUseIntervalFiltering(dataSource)) { return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), false)); + } else if (dataSource instanceof FilteredDataSource) { + // A filteredDS is created only inside the rel for Unnest, ensuring it only grabs the outermost filter + // and, if possible, pushes it down inside the data source. + // So a chain of Filter->Unnest->Filter is typically impossible when the query is done through SQL. + // Also, Calcite has filter reduction rules that push filters deep into base data sources for better query planning. + // A base table with a chain of filters is synonymous with a filteredDS. + // We recursively find all filters under a filteredDS and then + // 1) creating a filtration from the filteredDS's filters and + // 2) Updating the interval of the outer filter with the intervals in step 1, and you'll see these 2 calls in the code + List dimFilterList = getAllFiltersUnderDataSource(dataSource, new ArrayList<>()); + final FilteredDataSource filteredDataSource = (FilteredDataSource) dataSource; + // Defensive check as in the base of a filter cannot be another filter + final DataSource baseOfFilterDataSource = filteredDataSource.getBase(); + if (baseOfFilterDataSource instanceof FilteredDataSource) { + throw DruidException.defensive("Cannot create a filteredDataSource using another filteredDataSource as a base"); + } + final boolean useIntervalFiltering = canUseIntervalFiltering(filteredDataSource); + final Filtration baseFiltration = toFiltration( + new AndDimFilter(dimFilterList), + virtualColumnRegistry.getFullRowSignature(), + useIntervalFiltering + ); + // Adds the intervals from the filter of filtered data source to query filtration + final Filtration queryFiltration = Filtration.create(filter, baseFiltration.getIntervals()) + .optimize(virtualColumnRegistry.getFullRowSignature()); + return Pair.of(filteredDataSource, queryFiltration); } else if (dataSource instanceof JoinDataSource && ((JoinDataSource) dataSource).getLeftFilter() != null) { final JoinDataSource joinDataSource = (JoinDataSource) dataSource; @@ -809,7 +860,6 @@ public class DruidQuery leftFiltration.getDimFilter(), joinableFactoryWrapper ); - return Pair.of(newDataSource, queryFiltration); } else { return Pair.of(dataSource, toFiltration(filter, virtualColumnRegistry.getFullRowSignature(), true)); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 84fd4217c75..16f64270172 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -785,7 +785,7 @@ public class BaseCalciteQueryTest extends CalciteTestBase new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") .expectMessageIs( StringUtils.format( - "Query planning failed for unknown reason, our best guess is this [%s]", + "Query could not be planned. A possible reason is [%s]", expectedError ) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index ae4437faf69..8fc531d35ab 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -24,12 +24,14 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.HumanReadableBytes; +import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; import org.apache.druid.query.FilteredDataSource; import org.apache.druid.query.InlineDataSource; +import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; @@ -4869,4 +4871,362 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest ) ); } + + @Test + public void testUnnestWithTimeFilterOnly() + { + testQuery( + "select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c)" + + " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ), + expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"d"} + ) + ); + } + + @Test + public void testUnnestWithTimeFilterAndAnotherFilter() + { + testQuery( + "select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c) " + + " where m1=2 and __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + and( + useDefault ? equality("m1", 2, ColumnType.FLOAT) : + equality("m1", 2.0, ColumnType.FLOAT), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ) + ), + expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"} + ) + ); + } + + @Test + public void testUnnestWithTimeFilterOrAnotherFilter() + { + testQuery( + "select c from foo, unnest(MV_TO_ARRAY(dim3)) as u(c) " + + " where m1=2 or __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + or( + useDefault ? equality("m1", 2, ColumnType.FLOAT) : + equality("m1", 2.0, ColumnType.FLOAT), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ) + ), + expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"d"} + ) + ); + } + + @Test + public void testUnnestWithTimeFilterOnlyNested() + { + testQuery( + "select c from foo CROSS JOIN UNNEST(ARRAY[m1,m2]) as un(d) CROSS JOIN unnest(MV_TO_ARRAY(dim3)) as u(c)" + + " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ), + expressionVirtualColumn("j0.unnest", "array(\"m1\",\"m2\")", ColumnType.FLOAT_ARRAY), + null + ), + expressionVirtualColumn("_j0.unnest", "\"dim3\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("_j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"d"}, + new Object[]{"d"} + ) + ); + } + + @Test + public void testUnnestWithTimeFilterOnlyNestedAndNestedAgain() + { + testQuery( + "select c from foo CROSS JOIN UNNEST(ARRAY[m1,m2]) as un(d) CROSS JOIN UNNEST(ARRAY[dim1,dim2]) as ud(a) " + + " CROSS JOIN unnest(MV_TO_ARRAY(dim3)) as u(c)" + + " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + UnnestDataSource.create( + UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ), + expressionVirtualColumn("j0.unnest", "array(\"m1\",\"m2\")", ColumnType.FLOAT_ARRAY), + null + ), + expressionVirtualColumn("_j0.unnest", "array(\"dim1\",\"dim2\")", ColumnType.STRING_ARRAY), + null + ), + expressionVirtualColumn("__j0.unnest", "\"dim3\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("__j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"b"}, + new Object[]{"c"}, + new Object[]{"d"}, + new Object[]{"d"}, + new Object[]{"d"}, + new Object[]{"d"} + ) + ); + } + + @Test + public void testUnnestWithTimeFilterInsideSubquery() + { + testQuery( + "select d3 from (select * from foo, UNNEST(MV_TO_ARRAY(dim3)) as u(d3)" + + " where __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00' LIMIT 2) \n" + + " where m1 IN (1,2)", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + new QueryDataSource( + newScanQueryBuilder() + .dataSource( + UnnestDataSource.create( + FilteredDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ), + expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), + null + ) + ) + .intervals(querySegmentSpec(Intervals.of( + "2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .columns("j0.unnest", "m1") + .limit(2) + .context(QUERY_CONTEXT_UNNEST) + .build() + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters( + NullHandling.sqlCompatible() ? + or( + equality("m1", 1.0f, ColumnType.FLOAT), + equality("m1", 2.0f, ColumnType.FLOAT) + ) : + new InDimFilter("m1", ImmutableList.of("1", "2"), null) + ) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("j0.unnest")) + .build() + ), + ImmutableList.of( + new Object[]{"b"}, + new Object[]{"c"} + ) + ); + } + + @Test + public void testUnnestWithFilterAndUnnestNestedBackToBack() + { + testQuery( + "SELECT m1, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n" + + " ( SELECT * FROM \n" + + " ( SELECT * FROM foo, UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest1) ), \n" + + " UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest2) \n" + + " ), UNNEST(MV_TO_ARRAY(dim3)) as ut(dim3_unnest3) " + + " WHERE m1=2 AND (dim3_unnest1='a' OR dim3_unnest2='b') AND dim3_unnest3='c' " + + " AND __time >= TIMESTAMP '2000-01-02 00:00:00' and __time <= TIMESTAMP '2000-01-03 00:10:00'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource( + UnnestDataSource.create( + FilteredDataSource.create( + UnnestDataSource.create( + FilteredDataSource.create( + UnnestDataSource.create( + new TableDataSource(CalciteTests.DATASOURCE1), + expressionVirtualColumn( + "j0.unnest", + "\"dim3\"", + ColumnType.STRING + ), + null + ), + NullHandling.sqlCompatible() ? + and( + equality("m1", 2.0f, ColumnType.FLOAT), + range("__time", ColumnType.LONG, 946771200000L, 946858200000L, false, false) + ) : + and( + selector("m1", "2", null), + bound( + "__time", + "946771200000", + "946858200000", + false, + false, + null, + StringComparators.NUMERIC + ) + ) + ), + expressionVirtualColumn( + "_j0.unnest", + "\"dim3\"", + ColumnType.STRING + ), + null + ), + or( + equality("j0.unnest", "a", ColumnType.STRING), + equality("_j0.unnest", "b", ColumnType.STRING) + ) + ), + expressionVirtualColumn( + "__j0.unnest", + "\"dim3\"", + ColumnType.STRING + ), + equality("__j0.unnest", "c", ColumnType.STRING) + ) + ) + .intervals(querySegmentSpec(Intervals.of("2000-01-02T00:00:00.000Z/2000-01-03T00:10:00.001Z"))) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .virtualColumns(expressionVirtualColumn("v0", "2.0", ColumnType.FLOAT)) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("__j0.unnest", "_j0.unnest", "j0.unnest", "v0")) + .build() + ), + ImmutableList.of( + new Object[]{2.0f, "b", "b", "c"}, + new Object[]{2.0f, "c", "b", "c"} + ) + ); + } + + @Test + public void testUnnestWithLookup() + { + testQuery( + "SELECT * FROM lookup.lookyloo, unnest(mv_to_array(v)) as u(d) where k='a'", + QUERY_CONTEXT_UNNEST, + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(UnnestDataSource.create( + FilteredDataSource.create( + new LookupDataSource("lookyloo"), + equality("k", "a", ColumnType.STRING) + ), + expressionVirtualColumn("j0.unnest", "\"v\"", ColumnType.STRING), + null + )) + .intervals(querySegmentSpec(Filtration.eternity())) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .virtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) + .legacy(false) + .context(QUERY_CONTEXT_UNNEST) + .columns(ImmutableList.of("j0.unnest", "v", "v0")) + .build() + ), + ImmutableList.of( + new Object[]{"a", "xa", "xa"} + ) + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index fa159a0132e..bb1660f856a 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -1504,7 +1504,7 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest e, new DruidExceptionMatcher(DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, "general") .expectMessageIs( - "Query planning failed for unknown reason, our best guess is this " + "Query could not be planned. A possible reason is " + "[LATEST and EARLIEST aggregators implicitly depend on the __time column, " + "but the table queried doesn't contain a __time column. " + "Please use LATEST_BY or EARLIEST_BY and specify the column explicitly.]" diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 98f1f7b1cab..6e38c26e4f3 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -1390,7 +1390,7 @@ public class SqlResourceTest extends CalciteTestBase "general", DruidException.Persona.ADMIN, DruidException.Category.INVALID_INPUT, - "Query planning failed for unknown reason, our best guess is this " + "Query could not be planned. A possible reason is " + "[SQL query requires order by non-time column [[dim1 ASC]], which is not supported.]" ); checkSqlRequestLog(false);