diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java index 7fa2d38d6868..b62dcae37166 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java @@ -78,7 +78,6 @@ import org.apache.druid.sql.calcite.planner.DruidOperatorTable; import org.apache.druid.sql.calcite.planner.DruidPlanner; import org.apache.druid.sql.calcite.planner.PlannerConfig; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.planner.PlannerResult; import org.apache.druid.sql.calcite.run.SqlEngine; @@ -335,7 +334,6 @@ public void tearDown() throws Exception public void querySql(String sql, Blackhole blackhole) { final Map context = ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, QueryContexts.MAX_SUBQUERY_BYTES_KEY, "auto" ); try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, context)) { diff --git a/docs/api-reference/sql-api.md b/docs/api-reference/sql-api.md index 3a6234256188..f6a624c8ce23 100644 --- a/docs/api-reference/sql-api.md +++ b/docs/api-reference/sql-api.md @@ -933,7 +933,6 @@ Host: http://ROUTER_IP:ROUTER_PORT "context": { "__resultFormat": "array", "__user": "allowAll", - "enableWindowing": true, "executionMode": "async", "finalize": true, "maxNumTasks": 2, diff --git a/docs/querying/sql-window-functions.md b/docs/querying/sql-window-functions.md index 7c2c3aef53e9..d4293a852e70 100644 --- a/docs/querying/sql-window-functions.md +++ b/docs/querying/sql-window-functions.md @@ -30,10 +30,7 @@ This document describes the SQL language. Window functions are an [experimental](../development/experimental.md) feature. Development and testing are still at early stage. Feel free to try window functions and provide your feedback. -Windows functions are not currently supported by multi-stage-query engine so you cannot use them in SQL-based ingestion. - - -Set the context parameter `enableWindowing: true` to use window functions. +Windows functions are not currently supported by multi-stage-query engine so you cannot use them in SQL-based ingestion. ::: diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index edb7dc5a11f4..65451a3323e1 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -80,7 +80,6 @@ import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.apache.druid.sql.calcite.TempDirProducer; import org.apache.druid.sql.calcite.filtration.Filtration; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; @@ -1213,7 +1212,6 @@ public void testHllEstimateAsVirtualColumnWithTopN() public void testHllWithOrderedWindowing() { testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql( "SELECT dim1,coalesce(cast(l1 as integer),-999)," + " HLL_SKETCH_ESTIMATE( DS_HLL(dim1) OVER ( ORDER BY l1 ), true)" @@ -1238,7 +1236,6 @@ public void testResultCacheWithWindowing() { for (int i = 0; i < 2; i++) { testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql( "SELECT " + " TIME_FLOOR(__time, 'P1D') as dayLvl,\n" diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java index 1cc756d7c578..3e8e2796eeac 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQDrillWindowQueryTest.java @@ -39,7 +39,6 @@ import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.apache.druid.sql.calcite.TempDirProducer; import org.apache.druid.sql.calcite.planner.PlannerCaptureHook; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.run.SqlEngine; import org.junit.jupiter.api.Test; @@ -50,7 +49,6 @@ public class MSQDrillWindowQueryTest extends DrillWindowQueryTest { private final Map queryContext = new HashMap<>(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, PlannerCaptureHook.NEED_CAPTURE_HOOK, true, QueryContexts.ENABLE_DEBUG, true, MultiStageQueryContext.CTX_MAX_NUM_TASKS, 5 diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java index df9a00925cd4..bb8fbf3029ee 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/MSQTestBase.java @@ -174,7 +174,6 @@ import org.apache.druid.sql.calcite.planner.CalciteRulesManager; import org.apache.druid.sql.calcite.planner.CatalogResolver; import org.apache.druid.sql.calcite.planner.PlannerConfig; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.planner.PlannerFactory; import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.run.SqlEngine; @@ -270,7 +269,6 @@ public class MSQTestBase extends BaseCalciteQueryTest .put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2) .put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0) .put(MSQTaskQueryMaker.USER_KEY, "allowAll") - .put(PlannerContext.CTX_ENABLE_WINDOW_FNS, true) .build(); public static final Map DURABLE_STORAGE_MSQ_CONTEXT = diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index 9c62e2969ca6..0fcc17eee205 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -20,7 +20,6 @@ package org.apache.druid.query.aggregation.variance.sql; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Injector; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; @@ -34,7 +33,6 @@ 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.QueryContexts; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; @@ -64,7 +62,6 @@ import org.apache.druid.sql.calcite.SqlTestFrameworkConfig; import org.apache.druid.sql.calcite.TempDirProducer; import org.apache.druid.sql.calcite.filtration.Filtration; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.util.CalciteTests; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; @@ -715,10 +712,6 @@ public void testOverWindow() "select dim4, dim5, mod(m1, 3), var_pop(mod(m1, 3)) over (partition by dim4 order by dim5) c\n" + "from numfoo\n" + "group by dim4, dim5, mod(m1, 3)") - .queryContext(ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, - QueryContexts.ENABLE_DEBUG, true - )) .expectedResults(ImmutableList.of( new Object[]{"a", "aa", 1.0D, 0.0D}, new Object[]{"a", "ab", 2.0D, 0.25D}, @@ -738,7 +731,6 @@ public void testStddevNotSupportedOverWindow() DruidException e = assertThrows( DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT stddev(m1) OVER () from numfoo") .run() ); 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 251a5d7925cb..797b5397f1d7 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 @@ -778,8 +778,8 @@ public void validateCall(SqlCall call, SqlValidatorScope scope) if (!plannerContext.featureAvailable(EngineFeature.WINDOW_FUNCTIONS)) { throw buildCalciteContextException( StringUtils.format( - "The query contains window functions; To run these window functions, specify [%s] in query context.", - PlannerContext.CTX_ENABLE_WINDOW_FNS + "The query contains window functions; They are not supported on engine[%s].", + plannerContext.getEngine().name() ), call ); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index 79f9d79e8499..2dbf3960c3ae 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -90,11 +90,6 @@ public class PlannerContext */ public static final String CTX_SQL_OUTER_LIMIT = "sqlOuterLimit"; - /** - * Key to enable window functions. - */ - public static final String CTX_ENABLE_WINDOW_FNS = "enableWindowing"; - /** * Context key for {@link PlannerContext#isUseBoundsAndSelectors()}. */ @@ -634,15 +629,10 @@ public SqlEngine getEngine() * Checks if the current {@link SqlEngine} supports a particular feature. * * When executing a specific query, use this method instead of {@link SqlEngine#featureAvailable(EngineFeature)} - * because it also verifies feature flags such as {@link #CTX_ENABLE_WINDOW_FNS}. + * because it also verifies feature flags. */ public boolean featureAvailable(final EngineFeature feature) { - if (feature == EngineFeature.WINDOW_FUNCTIONS && - !QueryContexts.getAsBoolean(CTX_ENABLE_WINDOW_FNS, queryContext.get(CTX_ENABLE_WINDOW_FNS), false)) { - // Short-circuit: feature requires context flag. - return false; - } if (feature == EngineFeature.TIME_BOUNDARY_QUERY && !queryContext().isTimeBoundaryPlanningEnabled()) { // Short-circuit: feature requires context flag. return false; diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java index 813722e6990e..3083c6098e35 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCatalogIngestionDmlTest.java @@ -50,6 +50,7 @@ import org.apache.druid.sql.calcite.external.Externals; import org.apache.druid.sql.calcite.filtration.Filtration; import org.apache.druid.sql.calcite.planner.CatalogResolver; +import org.apache.druid.sql.calcite.run.EngineFeature; import org.apache.druid.sql.calcite.table.DatasourceTable; import org.apache.druid.sql.calcite.table.DruidTable; import org.apache.druid.sql.calcite.util.CalciteTests; @@ -59,6 +60,8 @@ import java.util.HashMap; import java.util.Map; +import static org.junit.jupiter.api.Assumptions.assumeFalse; + @SqlTestFrameworkConfig.ComponentSupplier(CatalogIngestionDmlComponentSupplier.class) public abstract class CalciteCatalogIngestionDmlTest extends CalciteIngestionDmlTest { @@ -1289,4 +1292,19 @@ public void testGroupByInsertIntoExistingWithIncompatibleTypeAssignmentAndValida ) .verify(); } + + @Test + public void testWindowingErrorWithEngineFeatureOff() + { + assumeFalse(queryFramework().engine().featureAvailable(EngineFeature.WINDOW_FUNCTIONS)); + testIngestionQuery() + .sql(StringUtils.format(dmlPrefixPattern, "foo") + "\n" + + "SELECT dim1, ROW_NUMBER() OVER () from foo\n" + + "PARTITIONED BY ALL TIME") + .expectValidationError( + DruidException.class, + "The query contains window functions; They are not supported on engine[ingestion-test]. (line [2], column [14])" + ) + .verify(); + } } 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 732681de238a..f4c1f1a49b04 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 @@ -15607,17 +15607,6 @@ public void testLatestByOnStringColumnWithoutMaxBytesSpecified() )); } - @Test - public void testWindowingErrorWithoutFeatureFlag() - { - DruidException e = assertThrows(DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, false)) - .sql("SELECT dim1,ROW_NUMBER() OVER () from druid.foo") - .run()); - - 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() { @@ -15636,7 +15625,6 @@ public void testDistinctSumNotSupportedWithApproximation() public void testUnSupportedNullsFirst() { DruidException e = assertThrows(DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT dim1,ROW_NUMBER() OVER (ORDER BY dim1 DESC NULLS FIRST) from druid.foo") .run()); @@ -15647,7 +15635,6 @@ public void testUnSupportedNullsFirst() public void testUnSupportedNullsLast() { DruidException e = assertThrows(DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT dim1,ROW_NUMBER() OVER (ORDER BY dim1 NULLS LAST) from druid.foo") .run()); assertThat(e, invalidSqlIs("ASCENDING ordering with NULLS LAST is not supported! (line [1], column [41])")); @@ -15659,7 +15646,6 @@ public void testUnSupportedRangeBounds() assumeFeatureAvailable(EngineFeature.WINDOW_FUNCTIONS); DruidException e = assertThrows(DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT dim1,ROW_NUMBER() OVER (ORDER BY dim1 RANGE BETWEEN 3 PRECEDING AND 2 FOLLOWING) from druid.foo") .run()); assertThat(e, invalidSqlIs("Order By with RANGE clause currently supports only UNBOUNDED or CURRENT ROW. Use ROWS clause instead. (line [1], column [31])")); @@ -15671,7 +15657,6 @@ public void testUnSupportedWindowBoundExpressions() assumeFeatureAvailable(EngineFeature.WINDOW_FUNCTIONS); DruidException e = assertThrows(DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT dim1,ROW_NUMBER() OVER (ORDER BY dim1 ROWS BETWEEN dim1 PRECEDING AND dim1 FOLLOWING) from druid.foo") .run()); assertThat(e, invalidSqlIs("Window frames with expression based lower/upper bounds are not supported. (line [1], column [31])")); @@ -15685,7 +15670,6 @@ public void testNtileNotSupportedWithFrame() DruidException e = assertThrows( DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT ntile(4) OVER (ORDER BY dim1 ROWS BETWEEN 1 FOLLOWING AND CURRENT ROW) from druid.foo") .run() ); @@ -15701,7 +15685,6 @@ public void testDistinctNotSupportedWithWindow() DruidException e = assertThrows( DruidException.class, () -> testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("SELECT count(distinct dim1) OVER () from druid.foo") .run() ); @@ -15719,7 +15702,6 @@ public void testUnSupportedAggInSelectWindow() ); 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()); @@ -15919,7 +15901,6 @@ public void testWindowingWithScanAndSort() testBuilder() .sql(sql) - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .expectedQuery( WindowOperatorQueryBuilder.builder() .setDataSource( @@ -16006,7 +15987,6 @@ public void testWindowingWithOrderBy() ) .queryContext( ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, QueryContexts.ENABLE_DEBUG, true ) ) @@ -16106,7 +16086,6 @@ public void testWindowingOverJoin() ) .queryContext( ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, QueryContexts.ENABLE_DEBUG, true ) ) 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 973b1eb4df1e..02b49af58807 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 @@ -2194,7 +2194,6 @@ public void testRejectHavingWithWindowExpression() testQueryThrows( "SELECT cityName,sum(1) OVER () as w FROM wikipedia group by cityName HAVING w > 10", - ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true), DruidException.class, invalidSqlContains("Window functions are not allowed in HAVING") ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSysQueryTest.java index f03b13d8e262..9f1ad49481a7 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSysQueryTest.java @@ -20,10 +20,8 @@ package org.apache.druid.sql.calcite; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.sql.calcite.NotYetSupported.Modes; import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -54,7 +52,6 @@ public void testTasksSumOver() msqIncompatible(); testBuilder() - .queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)) .sql("select datasource, sum(duration) over () from sys.tasks group by datasource") .expectedResults(ImmutableList.of( new Object[]{"foo", 11L}, 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 cd6aa514675f..fc86268ed591 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 @@ -37,7 +37,6 @@ import org.apache.druid.sql.calcite.CalciteWindowQueryTest.WindowQueryTestInputClass.TestType; import org.apache.druid.sql.calcite.QueryTestRunner.QueryResults; import org.apache.druid.sql.calcite.QueryVerification.QueryResultsVerifier; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.junit.Assert; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -70,7 +69,6 @@ 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, QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false ); 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 0cc30c49c34d..96e175957177 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 @@ -43,7 +43,6 @@ import org.apache.druid.sql.calcite.NotYetSupported.NotYetSupportedProcessor; import org.apache.druid.sql.calcite.QueryTestRunner.QueryResults; import org.apache.druid.sql.calcite.planner.PlannerCaptureHook; -import org.apache.druid.sql.calcite.planner.PlannerContext; import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier; import org.apache.druid.sql.calcite.util.TestDataBuilder; import org.joda.time.DateTime; @@ -413,7 +412,6 @@ public void windowQueryTest() protected Map getQueryContext() { return ImmutableMap.of( - PlannerContext.CTX_ENABLE_WINDOW_FNS, true, PlannerCaptureHook.NEED_CAPTURE_HOOK, true, QueryContexts.ENABLE_DEBUG, true ); 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 8b2ca6626211..17ca58cf2a2f 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 @@ -1,8 +1,7 @@ -# testWindowingWithScanAndSort@NullHandling=default case-crc:bb1d6e53 +# testWindowingWithScanAndSort@NullHandling=default case-crc:45108080 # quidem testcase reason: SLIGHTLY_WORSE_PLAN !set debug true !set defaultTimeout 300000 -!set enableWindowing true !set maxScatterGatherBytes 9223372036854775807 !set plannerStrategy DECOUPLED !set sqlCurrentTimestamp 2000-01-01T00:00:00Z 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 ae521d6b098a..b18827be5b03 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 @@ -1,8 +1,7 @@ -# testWindowingWithScanAndSort@NullHandling=sql case-crc:bb1d6e53 +# testWindowingWithScanAndSort@NullHandling=sql case-crc:45108080 # quidem testcase reason: SLIGHTLY_WORSE_PLAN !set debug true !set defaultTimeout 300000 -!set enableWindowing true !set maxScatterGatherBytes 9223372036854775807 !set plannerStrategy DECOUPLED !set sqlCurrentTimestamp 2000-01-01T00:00:00Z