Skip to content

Commit

Permalink
Window Functions : Remove enable windowing flag (apache#17087) (apach…
Browse files Browse the repository at this point in the history
…e#17128)

(cherry picked from commit 67d361c)
  • Loading branch information
sreemanamala authored Sep 24, 2024
1 parent 6c0ca77 commit b7cc0bb
Show file tree
Hide file tree
Showing 17 changed files with 24 additions and 68 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -335,7 +334,6 @@ public void tearDown() throws Exception
public void querySql(String sql, Blackhole blackhole)
{
final Map<String, Object> context = ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.MAX_SUBQUERY_BYTES_KEY, "auto"
);
try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, context)) {
Expand Down
1 change: 0 additions & 1 deletion docs/api-reference/sql-api.md
Original file line number Diff line number Diff line change
Expand Up @@ -933,7 +933,6 @@ Host: http://ROUTER_IP:ROUTER_PORT
"context": {
"__resultFormat": "array",
"__user": "allowAll",
"enableWindowing": true,
"executionMode": "async",
"finalize": true,
"maxNumTasks": 2,
Expand Down
5 changes: 1 addition & 4 deletions docs/querying/sql-window-functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -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.

:::

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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)"
Expand All @@ -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"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand All @@ -50,7 +49,6 @@
public class MSQDrillWindowQueryTest extends DrillWindowQueryTest
{
private final Map<String, Object> 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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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<String, Object> DURABLE_STORAGE_MSQ_CONTEXT =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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},
Expand All @@ -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()
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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()}.
*/
Expand Down Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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
{
Expand Down Expand Up @@ -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();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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()
{
Expand All @@ -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());

Expand All @@ -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])"));
Expand All @@ -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])"));
Expand All @@ -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])"));
Expand All @@ -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()
);
Expand All @@ -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()
);
Expand All @@ -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());

Expand Down Expand Up @@ -15919,7 +15901,6 @@ public void testWindowingWithScanAndSort()

testBuilder()
.sql(sql)
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
.expectedQuery(
WindowOperatorQueryBuilder.builder()
.setDataSource(
Expand Down Expand Up @@ -16006,7 +15987,6 @@ public void testWindowingWithOrderBy()
)
.queryContext(
ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.ENABLE_DEBUG, true
)
)
Expand Down Expand Up @@ -16106,7 +16086,6 @@ public void testWindowingOverJoin()
)
.queryContext(
ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.ENABLE_DEBUG, true
)
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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")
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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},
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -70,7 +69,6 @@ public class CalciteWindowQueryTest extends BaseCalciteQueryTest
private static final ObjectMapper YAML_JACKSON = new DefaultObjectMapper(new YAMLFactory(), "tests");

private static final Map<String, Object> DEFAULT_QUERY_CONTEXT = ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.ENABLE_DEBUG, true,
QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false
);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -413,7 +412,6 @@ public void windowQueryTest()
protected Map<String, Object> getQueryContext()
{
return ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
PlannerCaptureHook.NEED_CAPTURE_HOOK, true,
QueryContexts.ENABLE_DEBUG, true
);
Expand Down
Original file line number Diff line number Diff line change
@@ -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
Expand Down
Original file line number Diff line number Diff line change
@@ -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
Expand Down

0 comments on commit b7cc0bb

Please sign in to comment.